You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@beam.apache.org by ie...@apache.org on 2017/08/23 17:09:09 UTC

[01/55] [abbrv] beam git commit: NexMark

Repository: beam
Updated Branches:
  refs/heads/master f0ce31b9d -> 64ff21f35


http://git-wip-us.apache.org/repos/asf/beam/blob/1f08970a/integration/java/src/main/java/org/apache/beam/integration/nexmark/SellerPrice.java
----------------------------------------------------------------------
diff --git a/integration/java/src/main/java/org/apache/beam/integration/nexmark/SellerPrice.java b/integration/java/src/main/java/org/apache/beam/integration/nexmark/SellerPrice.java
new file mode 100644
index 0000000..4324b99
--- /dev/null
+++ b/integration/java/src/main/java/org/apache/beam/integration/nexmark/SellerPrice.java
@@ -0,0 +1,91 @@
+/*
+ * 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.beam.integration.nexmark;
+
+import org.apache.beam.sdk.coders.AtomicCoder;
+import org.apache.beam.sdk.coders.Coder;
+import org.apache.beam.sdk.coders.CoderException;
+import org.apache.beam.sdk.coders.VarLongCoder;
+
+import com.fasterxml.jackson.annotation.JsonProperty;
+import com.fasterxml.jackson.core.JsonProcessingException;
+
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
+import java.io.Serializable;
+
+/**
+ * Result of {@link Query6}.
+ */
+public class SellerPrice implements KnownSize, Serializable {
+  private static final Coder<Long> LONG_CODER = VarLongCoder.of();
+
+  public static final Coder<SellerPrice> CODER = new AtomicCoder<SellerPrice>() {
+    @Override
+    public void encode(SellerPrice value, OutputStream outStream,
+        Coder.Context context)
+        throws CoderException, IOException {
+      LONG_CODER.encode(value.seller, outStream, Context.NESTED);
+      LONG_CODER.encode(value.price, outStream, Context.NESTED);
+    }
+
+    @Override
+    public SellerPrice decode(
+        InputStream inStream, Coder.Context context)
+        throws CoderException, IOException {
+      long seller = LONG_CODER.decode(inStream, Context.NESTED);
+      long price = LONG_CODER.decode(inStream, Context.NESTED);
+      return new SellerPrice(seller, price);
+    }
+  };
+
+  @JsonProperty
+  public final long seller;
+
+  /** Price in cents. */
+  @JsonProperty
+  public final long price;
+
+  // For Avro only.
+  @SuppressWarnings("unused")
+  private SellerPrice() {
+    seller = 0;
+    price = 0;
+  }
+
+  public SellerPrice(long seller, long price) {
+    this.seller = seller;
+    this.price = price;
+  }
+
+  @Override
+  public long sizeInBytes() {
+    return 8 + 8;
+  }
+
+  @Override
+  public String toString() {
+    try {
+      return NexmarkUtils.MAPPER.writeValueAsString(this);
+    } catch (JsonProcessingException e) {
+      throw new RuntimeException(e);
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/1f08970a/integration/java/src/main/java/org/apache/beam/integration/nexmark/UnboundedEventSource.java
----------------------------------------------------------------------
diff --git a/integration/java/src/main/java/org/apache/beam/integration/nexmark/UnboundedEventSource.java b/integration/java/src/main/java/org/apache/beam/integration/nexmark/UnboundedEventSource.java
new file mode 100644
index 0000000..2898251
--- /dev/null
+++ b/integration/java/src/main/java/org/apache/beam/integration/nexmark/UnboundedEventSource.java
@@ -0,0 +1,329 @@
+/*
+ * 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.beam.integration.nexmark;
+
+import org.apache.beam.sdk.coders.Coder;
+import org.apache.beam.sdk.io.UnboundedSource;
+import org.apache.beam.sdk.options.PipelineOptions;
+import org.apache.beam.sdk.transforms.windowing.BoundedWindow;
+import org.apache.beam.sdk.values.TimestampedValue;
+
+import org.joda.time.Duration;
+import org.joda.time.Instant;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.NoSuchElementException;
+import java.util.PriorityQueue;
+import java.util.Queue;
+import java.util.concurrent.ThreadLocalRandom;
+
+import javax.annotation.Nullable;
+
+/**
+ * A custom, unbounded source of event records.
+ *
+ * <p>If {@code isRateLimited} is true, events become available for return from the reader such
+ * that the overall rate respect the {@code interEventDelayUs} period if possible. Otherwise,
+ * events are returned every time the system asks for one.
+ */
+class UnboundedEventSource extends UnboundedSource<Event, Generator.Checkpoint> {
+  private static final Duration BACKLOG_PERIOD = Duration.standardSeconds(30);
+
+  /** Configuration for generator to use when reading synthetic events. May be split. */
+  private final GeneratorConfig config;
+
+  /** How many unbounded sources to create. */
+  private final int numEventGenerators;
+
+  /** How many seconds to hold back the watermark. */
+  private final long watermarkHoldbackSec;
+
+  /** Are we rate limiting the events? */
+  private final boolean isRateLimited;
+
+  public UnboundedEventSource(GeneratorConfig config, int numEventGenerators,
+      long watermarkHoldbackSec, boolean isRateLimited) {
+    this.config = config;
+    this.numEventGenerators = numEventGenerators;
+    this.watermarkHoldbackSec = watermarkHoldbackSec;
+    this.isRateLimited = isRateLimited;
+  }
+
+  /** A reader to pull events from the generator. */
+  private class EventReader extends UnboundedReader<Event> {
+    /** Generator we are reading from. */
+    private final Generator generator;
+
+    /**
+     * Current watermark (ms since epoch). Initially set to beginning of time.
+     * Then updated to be the time of the next generated event.
+     * Then, once all events have been generated, set to the end of time.
+     */
+    private long watermark;
+
+    /**
+     * Current backlog (ms), as delay between timestamp of last returned event and the timestamp
+     * we should be up to according to wall-clock time. Used only for logging.
+     */
+    private long backlogDurationMs;
+
+    /**
+     * Current backlog, as estimated number of event bytes we are behind, or null if
+     * unknown. Reported to callers.
+     */
+    @Nullable
+    private Long backlogBytes;
+
+    /**
+     * Wallclock time (ms since epoch) we last reported the backlog, or -1 if never reported.
+     */
+    private long lastReportedBacklogWallclock;
+
+    /**
+     * Event time (ms since epoch) of pending event at last reported backlog, or -1 if never
+     * calculated.
+     */
+    private long timestampAtLastReportedBacklogMs;
+
+    /** Next event to make 'current' when wallclock time has advanced sufficiently. */
+    @Nullable
+    private TimestampedValue<Event> pendingEvent;
+
+    /** Wallclock time when {@link #pendingEvent} is due, or -1 if no pending event. */
+    private long pendingEventWallclockTime;
+
+    /** Current event to return from getCurrent. */
+    @Nullable
+    private TimestampedValue<Event> currentEvent;
+
+    /** Events which have been held back so as to force them to be late. */
+    private Queue<Generator.NextEvent> heldBackEvents = new PriorityQueue<>();
+
+    public EventReader(Generator generator) {
+      this.generator = generator;
+      watermark = NexmarkUtils.BEGINNING_OF_TIME.getMillis();
+      lastReportedBacklogWallclock = -1;
+      pendingEventWallclockTime = -1;
+      timestampAtLastReportedBacklogMs = -1;
+    }
+
+    public EventReader(GeneratorConfig config) {
+      this(new Generator(config));
+    }
+
+    @Override
+    public boolean start() {
+      NexmarkUtils.error("starting unbounded generator %s", generator);
+      return advance();
+    }
+
+
+    @Override
+    public boolean advance() {
+      long now = System.currentTimeMillis();
+
+      while (pendingEvent == null) {
+        if (!generator.hasNext() && heldBackEvents.isEmpty()) {
+          // No more events, EVER.
+          if (isRateLimited) {
+            updateBacklog(System.currentTimeMillis(), 0);
+          }
+          if (watermark < BoundedWindow.TIMESTAMP_MAX_VALUE.getMillis()) {
+            watermark = BoundedWindow.TIMESTAMP_MAX_VALUE.getMillis();
+            NexmarkUtils.error("stopped unbounded generator %s", generator);
+          }
+          return false;
+        }
+
+        Generator.NextEvent next = heldBackEvents.peek();
+        if (next != null && next.wallclockTimestamp <= now) {
+          // Time to use the held-back event.
+          heldBackEvents.poll();
+          NexmarkUtils.error("replaying held-back event %dms behind watermark",
+                             watermark - next.eventTimestamp);
+        } else if (generator.hasNext()) {
+          next = generator.nextEvent();
+          if (isRateLimited && config.configuration.probDelayedEvent > 0.0
+              && config.configuration.occasionalDelaySec > 0
+              && ThreadLocalRandom.current().nextDouble() < config.configuration.probDelayedEvent) {
+            // We'll hold back this event and go around again.
+            long delayMs =
+                ThreadLocalRandom.current().nextLong(config.configuration.occasionalDelaySec * 1000)
+                + 1L;
+            NexmarkUtils.error("delaying event by %dms", delayMs);
+            heldBackEvents.add(next.withDelay(delayMs));
+            continue;
+          }
+        } else {
+          // Waiting for held-back event to fire.
+          if (isRateLimited) {
+            updateBacklog(now, 0);
+          }
+          return false;
+        }
+
+        pendingEventWallclockTime = next.wallclockTimestamp;
+        pendingEvent = TimestampedValue.of(next.event, new Instant(next.eventTimestamp));
+        long newWatermark =
+            next.watermark - Duration.standardSeconds(watermarkHoldbackSec).getMillis();
+        if (newWatermark > watermark) {
+          watermark = newWatermark;
+        }
+      }
+
+      if (isRateLimited) {
+        if (pendingEventWallclockTime > now) {
+          // We want this event to fire in the future. Try again later.
+          updateBacklog(now, 0);
+          return false;
+        }
+        updateBacklog(now, now - pendingEventWallclockTime);
+      }
+
+      // This event is ready to fire.
+      currentEvent = pendingEvent;
+      pendingEvent = null;
+      return true;
+    }
+
+    private void updateBacklog(long now, long newBacklogDurationMs) {
+      backlogDurationMs = newBacklogDurationMs;
+      long interEventDelayUs = generator.currentInterEventDelayUs();
+      if (interEventDelayUs != 0) {
+        long backlogEvents = (backlogDurationMs * 1000 + interEventDelayUs - 1) / interEventDelayUs;
+        backlogBytes = generator.getCurrentConfig().estimatedBytesForEvents(backlogEvents);
+      }
+      if (lastReportedBacklogWallclock < 0
+          || now - lastReportedBacklogWallclock > BACKLOG_PERIOD.getMillis()) {
+        double timeDialation = Double.NaN;
+        if (pendingEvent != null
+            && lastReportedBacklogWallclock >= 0
+            && timestampAtLastReportedBacklogMs >= 0) {
+          long wallclockProgressionMs = now - lastReportedBacklogWallclock;
+          long eventTimeProgressionMs =
+              pendingEvent.getTimestamp().getMillis() - timestampAtLastReportedBacklogMs;
+          timeDialation = (double) eventTimeProgressionMs / (double) wallclockProgressionMs;
+        }
+        NexmarkUtils.error(
+            "unbounded generator backlog now %dms (%s bytes) at %dus interEventDelay "
+            + "with %f time dilation",
+            backlogDurationMs, backlogBytes, interEventDelayUs, timeDialation);
+        lastReportedBacklogWallclock = now;
+        if (pendingEvent != null) {
+          timestampAtLastReportedBacklogMs = pendingEvent.getTimestamp().getMillis();
+        }
+      }
+    }
+
+    @Override
+    public Event getCurrent() {
+      if (currentEvent == null) {
+        throw new NoSuchElementException();
+      }
+      return currentEvent.getValue();
+    }
+
+    @Override
+    public Instant getCurrentTimestamp() {
+      if (currentEvent == null) {
+        throw new NoSuchElementException();
+      }
+      return currentEvent.getTimestamp();
+    }
+
+    @Override
+    public void close() {
+      // Nothing to close.
+    }
+
+    @Override
+    public UnboundedEventSource getCurrentSource() {
+      return UnboundedEventSource.this;
+    }
+
+    @Override
+    public Instant getWatermark() {
+      return new Instant(watermark);
+    }
+
+    @Override
+    public Generator.Checkpoint getCheckpointMark() {
+      return generator.toCheckpoint();
+    }
+
+    @Override
+    public long getSplitBacklogBytes() {
+      return backlogBytes == null ? BACKLOG_UNKNOWN : backlogBytes;
+    }
+
+    @Override
+    public String toString() {
+      return String.format("EventReader(%d, %d, %d)",
+          generator.getCurrentConfig().getStartEventId(), generator.getNextEventId(),
+          generator.getCurrentConfig().getStopEventId());
+    }
+  }
+
+  @Override
+  public Coder<Generator.Checkpoint> getCheckpointMarkCoder() {
+    return Generator.Checkpoint.CODER_INSTANCE;
+  }
+
+  @Override
+  public List<UnboundedEventSource> generateInitialSplits(
+      int desiredNumSplits, PipelineOptions options) {
+    NexmarkUtils.error(
+        "splitting unbounded source %s into %d sub-sources", config, numEventGenerators);
+    List<UnboundedEventSource> results = new ArrayList<>();
+    // Ignore desiredNumSplits and use numEventGenerators instead.
+    for (GeneratorConfig subConfig : config.split(numEventGenerators)) {
+      results.add(new UnboundedEventSource(subConfig, 1, watermarkHoldbackSec, isRateLimited));
+    }
+    return results;
+  }
+
+  @Override
+  public EventReader createReader(
+      PipelineOptions options, @Nullable Generator.Checkpoint checkpoint) {
+    if (checkpoint == null) {
+      NexmarkUtils.error("creating initial unbounded reader for %s", config);
+      return new EventReader(config);
+    } else {
+      NexmarkUtils.error("resuming unbounded reader from %s", checkpoint);
+      return new EventReader(checkpoint.toGenerator(config));
+    }
+  }
+
+  @Override
+  public void validate() {
+    // Nothing to validate.
+  }
+
+  @Override
+  public Coder<Event> getDefaultOutputCoder() {
+    return Event.CODER;
+  }
+
+  @Override
+  public String toString() {
+    return String.format(
+        "UnboundedEventSource(%d, %d)", config.getStartEventId(), config.getStopEventId());
+  }
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/1f08970a/integration/java/src/main/java/org/apache/beam/integration/nexmark/WinningBids.java
----------------------------------------------------------------------
diff --git a/integration/java/src/main/java/org/apache/beam/integration/nexmark/WinningBids.java b/integration/java/src/main/java/org/apache/beam/integration/nexmark/WinningBids.java
new file mode 100644
index 0000000..16f901c
--- /dev/null
+++ b/integration/java/src/main/java/org/apache/beam/integration/nexmark/WinningBids.java
@@ -0,0 +1,378 @@
+/*
+ * 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.beam.integration.nexmark;
+
+import org.apache.beam.sdk.coders.AtomicCoder;
+import org.apache.beam.sdk.coders.Coder;
+import org.apache.beam.sdk.coders.CoderException;
+import org.apache.beam.sdk.coders.VarIntCoder;
+import org.apache.beam.sdk.coders.VarLongCoder;
+import org.apache.beam.sdk.transforms.Aggregator;
+import org.apache.beam.sdk.transforms.DoFn;
+import org.apache.beam.sdk.transforms.PTransform;
+import org.apache.beam.sdk.transforms.ParDo;
+import org.apache.beam.sdk.transforms.Sum.SumLongFn;
+import org.apache.beam.sdk.transforms.join.CoGbkResult;
+import org.apache.beam.sdk.transforms.join.CoGroupByKey;
+import org.apache.beam.sdk.transforms.join.KeyedPCollectionTuple;
+import org.apache.beam.sdk.transforms.windowing.BoundedWindow;
+import org.apache.beam.sdk.transforms.windowing.IntervalWindow;
+import org.apache.beam.sdk.transforms.windowing.Window;
+import org.apache.beam.sdk.transforms.windowing.WindowFn;
+import org.apache.beam.sdk.values.KV;
+import org.apache.beam.sdk.values.PCollection;
+import com.google.common.base.Preconditions;
+
+import com.fasterxml.jackson.annotation.JsonCreator;
+
+import org.joda.time.Instant;
+
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
+import java.io.Serializable;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.List;
+import java.util.Map;
+import java.util.TreeMap;
+
+/**
+ * A transform to find the winning bid for each closed auction. In pseudo CQL syntax:
+ *
+ * <pre>
+ * SELECT Rstream(A.*, B.auction, B.bidder, MAX(B.price), B.dateTime)
+ * FROM Auction A [ROWS UNBOUNDED], Bid B [ROWS UNBOUNDED]
+ * WHERE A.id=B.auction AND B.datetime < A.expires AND A.expires < CURRENT_TIME
+ * GROUP BY A.id
+ * </pre>
+ *
+ * <p>We will also check that the winning bid is above the auction reserve. Note that
+ * we ignore the auction opening bid value since it has no impact on which bid eventually wins,
+ * if any.
+ *
+ * <p>Our implementation will use a custom windowing function in order to bring bids and
+ * auctions together without requiring global state.
+ */
+public class WinningBids extends PTransform<PCollection<Event>, PCollection<AuctionBid>> {
+  /** Windows for open auctions and bids. */
+  private static class AuctionOrBidWindow extends IntervalWindow implements Serializable {
+    /** Id of auction this window is for. */
+    public final long auction;
+
+    /**
+     * True if this window represents an actual auction, and thus has a start/end
+     * time matching that of the auction. False if this window represents a bid, and
+     * thus has an unbounded start/end time.
+     */
+    public final boolean isAuctionWindow;
+
+    /** For avro only. */
+    private AuctionOrBidWindow() {
+      super(TIMESTAMP_MIN_VALUE, TIMESTAMP_MAX_VALUE);
+      auction = 0;
+      isAuctionWindow = false;
+    }
+
+    private AuctionOrBidWindow(
+        Instant start, Instant end, long auctionId, boolean isAuctionWindow) {
+      super(start, end);
+      this.auction = auctionId;
+      this.isAuctionWindow = isAuctionWindow;
+    }
+
+    /** Return an auction window for {@code auction}. */
+    public static AuctionOrBidWindow forAuction(Instant timestamp, Auction auction) {
+      AuctionOrBidWindow result =
+          new AuctionOrBidWindow(timestamp, new Instant(auction.expires), auction.id, true);
+      return result;
+    }
+
+    /**
+     * Return a bid window for {@code bid}. It should later be merged into
+     * the corresponding auction window. However, it is possible this bid is for an already
+     * expired auction, or for an auction which the system has not yet seen. So we
+     * give the bid a bit of wiggle room in its interval.
+     */
+    public static AuctionOrBidWindow forBid(
+        long expectedAuctionDurationMs, Instant timestamp, Bid bid) {
+      // At this point we don't know which auctions are still valid, and the bid may
+      // be for an auction which won't start until some unknown time in the future
+      // (due to Generator.AUCTION_ID_LEAD in Generator.nextBid).
+      // A real system would atomically reconcile bids and auctions by a separate mechanism.
+      // If we give bids an unbounded window it is possible a bid for an auction which
+      // has already expired would cause the system watermark to stall, since that window
+      // would never be retired.
+      // Instead, we will just give the bid a finite window which expires at
+      // the upper bound of auctions assuming the auction starts at the same time as the bid,
+      // and assuming the system is running at its lowest event rate (as per interEventDelayUs).
+      AuctionOrBidWindow result = new AuctionOrBidWindow(
+          timestamp, timestamp.plus(expectedAuctionDurationMs * 2), bid.auction, false);
+      return result;
+    }
+
+    /** Is this an auction window? */
+    public boolean isAuctionWindow() {
+      return isAuctionWindow;
+    }
+
+    @Override
+    public String toString() {
+      return String.format("AuctionOrBidWindow{start:%s; end:%s; auction:%d; isAuctionWindow:%s}",
+          start(), end(), auction, isAuctionWindow);
+    }
+  }
+
+  /**
+   * Encodes an {@link AuctionOrBidWindow} as an {@link IntervalWindow} and an auction id long.
+   */
+  private static class AuctionOrBidWindowCoder extends AtomicCoder<AuctionOrBidWindow> {
+    private static final AuctionOrBidWindowCoder INSTANCE = new AuctionOrBidWindowCoder();
+    private static final Coder<IntervalWindow> SUPER_CODER = IntervalWindow.getCoder();
+    private static final Coder<Long> ID_CODER = VarLongCoder.of();
+    private static final Coder<Integer> INT_CODER = VarIntCoder.of();
+
+    @JsonCreator
+    public static AuctionOrBidWindowCoder of() {
+      return INSTANCE;
+    }
+
+    @Override
+    public void encode(AuctionOrBidWindow window, OutputStream outStream, Context context)
+        throws IOException, CoderException {
+      SUPER_CODER.encode(window, outStream, Context.NESTED);
+      ID_CODER.encode(window.auction, outStream, Context.NESTED);
+      INT_CODER.encode(window.isAuctionWindow ? 1 : 0, outStream, Context.NESTED);
+    }
+
+    @Override
+    public AuctionOrBidWindow decode(InputStream inStream, Context context)
+        throws IOException, CoderException {
+      IntervalWindow superWindow = SUPER_CODER.decode(inStream, Context.NESTED);
+      long auction = ID_CODER.decode(inStream, Context.NESTED);
+      boolean isAuctionWindow = INT_CODER.decode(inStream, Context.NESTED) == 0 ? false : true;
+      return new AuctionOrBidWindow(
+          superWindow.start(), superWindow.end(), auction, isAuctionWindow);
+    }
+  }
+
+  /** Assign events to auction windows and merges them intelligently. */
+  private static class AuctionOrBidWindowFn extends WindowFn<Event, AuctionOrBidWindow> {
+    /** Expected duration of auctions in ms. */
+    private final long expectedAuctionDurationMs;
+
+    public AuctionOrBidWindowFn(long expectedAuctionDurationMs) {
+      this.expectedAuctionDurationMs = expectedAuctionDurationMs;
+    }
+
+    @Override
+    public Collection<AuctionOrBidWindow> assignWindows(AssignContext c) {
+      Event event = c.element();
+      if (event.newAuction != null) {
+        // Assign auctions to an auction window which expires at the auction's close.
+        return Arrays.asList(AuctionOrBidWindow.forAuction(c.timestamp(), event.newAuction));
+      } else if (event.bid != null) {
+        // Assign bids to a temporary bid window which will later be merged into the appropriate
+        // auction window.
+        return Arrays.asList(
+            AuctionOrBidWindow.forBid(expectedAuctionDurationMs, c.timestamp(), event.bid));
+      } else {
+        // Don't assign people to any window. They will thus be dropped.
+        return Arrays.asList();
+      }
+    }
+
+    @Override
+    public void mergeWindows(MergeContext c) throws Exception {
+      // Split and index the auction and bid windows by auction id.
+      Map<Long, AuctionOrBidWindow> idToTrueAuctionWindow = new TreeMap<>();
+      Map<Long, List<AuctionOrBidWindow>> idToBidAuctionWindows = new TreeMap<>();
+      for (AuctionOrBidWindow window : c.windows()) {
+        if (window.isAuctionWindow()) {
+          idToTrueAuctionWindow.put(window.auction, window);
+        } else {
+          List<AuctionOrBidWindow> bidWindows = idToBidAuctionWindows.get(window.auction);
+          if (bidWindows == null) {
+            bidWindows = new ArrayList<>();
+            idToBidAuctionWindows.put(window.auction, bidWindows);
+          }
+          bidWindows.add(window);
+        }
+      }
+
+      // Merge all 'bid' windows into their corresponding 'auction' window, provided the
+      // auction has not expired.
+      for (long auction : idToTrueAuctionWindow.keySet()) {
+        AuctionOrBidWindow auctionWindow = idToTrueAuctionWindow.get(auction);
+        List<AuctionOrBidWindow> bidWindows = idToBidAuctionWindows.get(auction);
+        if (bidWindows != null) {
+          List<AuctionOrBidWindow> toBeMerged = new ArrayList<>();
+          for (AuctionOrBidWindow bidWindow : bidWindows) {
+            if (bidWindow.start().isBefore(auctionWindow.end())) {
+              toBeMerged.add(bidWindow);
+            }
+            // else: This bid window will remain until its expire time, at which point it
+            // will expire without ever contributing to an output.
+          }
+          if (!toBeMerged.isEmpty()) {
+            toBeMerged.add(auctionWindow);
+            c.merge(toBeMerged, auctionWindow);
+          }
+        }
+      }
+    }
+
+    @Override
+    public boolean isCompatible(WindowFn<?, ?> other) {
+      return other instanceof AuctionOrBidWindowFn;
+    }
+
+    @Override
+    public Coder<AuctionOrBidWindow> windowCoder() {
+      return AuctionOrBidWindowCoder.of();
+    }
+
+    @Override
+    public AuctionOrBidWindow getSideInputWindow(BoundedWindow window) {
+      throw new UnsupportedOperationException("AuctionWindowFn not supported for side inputs");
+    }
+
+    /**
+     * Below we will GBK auctions and bids on their auction ids. Then we will reduce those
+     * per id to emit {@code (auction, winning bid)} pairs for auctions which have expired with at
+     * least one valid bid. We would like those output pairs to have a timestamp of the auction's
+     * expiry (since that's the earliest we know for sure we have the correct winner). We would
+     * also like to make that winning results are available to following stages at the auction's
+     * expiry.
+     *
+     * <p>
+     * Each result of the GBK will have a timestamp of the min of the result of this object's
+     * assignOutputTime over all records which end up in one of its iterables. Thus we get the
+     * desired behavior if we ignore each record's timestamp and always return the auction window's
+     * 'maxTimestamp', which will correspond to the auction's expiry.
+     *
+     * <p>
+     * In contrast, if this object's assignOutputTime were to return 'inputTimestamp'
+     * (the usual implementation), then each GBK record will take as its timestamp the minimum of
+     * the timestamps of all bids and auctions within it, which will always be the auction's
+     * timestamp. An auction which expires well into the future would thus hold up the watermark
+     * of the GBK results until that auction expired. That in turn would hold up all winning pairs.
+     */
+    @Override
+    public Instant getOutputTime(
+        Instant inputTimestamp, AuctionOrBidWindow window) {
+      return window.maxTimestamp();
+    }
+  }
+
+  private final AuctionOrBidWindowFn auctionOrBidWindowFn;
+
+  public WinningBids(String name, NexmarkConfiguration configuration) {
+    super(name);
+    // What's the expected auction time (when the system is running at the lowest event rate).
+    long[] interEventDelayUs = configuration.rateShape.interEventDelayUs(
+        configuration.firstEventRate, configuration.nextEventRate,
+        configuration.rateUnit, configuration.numEventGenerators);
+    long longestDelayUs = 0;
+    for (int i = 0; i < interEventDelayUs.length; i++) {
+      longestDelayUs = Math.max(longestDelayUs, interEventDelayUs[i]);
+    }
+    // Adjust for proportion of auction events amongst all events.
+    longestDelayUs =
+        (longestDelayUs * GeneratorConfig.PROPORTION_DENOMINATOR)
+        / GeneratorConfig.AUCTION_PROPORTION;
+    // Adjust for number of in-flight auctions.
+    longestDelayUs = longestDelayUs * configuration.numInFlightAuctions;
+    long expectedAuctionDurationMs = (longestDelayUs + 999) / 1000;
+    NexmarkUtils.console("Expected auction duration is %d ms", expectedAuctionDurationMs);
+    auctionOrBidWindowFn = new AuctionOrBidWindowFn(expectedAuctionDurationMs);
+  }
+
+  @Override
+  public PCollection<AuctionBid> apply(PCollection<Event> events) {
+    // Window auctions and bids into custom auction windows. New people events will be discarded.
+    // This will allow us to bring bids and auctions together irrespective of how long
+    // each auction is open for.
+    events = events.apply(Window.named("Window").into(auctionOrBidWindowFn));
+
+    // Key auctions by their id.
+    PCollection<KV<Long, Auction>> auctionsById =
+        events.apply(NexmarkQuery.JUST_NEW_AUCTIONS).apply(NexmarkQuery.AUCTION_BY_ID);
+
+    // Key bids by their auction id.
+    PCollection<KV<Long, Bid>> bidsByAuctionId =
+        events.apply(NexmarkQuery.JUST_BIDS).apply(NexmarkQuery.BID_BY_AUCTION);
+
+    // Find the highest price valid bid for each closed auction.
+    return
+        // Join auctions and bids.
+        KeyedPCollectionTuple.of(NexmarkQuery.AUCTION_TAG, auctionsById)
+            .and(NexmarkQuery.BID_TAG, bidsByAuctionId)
+            .apply(CoGroupByKey.<Long>create())
+
+            // Filter and select.
+            .apply(
+                ParDo.named(name + ".Join")
+                    .of(new DoFn<KV<Long, CoGbkResult>, AuctionBid>() {
+                      final Aggregator<Long, Long> noAuctionCounter =
+                          createAggregator("noAuction", new SumLongFn());
+                      final Aggregator<Long, Long> underReserveCounter =
+                          createAggregator("underReserve", new SumLongFn());
+                      final Aggregator<Long, Long> noValidBidsCounter =
+                          createAggregator("noValidBids", new SumLongFn());
+
+
+                      @Override
+                      public void processElement(ProcessContext c) {
+                        Auction auction =
+                            c.element().getValue().getOnly(NexmarkQuery.AUCTION_TAG, null);
+                        if (auction == null) {
+                          // We have bids without a matching auction. Give up.
+                          noAuctionCounter.addValue(1L);
+                          return;
+                        }
+                        // Find the current winning bid for auction.
+                        // The earliest bid with the maximum price above the reserve wins.
+                        Bid bestBid = null;
+                        for (Bid bid : c.element().getValue().getAll(NexmarkQuery.BID_TAG)) {
+                          // Bids too late for their auction will have been
+                          // filtered out by the window merge function.
+                          Preconditions.checkState(bid.dateTime < auction.expires);
+                          if (bid.price < auction.reserve) {
+                            // Bid price is below auction reserve.
+                            underReserveCounter.addValue(1L);
+                            continue;
+                          }
+
+                          if (bestBid == null
+                              || Bid.PRICE_THEN_DESCENDING_TIME.compare(bid, bestBid) > 0) {
+                            bestBid = bid;
+                          }
+                        }
+                        if (bestBid == null) {
+                          // We don't have any valid bids for auction.
+                          noValidBidsCounter.addValue(1L);
+                          return;
+                        }
+                        c.output(new AuctionBid(auction, bestBid));
+                      }
+                    }));
+  }
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/1f08970a/integration/java/src/main/java/org/apache/beam/integration/nexmark/WinningBidsSimulator.java
----------------------------------------------------------------------
diff --git a/integration/java/src/main/java/org/apache/beam/integration/nexmark/WinningBidsSimulator.java b/integration/java/src/main/java/org/apache/beam/integration/nexmark/WinningBidsSimulator.java
new file mode 100644
index 0000000..b61aed1
--- /dev/null
+++ b/integration/java/src/main/java/org/apache/beam/integration/nexmark/WinningBidsSimulator.java
@@ -0,0 +1,203 @@
+/*
+ * 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.beam.integration.nexmark;
+
+import org.apache.beam.sdk.transforms.windowing.BoundedWindow;
+import org.apache.beam.sdk.values.TimestampedValue;
+
+import org.joda.time.Instant;
+
+import java.util.ArrayList;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.TreeMap;
+import java.util.TreeSet;
+
+import javax.annotation.Nullable;
+
+/**
+ * A simulator of the {@code WinningBids} query.
+ */
+public class WinningBidsSimulator extends AbstractSimulator<Event, AuctionBid> {
+  /** Auctions currently still open, indexed by auction id. */
+  private final Map<Long, Auction> openAuctions;
+
+  /** The ids of auctions known to be closed. */
+  private final Set<Long> closedAuctions;
+
+  /** Current best valid bids for open auctions, indexed by auction id. */
+  private final Map<Long, Bid> bestBids;
+
+  /** Bids for auctions we havn't seen yet. */
+  private final List<Bid> bidsWithoutAuctions;
+
+  /**
+   * Timestamp of last new auction or bid event (ms since epoch).
+   */
+  private long lastTimestamp;
+
+  public WinningBidsSimulator(NexmarkConfiguration configuration) {
+    super(NexmarkUtils.standardEventIterator(configuration));
+    openAuctions = new TreeMap<>();
+    closedAuctions = new TreeSet<>();
+    bestBids = new TreeMap<>();
+    bidsWithoutAuctions = new ArrayList<>();
+    lastTimestamp = BoundedWindow.TIMESTAMP_MIN_VALUE.getMillis();
+  }
+
+  /**
+   * Try to account for {@code bid} in state. Return true if bid has now been
+   * accounted for by {@code bestBids}.
+   */
+  private boolean captureBestBid(Bid bid, boolean shouldLog) {
+    if (closedAuctions.contains(bid.auction)) {
+      // Ignore bids for known, closed auctions.
+      if (shouldLog) {
+        NexmarkUtils.info("closed auction: %s", bid);
+      }
+      return true;
+    }
+    Auction auction = openAuctions.get(bid.auction);
+    if (auction == null) {
+      // We don't have an auction for this bid yet, so can't determine if it is
+      // winning or not.
+      if (shouldLog) {
+        NexmarkUtils.info("pending auction: %s", bid);
+      }
+      return false;
+    }
+    if (bid.price < auction.reserve) {
+      // Bid price is too low.
+      if (shouldLog) {
+        NexmarkUtils.info("below reserve: %s", bid);
+      }
+      return true;
+    }
+    Bid existingBid = bestBids.get(bid.auction);
+    if (existingBid == null || Bid.PRICE_THEN_DESCENDING_TIME.compare(existingBid, bid) < 0) {
+      // We've found a (new) best bid for a known auction.
+      bestBids.put(bid.auction, bid);
+      if (shouldLog) {
+        NexmarkUtils.info("new winning bid: %s", bid);
+      }
+    } else {
+      if (shouldLog) {
+        NexmarkUtils.info("ignoring low bid: %s", bid);
+      }
+    }
+    return true;
+  }
+
+  /**
+   * Try to match bids without auctions to auctions.
+   */
+  private void flushBidsWithoutAuctions() {
+    Iterator<Bid> itr = bidsWithoutAuctions.iterator();
+    while (itr.hasNext()) {
+      Bid bid = itr.next();
+      if (captureBestBid(bid, false)) {
+        NexmarkUtils.info("bid now accounted for: %s", bid);
+        itr.remove();
+      }
+    }
+  }
+
+  /**
+   * Return the next winning bid for an expired auction relative to {@code timestamp}.
+   * Return null if no more winning bids, in which case all expired auctions will
+   * have been removed from our state. Retire auctions in order of expire time.
+   */
+  @Nullable
+  private TimestampedValue<AuctionBid> nextWinningBid(long timestamp) {
+    Map<Long, List<Long>> toBeRetired = new TreeMap<>();
+    for (Map.Entry<Long, Auction> entry : openAuctions.entrySet()) {
+      if (entry.getValue().expires <= timestamp) {
+        List<Long> idsAtTime = toBeRetired.get(entry.getValue().expires);
+        if (idsAtTime == null) {
+          idsAtTime = new ArrayList<>();
+          toBeRetired.put(entry.getValue().expires, idsAtTime);
+        }
+        idsAtTime.add(entry.getKey());
+      }
+    }
+    for (Map.Entry<Long, List<Long>> entry : toBeRetired.entrySet()) {
+      for (long id : entry.getValue()) {
+        Auction auction = openAuctions.get(id);
+        NexmarkUtils.info("retiring auction: %s", auction);
+        openAuctions.remove(id);
+        Bid bestBid = bestBids.get(id);
+        if (bestBid != null) {
+          TimestampedValue<AuctionBid> result =
+              TimestampedValue.of(new AuctionBid(auction, bestBid), new Instant(auction.expires));
+          NexmarkUtils.info("winning: %s", result);
+          return result;
+        }
+      }
+    }
+    return null;
+  }
+
+  @Override
+  protected void run() {
+    if (lastTimestamp > BoundedWindow.TIMESTAMP_MIN_VALUE.getMillis()) {
+      // We may have finally seen the auction a bid was intended for.
+      flushBidsWithoutAuctions();
+      TimestampedValue<AuctionBid> result = nextWinningBid(lastTimestamp);
+      if (result != null) {
+        addResult(result);
+        return;
+      }
+    }
+
+    TimestampedValue<Event> timestampedEvent = nextInput();
+    if (timestampedEvent == null) {
+      // No more events. Flush any still open auctions.
+      TimestampedValue<AuctionBid> result =
+          nextWinningBid(BoundedWindow.TIMESTAMP_MAX_VALUE.getMillis());
+      if (result == null) {
+        // We are done.
+        allDone();
+        return;
+      }
+      addResult(result);
+      return;
+    }
+
+    Event event = timestampedEvent.getValue();
+    if (event.newPerson != null) {
+      // Ignore new person events.
+      return;
+    }
+
+    lastTimestamp = timestampedEvent.getTimestamp().getMillis();
+    if (event.newAuction != null) {
+      // Add this new open auction to our state.
+      openAuctions.put(event.newAuction.id, event.newAuction);
+    } else {
+      if (!captureBestBid(event.bid, true)) {
+        // We don't know what to do with this bid yet.
+        NexmarkUtils.info("bid not yet accounted for: %s", event.bid);
+        bidsWithoutAuctions.add(event.bid);
+      }
+    }
+    // Keep looking for winning bids.
+  }
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/1f08970a/integration/java/src/test/java/org/apache/beam/integration/nexmark/BoundedEventSourceTest.java
----------------------------------------------------------------------
diff --git a/integration/java/src/test/java/org/apache/beam/integration/nexmark/BoundedEventSourceTest.java b/integration/java/src/test/java/org/apache/beam/integration/nexmark/BoundedEventSourceTest.java
new file mode 100644
index 0000000..f017267
--- /dev/null
+++ b/integration/java/src/test/java/org/apache/beam/integration/nexmark/BoundedEventSourceTest.java
@@ -0,0 +1,71 @@
+/*
+ * 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.beam.integration.nexmark;
+
+import org.apache.beam.runners.dataflow.options.DataflowPipelineOptions;
+import org.apache.beam.sdk.options.PipelineOptionsFactory;
+import org.apache.beam.sdk.testing.SourceTestUtils;
+
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.JUnit4;
+
+/**
+ * Test {@link BoundedEventSource}.
+ */
+@RunWith(JUnit4.class)
+public class BoundedEventSourceTest {
+  private GeneratorConfig makeConfig(long n) {
+    return new GeneratorConfig(
+        NexmarkConfiguration.DEFAULT, System.currentTimeMillis(), 0, n, 0);
+  }
+
+  @Test
+  public void sourceAndReadersWork() throws Exception {
+    DataflowPipelineOptions options = PipelineOptionsFactory.as(DataflowPipelineOptions.class);
+    long n = 200L;
+    BoundedEventSource source = new BoundedEventSource(makeConfig(n), 1);
+
+    SourceTestUtils.assertUnstartedReaderReadsSameAsItsSource(
+        source.createReader(options), options);
+  }
+
+  @Test
+  public void splitAtFractionRespectsContract() throws Exception {
+    DataflowPipelineOptions options = PipelineOptionsFactory.as(DataflowPipelineOptions.class);
+    long n = 20L;
+    BoundedEventSource source = new BoundedEventSource(makeConfig(n), 1);
+
+    // Can't split if already consumed.
+    SourceTestUtils.assertSplitAtFractionFails(source, 10, 0.3, options);
+
+    SourceTestUtils.assertSplitAtFractionSucceedsAndConsistent(source, 5, 0.3, options);
+
+    SourceTestUtils.assertSplitAtFractionExhaustive(source, options);
+  }
+
+  @Test
+  public void splitIntoBundlesRespectsContract() throws Exception {
+    DataflowPipelineOptions options = PipelineOptionsFactory.as(DataflowPipelineOptions.class);
+    long n = 200L;
+    BoundedEventSource source = new BoundedEventSource(makeConfig(n), 1);
+    SourceTestUtils.assertSourcesEqualReferenceSource(
+        source, source.splitIntoBundles(10, options), options);
+  }
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/1f08970a/integration/java/src/test/java/org/apache/beam/integration/nexmark/GeneratorTest.java
----------------------------------------------------------------------
diff --git a/integration/java/src/test/java/org/apache/beam/integration/nexmark/GeneratorTest.java b/integration/java/src/test/java/org/apache/beam/integration/nexmark/GeneratorTest.java
new file mode 100644
index 0000000..bbaee26
--- /dev/null
+++ b/integration/java/src/test/java/org/apache/beam/integration/nexmark/GeneratorTest.java
@@ -0,0 +1,111 @@
+/*
+ * 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.beam.integration.nexmark;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.JUnit4;
+
+import java.util.ArrayList;
+import java.util.Iterator;
+import java.util.List;
+
+/**
+ * Test {@link Generator}.
+ */
+@RunWith(JUnit4.class)
+public class GeneratorTest {
+  private GeneratorConfig makeConfig(long n) {
+    return new GeneratorConfig(
+        NexmarkConfiguration.DEFAULT, System.currentTimeMillis(), 0, n, 0);
+  }
+
+  private <T> long consume(long n, Iterator<T> itr) {
+    for (long i = 0; i < n; i++) {
+      assertTrue(itr.hasNext());
+      itr.next();
+    }
+    return n;
+  }
+
+  private <T> long consume(Iterator<T> itr) {
+    long n = 0;
+    while (itr.hasNext()) {
+      itr.next();
+      n++;
+    }
+    return n;
+  }
+
+  @Test
+  public void splitAtFractionPreservesOverallEventCount() {
+    long n = 55729L;
+    GeneratorConfig initialConfig = makeConfig(n);
+    long expected = initialConfig.getStopEventId() - initialConfig.getStartEventId();
+
+    long actual = 0;
+
+    Generator initialGenerator = new Generator(initialConfig);
+
+    // Consume some events.
+    actual += consume(5000, initialGenerator);
+
+
+    // Split once.
+    GeneratorConfig remainConfig1 = initialGenerator.splitAtEventId(9000L);
+    Generator remainGenerator1 = new Generator(remainConfig1);
+
+    // Consume some more events.
+    actual += consume(2000, initialGenerator);
+    actual += consume(3000, remainGenerator1);
+
+    // Split again.
+    GeneratorConfig remainConfig2 = remainGenerator1.splitAtEventId(30000L);
+    Generator remainGenerator2 = new Generator(remainConfig2);
+
+    // Run to completion.
+    actual += consume(initialGenerator);
+    actual += consume(remainGenerator1);
+    actual += consume(remainGenerator2);
+
+    assertEquals(expected, actual);
+  }
+
+  @Test
+  public void splitPreservesOverallEventCount() {
+    long n = 51237L;
+    GeneratorConfig initialConfig = makeConfig(n);
+    long expected = initialConfig.getStopEventId() - initialConfig.getStartEventId();
+
+    List<Generator> generators = new ArrayList<>();
+    for (GeneratorConfig subConfig : initialConfig.split(20)) {
+      generators.add(new Generator(subConfig));
+    }
+
+    long actual = 0;
+    for (Generator generator : generators) {
+      actual += consume(generator);
+    }
+
+    assertEquals(expected, actual);
+  }
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/1f08970a/integration/java/src/test/java/org/apache/beam/integration/nexmark/QueryTest.java
----------------------------------------------------------------------
diff --git a/integration/java/src/test/java/org/apache/beam/integration/nexmark/QueryTest.java b/integration/java/src/test/java/org/apache/beam/integration/nexmark/QueryTest.java
new file mode 100644
index 0000000..860fa78
--- /dev/null
+++ b/integration/java/src/test/java/org/apache/beam/integration/nexmark/QueryTest.java
@@ -0,0 +1,103 @@
+/*
+ * 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.beam.integration.nexmark;
+
+import org.apache.beam.sdk.Pipeline;
+import org.apache.beam.sdk.testing.PAssert;
+import org.apache.beam.sdk.testing.TestPipeline;
+import org.apache.beam.sdk.values.PCollection;
+import org.apache.beam.sdk.values.PCollection.IsBounded;
+import org.apache.beam.sdk.values.TimestampedValue;
+
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.JUnit4;
+
+/**
+ * Test the various NEXMark queries yield results coherent with their models.
+ */
+@RunWith(JUnit4.class)
+public class QueryTest {
+  private static final NexmarkConfiguration CONFIG = NexmarkConfiguration.DEFAULT.clone();
+
+  static {
+    CONFIG.numEvents = 2000;
+  }
+
+  /** Test {@code query} matches {@code model}. */
+  private static void queryMatchesModel(String name, NexmarkQuery query, NexmarkQueryModel model) {
+    Pipeline p = TestPipeline.create();
+    NexmarkUtils.setupPipeline(NexmarkUtils.CoderStrategy.HAND, p);
+    PCollection<TimestampedValue<KnownSize>> results =
+        p.apply(NexmarkUtils.batchEventsSource(name, CONFIG)).apply(query);
+    results.setIsBoundedInternal(IsBounded.BOUNDED);
+    PAssert.that(results).satisfies(model.assertionFor());
+    p.run();
+  }
+
+  @Test
+  public void query0MatchesModel() {
+    queryMatchesModel("Query0Test", new Query0(CONFIG), new Query0Model(CONFIG));
+  }
+
+  @Test
+  public void query1MatchesModel() {
+    queryMatchesModel("Query1Test", new Query1(CONFIG), new Query1Model(CONFIG));
+  }
+
+  @Test
+  public void query2MatchesModel() {
+    queryMatchesModel("Query2Test", new Query2(CONFIG), new Query2Model(CONFIG));
+  }
+
+  @Test
+  public void query3MatchesModel() {
+    queryMatchesModel("Query3Test", new Query3(CONFIG), new Query3Model(CONFIG));
+  }
+
+  @Test
+  public void query4MatchesModel() {
+    queryMatchesModel("Query4Test", new Query4(CONFIG), new Query4Model(CONFIG));
+  }
+
+  @Test
+  public void query5MatchesModel() {
+    queryMatchesModel("Query5Test", new Query5(CONFIG), new Query5Model(CONFIG));
+  }
+
+  @Test
+  public void query6MatchesModel() {
+    queryMatchesModel("Query6Test", new Query6(CONFIG), new Query6Model(CONFIG));
+  }
+
+  @Test
+  public void query7MatchesModel() {
+    queryMatchesModel("Query7Test", new Query7(CONFIG), new Query7Model(CONFIG));
+  }
+
+  @Test
+  public void query8MatchesModel() {
+    queryMatchesModel("Query8Test", new Query8(CONFIG), new Query8Model(CONFIG));
+  }
+
+  @Test
+  public void query9MatchesModel() {
+    queryMatchesModel("Query9Test", new Query9(CONFIG), new Query9Model(CONFIG));
+  }
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/1f08970a/integration/java/src/test/java/org/apache/beam/integration/nexmark/UnboundedEventSourceTest.java
----------------------------------------------------------------------
diff --git a/integration/java/src/test/java/org/apache/beam/integration/nexmark/UnboundedEventSourceTest.java b/integration/java/src/test/java/org/apache/beam/integration/nexmark/UnboundedEventSourceTest.java
new file mode 100644
index 0000000..5d72f77
--- /dev/null
+++ b/integration/java/src/test/java/org/apache/beam/integration/nexmark/UnboundedEventSourceTest.java
@@ -0,0 +1,109 @@
+/*
+ * 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.beam.integration.nexmark;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+
+import org.apache.beam.sdk.Pipeline;
+import org.apache.beam.sdk.io.UnboundedSource.CheckpointMark;
+import org.apache.beam.sdk.io.UnboundedSource.UnboundedReader;
+import org.apache.beam.sdk.options.PipelineOptions;
+import org.apache.beam.sdk.testing.TestPipeline;
+
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.JUnit4;
+
+import java.io.IOException;
+import java.util.HashSet;
+import java.util.Random;
+import java.util.Set;
+
+/**
+ * Test UnboundedEventSource.
+ */
+@RunWith(JUnit4.class)
+public class UnboundedEventSourceTest {
+  private GeneratorConfig makeConfig(long n) {
+    return new GeneratorConfig(
+        NexmarkConfiguration.DEFAULT, System.currentTimeMillis(), 0, n, 0);
+  }
+
+  /**
+   * Helper for tracking which ids we've seen (so we can detect dups) and
+   * confirming reading events match the model events.
+   */
+  private static class EventIdChecker {
+    private Set<Long> seenPersonIds = new HashSet<>();
+    private Set<Long> seenAuctionIds = new HashSet<>();
+
+    public void add(Event event) {
+      if (event.newAuction != null) {
+        assertTrue(seenAuctionIds.add(event.newAuction.id));
+      } else if (event.newPerson != null) {
+        assertTrue(seenPersonIds.add(event.newPerson.id));
+      }
+    }
+
+    public void add(int n, UnboundedReader<Event> reader, Generator modelGenerator)
+        throws IOException {
+      for (int i = 0; i < n; i++) {
+        assertTrue(modelGenerator.hasNext());
+        Event modelEvent = modelGenerator.next().getValue();
+        assertTrue(reader.advance());
+        Event actualEvent = reader.getCurrent();
+        assertEquals(modelEvent.toString(), actualEvent.toString());
+        add(actualEvent);
+      }
+    }
+  }
+
+  /**
+   * Check aggressively checkpointing and resuming a reader gives us exactly the
+   * same event stream as reading directly.
+   */
+  @Test
+  public void resumeFromCheckpoint() throws IOException {
+    Random random = new Random(297);
+    int n = 47293;
+    GeneratorConfig config = makeConfig(n);
+    Generator modelGenerator = new Generator(config);
+
+    EventIdChecker checker = new EventIdChecker();
+    Pipeline p = TestPipeline.create();
+    PipelineOptions options = p.getOptions();
+    UnboundedEventSource source = new UnboundedEventSource(config, 1, 0, false);
+    UnboundedReader<Event> reader = source.createReader(options, null);
+
+    while (n > 0) {
+      int m = Math.min(459 + random.nextInt(455), n);
+      System.out.printf("reading %d...\n", m);
+      checker.add(m, reader, modelGenerator);
+      n -= m;
+      System.out.printf("splitting with %d remaining...\n", n);
+      CheckpointMark checkpointMark = reader.getCheckpointMark();
+      assertTrue(checkpointMark instanceof Generator.Checkpoint);
+      reader = source.createReader(options, (Generator.Checkpoint) checkpointMark);
+    }
+
+    assertFalse(reader.advance());
+  }
+}


[24/55] [abbrv] beam git commit: Fix and improve query3 and query12

Posted by ie...@apache.org.
Fix and improve query3 and query12

query3: Use GlobalWindow to comply with the State/Timer APIs (issue #7). Use timer for personState expiration in GlobalWindow (issue #29). Add trigger to GlobalWindow

query12: Replace Count.perKey by Count.perElement (issue #34)


Project: http://git-wip-us.apache.org/repos/asf/beam/repo
Commit: http://git-wip-us.apache.org/repos/asf/beam/commit/7c28b492
Tree: http://git-wip-us.apache.org/repos/asf/beam/tree/7c28b492
Diff: http://git-wip-us.apache.org/repos/asf/beam/diff/7c28b492

Branch: refs/heads/master
Commit: 7c28b492aa17160d9a84914814e618716b7beb9f
Parents: bd93c8b
Author: Etienne Chauchot <ec...@gmail.com>
Authored: Mon Apr 3 15:18:04 2017 +0200
Committer: Ismaël Mejía <ie...@gmail.com>
Committed: Wed Aug 23 19:07:27 2017 +0200

----------------------------------------------------------------------
 .../nexmark/NexmarkConfiguration.java           |  19 +-
 .../integration/nexmark/NexmarkOptions.java     |   7 +
 .../integration/nexmark/queries/Query12.java    |  19 +-
 .../integration/nexmark/queries/Query3.java     | 263 +++++++++++--------
 .../integration/nexmark/queries/QueryTest.java  |   4 +
 5 files changed, 195 insertions(+), 117 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/beam/blob/7c28b492/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkConfiguration.java
----------------------------------------------------------------------
diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkConfiguration.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkConfiguration.java
index e2890ed..d6cd808 100644
--- a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkConfiguration.java
+++ b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkConfiguration.java
@@ -195,6 +195,13 @@ public class NexmarkConfiguration implements Serializable {
   public int fanout = 5;
 
   /**
+   * Maximum waiting time to clean personState in query3
+   * (ie maximum waiting of the auctions related to person in state in seconds in event time).
+   */
+  @JsonProperty
+  public int maxAuctionsWaitingTime = 600;
+
+  /**
    * Length of occasional delay to impose on events (in seconds).
    */
   @JsonProperty
@@ -322,6 +329,9 @@ public class NexmarkConfiguration implements Serializable {
     if (options.getFanout() != null) {
       fanout = options.getFanout();
     }
+    if (options.getMaxAuctionsWaitingTime() != null) {
+      fanout = options.getMaxAuctionsWaitingTime();
+    }
     if (options.getOccasionalDelaySec() != null) {
       occasionalDelaySec = options.getOccasionalDelaySec();
     }
@@ -376,6 +386,7 @@ public class NexmarkConfiguration implements Serializable {
     result.diskBusyBytes = diskBusyBytes;
     result.auctionSkip = auctionSkip;
     result.fanout = fanout;
+    result.maxAuctionsWaitingTime = maxAuctionsWaitingTime;
     result.occasionalDelaySec = occasionalDelaySec;
     result.probDelayedEvent = probDelayedEvent;
     result.maxLogEvents = maxLogEvents;
@@ -479,6 +490,9 @@ public class NexmarkConfiguration implements Serializable {
     if (fanout != DEFAULT.fanout) {
       sb.append(String.format("; fanout:%d", fanout));
     }
+    if (maxAuctionsWaitingTime != DEFAULT.maxAuctionsWaitingTime) {
+      sb.append(String.format("; maxAuctionsWaitingTime:%d", fanout));
+    }
     if (occasionalDelaySec != DEFAULT.occasionalDelaySec) {
       sb.append(String.format("; occasionalDelaySec:%d", occasionalDelaySec));
     }
@@ -527,7 +541,7 @@ public class NexmarkConfiguration implements Serializable {
         ratePeriodSec, preloadSeconds, isRateLimited, useWallclockEventTime, avgPersonByteSize,
         avgAuctionByteSize, avgBidByteSize, hotAuctionRatio, hotSellersRatio, hotBiddersRatio,
         windowSizeSec, windowPeriodSec, watermarkHoldbackSec, numInFlightAuctions, numActivePeople,
-        coderStrategy, cpuDelayMs, diskBusyBytes, auctionSkip, fanout,
+        coderStrategy, cpuDelayMs, diskBusyBytes, auctionSkip, fanout, maxAuctionsWaitingTime,
         occasionalDelaySec, probDelayedEvent, maxLogEvents, usePubsubPublishTime,
         outOfOrderGroupSize);
   }
@@ -571,6 +585,9 @@ public class NexmarkConfiguration implements Serializable {
     if (fanout != other.fanout) {
       return false;
     }
+    if (maxAuctionsWaitingTime != other.maxAuctionsWaitingTime) {
+      return false;
+    }
     if (firstEventRate != other.firstEventRate) {
       return false;
     }

http://git-wip-us.apache.org/repos/asf/beam/blob/7c28b492/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkOptions.java
----------------------------------------------------------------------
diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkOptions.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkOptions.java
index 1be974f..e39f0a4 100644
--- a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkOptions.java
+++ b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkOptions.java
@@ -309,6 +309,13 @@ public interface NexmarkOptions extends PubsubOptions {
 
   void setFanout(Integer fanout);
 
+  @Description("Maximum waiting time to clean personState in query3 "
+      + "(ie maximum waiting of the auctions related to person in state in seconds in event time).")
+  @Nullable
+  Integer getMaxAuctionsWaitingTime();
+
+  void setMaxAuctionsWaitingTime(Integer fanout);
+
   @Description("Length of occasional delay to impose on events (in seconds).")
   @Nullable
   Long getOccasionalDelaySec();

http://git-wip-us.apache.org/repos/asf/beam/blob/7c28b492/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query12.java
----------------------------------------------------------------------
diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query12.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query12.java
index c67401b..a5db504 100644
--- a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query12.java
+++ b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query12.java
@@ -49,16 +49,13 @@ public class Query12 extends NexmarkQuery {
   private PCollection<BidsPerSession> applyTyped(PCollection<Event> events) {
     return events
         .apply(JUST_BIDS)
-        .apply(name + ".Rekey",
-          // TODO etienne: why not avoid this ParDo and do a Cont.perElement?
-            ParDo.of(new DoFn<Bid, KV<Long, Void>>() {
-                   @ProcessElement
-                   public void processElement(ProcessContext c) {
-                     Bid bid = c.element();
-                     c.output(KV.of(bid.bidder, (Void) null));
-                   }
-                 }))
-        .apply(Window.<KV<Long, Void>>into(new GlobalWindows())
+        .apply(ParDo.of(new DoFn<Bid, Long>() {
+          @ProcessElement
+          public void processElement(ProcessContext c){
+            c.output(c.element().bidder);
+          }
+        }))
+        .apply(Window.<Long>into(new GlobalWindows())
             .triggering(
                 Repeatedly.forever(
                     AfterProcessingTime.pastFirstElementInPane()
@@ -66,7 +63,7 @@ public class Query12 extends NexmarkQuery {
                                            Duration.standardSeconds(configuration.windowSizeSec))))
             .discardingFiredPanes()
             .withAllowedLateness(Duration.ZERO))
-        .apply(Count.<Long, Void>perKey())
+        .apply(Count.<Long>perElement())
         .apply(name + ".ToResult",
             ParDo.of(new DoFn<KV<Long, Long>, BidsPerSession>() {
                    @ProcessElement

http://git-wip-us.apache.org/repos/asf/beam/blob/7c28b492/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query3.java
----------------------------------------------------------------------
diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query3.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query3.java
index 128c2b7..ba31e9f 100644
--- a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query3.java
+++ b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query3.java
@@ -39,14 +39,21 @@ import org.apache.beam.sdk.transforms.Sum;
 import org.apache.beam.sdk.transforms.join.CoGbkResult;
 import org.apache.beam.sdk.transforms.join.CoGroupByKey;
 import org.apache.beam.sdk.transforms.join.KeyedPCollectionTuple;
-import org.apache.beam.sdk.transforms.windowing.FixedWindows;
+import org.apache.beam.sdk.transforms.windowing.AfterPane;
+import org.apache.beam.sdk.transforms.windowing.GlobalWindows;
+import org.apache.beam.sdk.transforms.windowing.Repeatedly;
 import org.apache.beam.sdk.transforms.windowing.Window;
+import org.apache.beam.sdk.util.TimeDomain;
+import org.apache.beam.sdk.util.Timer;
+import org.apache.beam.sdk.util.TimerSpec;
+import org.apache.beam.sdk.util.TimerSpecs;
 import org.apache.beam.sdk.util.state.StateSpec;
 import org.apache.beam.sdk.util.state.StateSpecs;
 import org.apache.beam.sdk.util.state.ValueState;
 import org.apache.beam.sdk.values.KV;
 import org.apache.beam.sdk.values.PCollection;
 import org.joda.time.Duration;
+import org.joda.time.Instant;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -62,31 +69,141 @@ import org.slf4j.LoggerFactory;
  * </pre>
  *
  * <p>We'll implement this query to allow 'new auction' events to come before the 'new person'
- * events for the auction seller. Those auctions will be stored until the matching person is
- * seen. Then all subsequent auctions for a person will use the stored person record.
+ * events for the auction seller. Those auctions will be stored until the matching person is seen.
+ * Then all subsequent auctions for a person will use the stored person record.
  *
  * <p>A real system would use an external system to maintain the id-to-person association.
  */
 public class Query3 extends NexmarkQuery {
+
   private static final Logger LOG = LoggerFactory.getLogger(Query3.class);
-//  private static final StateContext GLOBAL_NAMESPACE = StateContexts.global();
-  private static final StateSpec<Object, ValueState<List<Auction>>> AUCTION_LIST_CODED_TAG =
-      StateSpecs.value(ListCoder.of(Auction.CODER));
-  private static final StateSpec<Object, ValueState<Person>> PERSON_CODED_TAG =
-      StateSpecs.value(Person.CODER);
+  private final JoinDoFn joinDoFn;
+
+  public Query3(NexmarkConfiguration configuration) {
+    super(configuration, "Query3");
+    joinDoFn = new JoinDoFn(configuration.maxAuctionsWaitingTime);
+
+  }
+
+  @Override
+  @Nullable
+  public Aggregator<Long, Long> getFatalCount() {
+    return joinDoFn.fatalCounter;
+  }
+
+  private PCollection<NameCityStateId> applyTyped(PCollection<Event> events) {
+    int numEventsInPane = 30;
+
+    PCollection<Event> eventsWindowed =
+        events.apply(
+            Window.<Event>into(new GlobalWindows())
+                .triggering(Repeatedly.forever((AfterPane.elementCountAtLeast(numEventsInPane))))
+                .discardingFiredPanes()
+                .withAllowedLateness(Duration.ZERO));
+    PCollection<KV<Long, Auction>> auctionsBySellerId =
+        eventsWindowed
+            // Only want the new auction events.
+            .apply(JUST_NEW_AUCTIONS)
+
+            // We only want auctions in category 10.
+            .apply(
+                name + ".InCategory",
+                Filter.by(
+                    new SerializableFunction<Auction, Boolean>() {
+
+                      @Override
+                      public Boolean apply(Auction auction) {
+                        return auction.category == 10;
+                      }
+                    }))
+
+            // Key auctions by their seller id.
+            .apply("AuctionBySeller", AUCTION_BY_SELLER);
+
+    PCollection<KV<Long, Person>> personsById =
+        eventsWindowed
+            // Only want the new people events.
+            .apply(JUST_NEW_PERSONS)
+
+            // We only want people in OR, ID, CA.
+            .apply(
+                name + ".InState",
+                Filter.by(
+                    new SerializableFunction<Person, Boolean>() {
+
+                      @Override
+                      public Boolean apply(Person person) {
+                        return person.state.equals("OR")
+                            || person.state.equals("ID")
+                            || person.state.equals("CA");
+                      }
+                    }))
+
+            // Key people by their id.
+            .apply("PersonById", PERSON_BY_ID);
+
+    return
+    // Join auctions and people.
+    // concatenate KeyedPCollections
+    KeyedPCollectionTuple.of(AUCTION_TAG, auctionsBySellerId)
+        .and(PERSON_TAG, personsById)
+        // group auctions and persons by personId
+        .apply(CoGroupByKey.<Long>create())
+        .apply(name + ".Join", ParDo.of(joinDoFn))
+
+        // Project what we want.
+        .apply(
+            name + ".Project",
+            ParDo.of(
+                new DoFn<KV<Auction, Person>, NameCityStateId>() {
+
+                  @ProcessElement
+                  public void processElement(ProcessContext c) {
+                    Auction auction = c.element().getKey();
+                    Person person = c.element().getValue();
+                    c.output(
+                        new NameCityStateId(person.name, person.city, person.state, auction.id));
+                  }
+                }));
+  }
+
+  @Override
+  protected PCollection<KnownSize> applyPrim(PCollection<Event> events) {
+    return NexmarkUtils.castToKnownSize(name, applyTyped(events));
+  }
 
   /**
-   * Join {@code auctions} and {@code people} by person id and emit their cross-product one pair
-   * at a time.
+   * Join {@code auctions} and {@code people} by person id and emit their cross-product one pair at
+   * a time.
    *
    * <p>We know a person may submit any number of auctions. Thus new person event must have the
    * person record stored in persistent state in order to match future auctions by that person.
    *
-   * <p>However we know that each auction is associated with at most one person, so only need
-   * to store auction records in persistent state until we have seen the corresponding person
-   * record. And of course may have already seen that record.
+   * <p>However we know that each auction is associated with at most one person, so only need to
+   * store auction records in persistent state until we have seen the corresponding person record.
+   * And of course may have already seen that record.
    */
   private static class JoinDoFn extends DoFn<KV<Long, CoGbkResult>, KV<Auction, Person>> {
+
+    private int maxAuctionsWaitingTime;
+    private static final String AUCTIONS = "auctions";
+    private static final String PERSON = "person";
+
+    @StateId(PERSON)
+    private static final StateSpec<Object, ValueState<Person>> personSpec =
+        StateSpecs.value(Person.CODER);
+
+    private static final String PERSON_STATE_EXPIRING = "personStateExpiring";
+
+    public final Aggregator<Long, Long> fatalCounter = createAggregator("fatal", Sum.ofLongs());
+
+    @StateId(AUCTIONS)
+    private final StateSpec<Object, ValueState<List<Auction>>> auctionsSpec =
+        StateSpecs.value(ListCoder.of(Auction.CODER));
+
+    @TimerId(PERSON_STATE_EXPIRING)
+    private final TimerSpec timerSpec = TimerSpecs.timer(TimeDomain.EVENT_TIME);
+
     private final Aggregator<Long, Long> newAuctionCounter =
         createAggregator("newAuction", Sum.ofLongs());
     private final Aggregator<Long, Long> newPersonCounter =
@@ -97,20 +214,25 @@ public class Query3 extends NexmarkQuery {
         createAggregator("newOldOutput", Sum.ofLongs());
     private final Aggregator<Long, Long> oldNewOutputCounter =
         createAggregator("oldNewOutput", Sum.ofLongs());
-    public final Aggregator<Long, Long> fatalCounter = createAggregator("fatal", Sum.ofLongs());
+
+    private JoinDoFn(int maxAuctionsWaitingTime) {
+      this.maxAuctionsWaitingTime = maxAuctionsWaitingTime;
+    }
 
     @ProcessElement
-    public void processElement(ProcessContext c) throws IOException {
-      //TODO: This is using the internal state API. Rework to use the
-      //TODO Ismael this is broken for not access to state
+    public void processElement(
+        ProcessContext c,
+        @TimerId(PERSON_STATE_EXPIRING) Timer timer,
+        @StateId(PERSON) ValueState<Person> personState,
+        @StateId(AUCTIONS) ValueState<List<Auction>> auctionsState)
+        throws IOException {
       // We would *almost* implement this by  rewindowing into the global window and
       // running a combiner over the result. The combiner's accumulator would be the
       // state we use below. However, combiners cannot emit intermediate results, thus
       // we need to wait for the pending ReduceFn API.
-//      StateInternals<?> stateInternals = c.windowingInternals().stateInternals();
-//      ValueState<Person> personState = stateInternals.state(GLOBAL_NAMESPACE, PERSON_CODED_TAG);
-//      Person existingPerson = personState.read();
-      Person existingPerson = null;
+
+      Person existingPerson = personState.read();
+
       if (existingPerson != null) {
         // We've already seen the new person event for this person id.
         // We can join with any new auctions on-the-fly without needing any
@@ -123,8 +245,6 @@ public class Query3 extends NexmarkQuery {
         return;
       }
 
-//      ValueState<List<Auction>> auctionsState =
-//          stateInternals.state(GLOBAL_NAMESPACE, AUCTION_LIST_CODED_TAG);
       Person theNewPerson = null;
       for (Person newPerson : c.element().getValue().getAll(PERSON_TAG)) {
         if (theNewPerson == null) {
@@ -140,14 +260,14 @@ public class Query3 extends NexmarkQuery {
         }
         newPersonCounter.addValue(1L);
         // We've now seen the person for this person id so can flush any
-        // pending auctions for the same seller id.
-        List<Auction> pendingAuctions = null; //auctionsState.read();
+        // pending auctions for the same seller id (an auction is done by only one seller).
+        List<Auction> pendingAuctions = auctionsState.read();
         if (pendingAuctions != null) {
           for (Auction pendingAuction : pendingAuctions) {
             oldNewOutputCounter.addValue(1L);
             c.output(KV.of(pendingAuction, newPerson));
           }
-//          auctionsState.clear();
+          auctionsState.clear();
         }
         // Also deal with any new auctions.
         for (Auction newAuction : c.element().getValue().getAll(AUCTION_TAG)) {
@@ -156,8 +276,11 @@ public class Query3 extends NexmarkQuery {
           c.output(KV.of(newAuction, newPerson));
         }
         // Remember this person for any future auctions.
-
-//        personState.write(newPerson);
+        personState.write(newPerson);
+        //set a time out to clear this state
+        Instant firingTime = new Instant(newPerson.dateTime)
+                                  .plus(Duration.standardSeconds(maxAuctionsWaitingTime));
+        timer.set(firingTime);
       }
       if (theNewPerson != null) {
         return;
@@ -165,7 +288,7 @@ public class Query3 extends NexmarkQuery {
 
       // We'll need to remember the auctions until we see the corresponding
       // new person event.
-      List<Auction> pendingAuctions = null; //auctionsState.read();
+      List<Auction> pendingAuctions = auctionsState.read();
       if (pendingAuctions == null) {
         pendingAuctions = new ArrayList<>();
       }
@@ -173,84 +296,14 @@ public class Query3 extends NexmarkQuery {
         newAuctionCounter.addValue(1L);
         pendingAuctions.add(newAuction);
       }
-//      auctionsState.write(pendingAuctions);
+      auctionsState.write(pendingAuctions);
     }
+  @OnTimer(PERSON_STATE_EXPIRING)
+  public void onTimerCallback(
+      OnTimerContext context,
+      @StateId(PERSON) ValueState<Person> personState) {
+      personState.clear();
   }
 
-  private final JoinDoFn joinDoFn = new JoinDoFn();
-
-  public Query3(NexmarkConfiguration configuration) {
-    super(configuration, "Query3");
-  }
-
-  @Override
-  @Nullable
-  public Aggregator<Long, Long> getFatalCount() {
-    return joinDoFn.fatalCounter;
-  }
-
-  private PCollection<NameCityStateId> applyTyped(PCollection<Event> events) {
-    // Batch into incremental results windows.
-    events = events.apply(
-        Window.<Event>into(FixedWindows.of(Duration.standardSeconds(configuration.windowSizeSec))));
-
-    PCollection<KV<Long, Auction>> auctionsBySellerId =
-        events
-            // Only want the new auction events.
-            .apply(JUST_NEW_AUCTIONS)
-
-            // We only want auctions in category 10.
-            .apply(name + ".InCategory", Filter.by(new SerializableFunction<Auction, Boolean>() {
-              @Override
-              public Boolean apply(Auction auction) {
-                return auction.category == 10;
-              }
-            }))
-
-            // Key auctions by their seller id.
-            .apply("AuctionBySeller", AUCTION_BY_SELLER);
-
-    PCollection<KV<Long, Person>> personsById =
-        events
-            // Only want the new people events.
-            .apply(JUST_NEW_PERSONS)
-
-            // We only want people in OR, ID, CA.
-            .apply(name + ".InState", Filter.by(new SerializableFunction<Person, Boolean>() {
-              @Override
-              public Boolean apply(Person person) {
-                return person.state.equals("OR") || person.state.equals("ID")
-                    || person.state.equals("CA");
-              }
-            }))
-
-            // Key people by their id.
-            .apply("PersonById", PERSON_BY_ID);
-
-    return
-      // Join auctions and people.
-        // concatenate KeyedPCollections
-      KeyedPCollectionTuple.of(AUCTION_TAG, auctionsBySellerId)
-            .and(PERSON_TAG, personsById)
-        // group auctions and persons by personId
-        .apply(CoGroupByKey.<Long>create())
-            .apply(name + ".Join", ParDo.of(joinDoFn))
-
-            // Project what we want.
-            .apply(name + ".Project",
-                ParDo.of(new DoFn<KV<Auction, Person>, NameCityStateId>() {
-                      @ProcessElement
-                      public void processElement(ProcessContext c) {
-                        Auction auction = c.element().getKey();
-                        Person person = c.element().getValue();
-                        c.output(new NameCityStateId(
-                            person.name, person.city, person.state, auction.id));
-                      }
-                    }));
-  }
-
-  @Override
-  protected PCollection<KnownSize> applyPrim(PCollection<Event> events) {
-    return NexmarkUtils.castToKnownSize(name, applyTyped(events));
   }
 }

http://git-wip-us.apache.org/repos/asf/beam/blob/7c28b492/integration/java/nexmark/src/test/java/org/apache/beam/integration/nexmark/queries/QueryTest.java
----------------------------------------------------------------------
diff --git a/integration/java/nexmark/src/test/java/org/apache/beam/integration/nexmark/queries/QueryTest.java b/integration/java/nexmark/src/test/java/org/apache/beam/integration/nexmark/queries/QueryTest.java
index 5cf4287..dca2887 100644
--- a/integration/java/nexmark/src/test/java/org/apache/beam/integration/nexmark/queries/QueryTest.java
+++ b/integration/java/nexmark/src/test/java/org/apache/beam/integration/nexmark/queries/QueryTest.java
@@ -25,10 +25,13 @@ import org.apache.beam.integration.nexmark.model.KnownSize;
 import org.apache.beam.sdk.PipelineResult;
 import org.apache.beam.sdk.testing.PAssert;
 import org.apache.beam.sdk.testing.TestPipeline;
+import org.apache.beam.sdk.testing.UsesStatefulParDo;
+import org.apache.beam.sdk.testing.UsesTimersInParDo;
 import org.apache.beam.sdk.values.PCollection;
 import org.apache.beam.sdk.values.TimestampedValue;
 import org.junit.Rule;
 import org.junit.Test;
+import org.junit.experimental.categories.Category;
 import org.junit.runner.RunWith;
 import org.junit.runners.JUnit4;
 
@@ -95,6 +98,7 @@ public class QueryTest {
   }
 
   @Test
+  @Category({UsesStatefulParDo.class, UsesTimersInParDo.class})
   public void query7MatchesModel() {
     queryMatchesModel("Query7Test", new Query7(CONFIG), new Query7Model(CONFIG));
   }


[10/55] [abbrv] beam git commit: Refactor classes into packages

Posted by ie...@apache.org.
http://git-wip-us.apache.org/repos/asf/beam/blob/a7f9f7d0/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query3.java
----------------------------------------------------------------------
diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query3.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query3.java
new file mode 100644
index 0000000..128c2b7
--- /dev/null
+++ b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query3.java
@@ -0,0 +1,256 @@
+/*
+ * 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.beam.integration.nexmark.queries;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+import javax.annotation.Nullable;
+import org.apache.beam.integration.nexmark.NexmarkConfiguration;
+import org.apache.beam.integration.nexmark.NexmarkQuery;
+import org.apache.beam.integration.nexmark.NexmarkUtils;
+import org.apache.beam.integration.nexmark.model.Auction;
+import org.apache.beam.integration.nexmark.model.Event;
+import org.apache.beam.integration.nexmark.model.KnownSize;
+import org.apache.beam.integration.nexmark.model.NameCityStateId;
+import org.apache.beam.integration.nexmark.model.Person;
+import org.apache.beam.sdk.coders.ListCoder;
+import org.apache.beam.sdk.transforms.Aggregator;
+import org.apache.beam.sdk.transforms.DoFn;
+import org.apache.beam.sdk.transforms.Filter;
+import org.apache.beam.sdk.transforms.ParDo;
+import org.apache.beam.sdk.transforms.SerializableFunction;
+import org.apache.beam.sdk.transforms.Sum;
+import org.apache.beam.sdk.transforms.join.CoGbkResult;
+import org.apache.beam.sdk.transforms.join.CoGroupByKey;
+import org.apache.beam.sdk.transforms.join.KeyedPCollectionTuple;
+import org.apache.beam.sdk.transforms.windowing.FixedWindows;
+import org.apache.beam.sdk.transforms.windowing.Window;
+import org.apache.beam.sdk.util.state.StateSpec;
+import org.apache.beam.sdk.util.state.StateSpecs;
+import org.apache.beam.sdk.util.state.ValueState;
+import org.apache.beam.sdk.values.KV;
+import org.apache.beam.sdk.values.PCollection;
+import org.joda.time.Duration;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Query 3, 'Local Item Suggestion'. Who is selling in OR, ID or CA in category 10, and for what
+ * auction ids? In CQL syntax:
+ *
+ * <pre>
+ * SELECT Istream(P.name, P.city, P.state, A.id)
+ * FROM Auction A [ROWS UNBOUNDED], Person P [ROWS UNBOUNDED]
+ * WHERE A.seller = P.id AND (P.state = `OR' OR P.state = `ID' OR P.state = `CA') AND A.category
+ * = 10;
+ * </pre>
+ *
+ * <p>We'll implement this query to allow 'new auction' events to come before the 'new person'
+ * events for the auction seller. Those auctions will be stored until the matching person is
+ * seen. Then all subsequent auctions for a person will use the stored person record.
+ *
+ * <p>A real system would use an external system to maintain the id-to-person association.
+ */
+public class Query3 extends NexmarkQuery {
+  private static final Logger LOG = LoggerFactory.getLogger(Query3.class);
+//  private static final StateContext GLOBAL_NAMESPACE = StateContexts.global();
+  private static final StateSpec<Object, ValueState<List<Auction>>> AUCTION_LIST_CODED_TAG =
+      StateSpecs.value(ListCoder.of(Auction.CODER));
+  private static final StateSpec<Object, ValueState<Person>> PERSON_CODED_TAG =
+      StateSpecs.value(Person.CODER);
+
+  /**
+   * Join {@code auctions} and {@code people} by person id and emit their cross-product one pair
+   * at a time.
+   *
+   * <p>We know a person may submit any number of auctions. Thus new person event must have the
+   * person record stored in persistent state in order to match future auctions by that person.
+   *
+   * <p>However we know that each auction is associated with at most one person, so only need
+   * to store auction records in persistent state until we have seen the corresponding person
+   * record. And of course may have already seen that record.
+   */
+  private static class JoinDoFn extends DoFn<KV<Long, CoGbkResult>, KV<Auction, Person>> {
+    private final Aggregator<Long, Long> newAuctionCounter =
+        createAggregator("newAuction", Sum.ofLongs());
+    private final Aggregator<Long, Long> newPersonCounter =
+        createAggregator("newPerson", Sum.ofLongs());
+    private final Aggregator<Long, Long> newNewOutputCounter =
+        createAggregator("newNewOutput", Sum.ofLongs());
+    private final Aggregator<Long, Long> newOldOutputCounter =
+        createAggregator("newOldOutput", Sum.ofLongs());
+    private final Aggregator<Long, Long> oldNewOutputCounter =
+        createAggregator("oldNewOutput", Sum.ofLongs());
+    public final Aggregator<Long, Long> fatalCounter = createAggregator("fatal", Sum.ofLongs());
+
+    @ProcessElement
+    public void processElement(ProcessContext c) throws IOException {
+      //TODO: This is using the internal state API. Rework to use the
+      //TODO Ismael this is broken for not access to state
+      // We would *almost* implement this by  rewindowing into the global window and
+      // running a combiner over the result. The combiner's accumulator would be the
+      // state we use below. However, combiners cannot emit intermediate results, thus
+      // we need to wait for the pending ReduceFn API.
+//      StateInternals<?> stateInternals = c.windowingInternals().stateInternals();
+//      ValueState<Person> personState = stateInternals.state(GLOBAL_NAMESPACE, PERSON_CODED_TAG);
+//      Person existingPerson = personState.read();
+      Person existingPerson = null;
+      if (existingPerson != null) {
+        // We've already seen the new person event for this person id.
+        // We can join with any new auctions on-the-fly without needing any
+        // additional persistent state.
+        for (Auction newAuction : c.element().getValue().getAll(AUCTION_TAG)) {
+          newAuctionCounter.addValue(1L);
+          newOldOutputCounter.addValue(1L);
+          c.output(KV.of(newAuction, existingPerson));
+        }
+        return;
+      }
+
+//      ValueState<List<Auction>> auctionsState =
+//          stateInternals.state(GLOBAL_NAMESPACE, AUCTION_LIST_CODED_TAG);
+      Person theNewPerson = null;
+      for (Person newPerson : c.element().getValue().getAll(PERSON_TAG)) {
+        if (theNewPerson == null) {
+          theNewPerson = newPerson;
+        } else {
+          if (theNewPerson.equals(newPerson)) {
+            LOG.error("**** duplicate person {} ****", theNewPerson);
+          } else {
+            LOG.error("**** conflicting persons {} and {} ****", theNewPerson, newPerson);
+          }
+          fatalCounter.addValue(1L);
+          continue;
+        }
+        newPersonCounter.addValue(1L);
+        // We've now seen the person for this person id so can flush any
+        // pending auctions for the same seller id.
+        List<Auction> pendingAuctions = null; //auctionsState.read();
+        if (pendingAuctions != null) {
+          for (Auction pendingAuction : pendingAuctions) {
+            oldNewOutputCounter.addValue(1L);
+            c.output(KV.of(pendingAuction, newPerson));
+          }
+//          auctionsState.clear();
+        }
+        // Also deal with any new auctions.
+        for (Auction newAuction : c.element().getValue().getAll(AUCTION_TAG)) {
+          newAuctionCounter.addValue(1L);
+          newNewOutputCounter.addValue(1L);
+          c.output(KV.of(newAuction, newPerson));
+        }
+        // Remember this person for any future auctions.
+
+//        personState.write(newPerson);
+      }
+      if (theNewPerson != null) {
+        return;
+      }
+
+      // We'll need to remember the auctions until we see the corresponding
+      // new person event.
+      List<Auction> pendingAuctions = null; //auctionsState.read();
+      if (pendingAuctions == null) {
+        pendingAuctions = new ArrayList<>();
+      }
+      for (Auction newAuction : c.element().getValue().getAll(AUCTION_TAG)) {
+        newAuctionCounter.addValue(1L);
+        pendingAuctions.add(newAuction);
+      }
+//      auctionsState.write(pendingAuctions);
+    }
+  }
+
+  private final JoinDoFn joinDoFn = new JoinDoFn();
+
+  public Query3(NexmarkConfiguration configuration) {
+    super(configuration, "Query3");
+  }
+
+  @Override
+  @Nullable
+  public Aggregator<Long, Long> getFatalCount() {
+    return joinDoFn.fatalCounter;
+  }
+
+  private PCollection<NameCityStateId> applyTyped(PCollection<Event> events) {
+    // Batch into incremental results windows.
+    events = events.apply(
+        Window.<Event>into(FixedWindows.of(Duration.standardSeconds(configuration.windowSizeSec))));
+
+    PCollection<KV<Long, Auction>> auctionsBySellerId =
+        events
+            // Only want the new auction events.
+            .apply(JUST_NEW_AUCTIONS)
+
+            // We only want auctions in category 10.
+            .apply(name + ".InCategory", Filter.by(new SerializableFunction<Auction, Boolean>() {
+              @Override
+              public Boolean apply(Auction auction) {
+                return auction.category == 10;
+              }
+            }))
+
+            // Key auctions by their seller id.
+            .apply("AuctionBySeller", AUCTION_BY_SELLER);
+
+    PCollection<KV<Long, Person>> personsById =
+        events
+            // Only want the new people events.
+            .apply(JUST_NEW_PERSONS)
+
+            // We only want people in OR, ID, CA.
+            .apply(name + ".InState", Filter.by(new SerializableFunction<Person, Boolean>() {
+              @Override
+              public Boolean apply(Person person) {
+                return person.state.equals("OR") || person.state.equals("ID")
+                    || person.state.equals("CA");
+              }
+            }))
+
+            // Key people by their id.
+            .apply("PersonById", PERSON_BY_ID);
+
+    return
+      // Join auctions and people.
+        // concatenate KeyedPCollections
+      KeyedPCollectionTuple.of(AUCTION_TAG, auctionsBySellerId)
+            .and(PERSON_TAG, personsById)
+        // group auctions and persons by personId
+        .apply(CoGroupByKey.<Long>create())
+            .apply(name + ".Join", ParDo.of(joinDoFn))
+
+            // Project what we want.
+            .apply(name + ".Project",
+                ParDo.of(new DoFn<KV<Auction, Person>, NameCityStateId>() {
+                      @ProcessElement
+                      public void processElement(ProcessContext c) {
+                        Auction auction = c.element().getKey();
+                        Person person = c.element().getValue();
+                        c.output(new NameCityStateId(
+                            person.name, person.city, person.state, auction.id));
+                      }
+                    }));
+  }
+
+  @Override
+  protected PCollection<KnownSize> applyPrim(PCollection<Event> events) {
+    return NexmarkUtils.castToKnownSize(name, applyTyped(events));
+  }
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/a7f9f7d0/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query3Model.java
----------------------------------------------------------------------
diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query3Model.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query3Model.java
new file mode 100644
index 0000000..e4b72d2
--- /dev/null
+++ b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query3Model.java
@@ -0,0 +1,126 @@
+/*
+ * 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.beam.integration.nexmark.queries;
+
+import com.google.common.collect.ArrayListMultimap;
+import com.google.common.collect.Multimap;
+
+import java.io.Serializable;
+import java.util.Collection;
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.Map;
+
+import org.apache.beam.integration.nexmark.AbstractSimulator;
+import org.apache.beam.integration.nexmark.NexmarkConfiguration;
+import org.apache.beam.integration.nexmark.NexmarkQueryModel;
+import org.apache.beam.integration.nexmark.NexmarkUtils;
+import org.apache.beam.integration.nexmark.model.Auction;
+import org.apache.beam.integration.nexmark.model.Event;
+import org.apache.beam.integration.nexmark.model.NameCityStateId;
+import org.apache.beam.integration.nexmark.model.Person;
+import org.apache.beam.sdk.values.TimestampedValue;
+import org.joda.time.Instant;
+
+/**
+ * A direct implementation of {@link Query3}.
+ */
+public class Query3Model extends NexmarkQueryModel implements Serializable {
+  /**
+   * Simulator for query 3.
+   */
+  private class Simulator extends AbstractSimulator<Event, NameCityStateId> {
+    /** Auctions, indexed by seller id. */
+    private final Multimap<Long, Auction> newAuctions;
+
+    /** Persons, indexed by id. */
+    private final Map<Long, Person> newPersons;
+
+    public Simulator(NexmarkConfiguration configuration) {
+      super(NexmarkUtils.standardEventIterator(configuration));
+      newPersons = new HashMap<>();
+      newAuctions = ArrayListMultimap.create();
+    }
+
+    /**
+     * Capture new result.
+     */
+    private void addResult(Auction auction, Person person, Instant timestamp) {
+      TimestampedValue<NameCityStateId> result = TimestampedValue.of(
+          new NameCityStateId(person.name, person.city, person.state, auction.id), timestamp);
+      addResult(result);
+    }
+
+    @Override
+    protected void run() {
+      TimestampedValue<Event> timestampedEvent = nextInput();
+      if (timestampedEvent == null) {
+        allDone();
+        return;
+      }
+      Event event = timestampedEvent.getValue();
+      if (event.bid != null) {
+        // Ignore bid events.
+        return;
+      }
+
+      Instant timestamp = timestampedEvent.getTimestamp();
+
+      if (event.newAuction != null) {
+        // Only want auctions in category 10.
+        if (event.newAuction.category == 10) {
+          // Join new auction with existing person, if any.
+          Person person = newPersons.get(event.newAuction.seller);
+          if (person != null) {
+            addResult(event.newAuction, person, timestamp);
+          } else {
+            // Remember auction for future new person event.
+            newAuctions.put(event.newAuction.seller, event.newAuction);
+          }
+        }
+      } else {
+        // Only want people in OR, ID or CA.
+        if (event.newPerson.state.equals("OR") || event.newPerson.state.equals("ID")
+            || event.newPerson.state.equals("CA")) {
+          // Join new person with existing auctions.
+          for (Auction auction : newAuctions.get(event.newPerson.id)) {
+            addResult(auction, event.newPerson, timestamp);
+          }
+          // We'll never need these auctions again.
+          newAuctions.removeAll(event.newPerson.id);
+          // Remember person for future auctions.
+          newPersons.put(event.newPerson.id, event.newPerson);
+        }
+      }
+    }
+  }
+
+  public Query3Model(NexmarkConfiguration configuration) {
+    super(configuration);
+  }
+
+  @Override
+  public AbstractSimulator<?, ?> simulator() {
+    return new Simulator(configuration);
+  }
+
+  @Override
+  protected <T> Collection<String> toCollection(Iterator<TimestampedValue<T>> itr) {
+    return toValue(itr);
+  }
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/a7f9f7d0/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query4.java
----------------------------------------------------------------------
diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query4.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query4.java
new file mode 100644
index 0000000..61991c8
--- /dev/null
+++ b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query4.java
@@ -0,0 +1,118 @@
+/*
+ * 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.beam.integration.nexmark.queries;
+
+import org.apache.beam.integration.nexmark.Monitor;
+import org.apache.beam.integration.nexmark.NexmarkConfiguration;
+import org.apache.beam.integration.nexmark.NexmarkQuery;
+import org.apache.beam.integration.nexmark.NexmarkUtils;
+import org.apache.beam.integration.nexmark.WinningBids;
+import org.apache.beam.integration.nexmark.model.Auction;
+import org.apache.beam.integration.nexmark.model.AuctionBid;
+import org.apache.beam.integration.nexmark.model.Bid;
+import org.apache.beam.integration.nexmark.model.CategoryPrice;
+import org.apache.beam.integration.nexmark.model.Event;
+import org.apache.beam.integration.nexmark.model.KnownSize;
+import org.apache.beam.sdk.transforms.DoFn;
+import org.apache.beam.sdk.transforms.Mean;
+import org.apache.beam.sdk.transforms.ParDo;
+import org.apache.beam.sdk.transforms.windowing.SlidingWindows;
+import org.apache.beam.sdk.transforms.windowing.Window;
+import org.apache.beam.sdk.values.KV;
+import org.apache.beam.sdk.values.PCollection;
+import org.joda.time.Duration;
+
+/**
+ * Query 4, 'Average Price for a Category'. Select the average of the wining bid prices for all
+ * closed auctions in each category. In CQL syntax:
+ *
+ * <pre>{@code
+ * SELECT Istream(AVG(Q.final))
+ * FROM Category C, (SELECT Rstream(MAX(B.price) AS final, A.category)
+ *                   FROM Auction A [ROWS UNBOUNDED], Bid B [ROWS UNBOUNDED]
+ *                   WHERE A.id=B.auction AND B.datetime < A.expires AND A.expires < CURRENT_TIME
+ *                   GROUP BY A.id, A.category) Q
+ * WHERE Q.category = C.id
+ * GROUP BY C.id;
+ * }</pre>
+ *
+ * <p>For extra spiciness our implementation differs slightly from the above:
+ * <ul>
+ * <li>We select both the average winning price and the category.
+ * <li>We don't bother joining with a static category table, since it's contents are never used.
+ * <li>We only consider bids which are above the auction's reserve price.
+ * <li>We accept the highest-price, earliest valid bid as the winner.
+ * <li>We calculate the averages oven a sliding window of size {@code windowSizeSec} and
+ * period {@code windowPeriodSec}.
+ * </ul>
+ */
+public class Query4 extends NexmarkQuery {
+  private final Monitor<AuctionBid> winningBidsMonitor;
+
+  public Query4(NexmarkConfiguration configuration) {
+    super(configuration, "Query4");
+    winningBidsMonitor = new Monitor<>(name + ".WinningBids", "winning");
+  }
+
+  private PCollection<CategoryPrice> applyTyped(PCollection<Event> events) {
+    PCollection<AuctionBid> winningBids =
+        events
+            // Find the winning bid for each closed auction.
+            .apply(new WinningBids(name + ".WinningBids", configuration));
+
+    // Monitor winning bids
+    winningBids = winningBids.apply(name + ".WinningBidsMonitor",
+            winningBidsMonitor.getTransform());
+
+    return winningBids
+        // Key the winning bid price by the auction category.
+        .apply(name + ".Rekey",
+            ParDo.of(new DoFn<AuctionBid, KV<Long, Long>>() {
+                  @ProcessElement
+                  public void processElement(ProcessContext c) {
+                    Auction auction = c.element().auction;
+                    Bid bid = c.element().bid;
+                    c.output(KV.of(auction.category, bid.price));
+                  }
+                }))
+
+        // Re-window so we can calculate a sliding average
+        .apply(Window.<KV<Long, Long>>into(
+            SlidingWindows.of(Duration.standardSeconds(configuration.windowSizeSec))
+                .every(Duration.standardSeconds(configuration.windowPeriodSec))))
+
+        // Find the average of the winning bids for each category.
+        // Make sure we share the work for each category between workers.
+        .apply(Mean.<Long, Long>perKey().withHotKeyFanout(configuration.fanout))
+
+        // For testing against Query4Model, capture which results are 'final'.
+        .apply(name + ".Project",
+            ParDo.of(new DoFn<KV<Long, Double>, CategoryPrice>() {
+                  @ProcessElement
+                  public void processElement(ProcessContext c) {
+                    c.output(new CategoryPrice(c.element().getKey(),
+                        Math.round(c.element().getValue()), c.pane().isLast()));
+                  }
+                }));
+  }
+
+  @Override
+  protected PCollection<KnownSize> applyPrim(PCollection<Event> events) {
+    return NexmarkUtils.castToKnownSize(name, applyTyped(events));
+  }
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/a7f9f7d0/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query4Model.java
----------------------------------------------------------------------
diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query4Model.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query4Model.java
new file mode 100644
index 0000000..9405ac8
--- /dev/null
+++ b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query4Model.java
@@ -0,0 +1,188 @@
+/*
+ * 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.beam.integration.nexmark.queries;
+
+import java.io.Serializable;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.TreeMap;
+import org.apache.beam.integration.nexmark.AbstractSimulator;
+import org.apache.beam.integration.nexmark.NexmarkConfiguration;
+import org.apache.beam.integration.nexmark.NexmarkQueryModel;
+import org.apache.beam.integration.nexmark.NexmarkUtils;
+import org.apache.beam.integration.nexmark.WinningBidsSimulator;
+import org.apache.beam.integration.nexmark.model.Auction;
+import org.apache.beam.integration.nexmark.model.AuctionBid;
+import org.apache.beam.integration.nexmark.model.Bid;
+import org.apache.beam.integration.nexmark.model.CategoryPrice;
+import org.apache.beam.integration.nexmark.model.KnownSize;
+import org.apache.beam.sdk.transforms.windowing.BoundedWindow;
+import org.apache.beam.sdk.values.TimestampedValue;
+import org.joda.time.Duration;
+import org.joda.time.Instant;
+import org.junit.Assert;
+
+/**
+ * A direct implementation of {@link Query4}.
+ */
+public class Query4Model extends NexmarkQueryModel implements Serializable {
+  /**
+   * Simulator for query 4.
+   */
+  private class Simulator extends AbstractSimulator<AuctionBid, CategoryPrice> {
+    /** The prices and categories for all winning bids in the last window size. */
+    private final List<TimestampedValue<CategoryPrice>> winningPricesByCategory;
+
+    /** Timestamp of last result (ms since epoch). */
+    private Instant lastTimestamp;
+
+    /** When oldest active window starts. */
+    private Instant windowStart;
+
+    /** The last seen result for each category. */
+    private final Map<Long, TimestampedValue<CategoryPrice>> lastSeenResults;
+
+    public Simulator(NexmarkConfiguration configuration) {
+      super(new WinningBidsSimulator(configuration).results());
+      winningPricesByCategory = new ArrayList<>();
+      lastTimestamp = BoundedWindow.TIMESTAMP_MIN_VALUE;
+      windowStart = NexmarkUtils.BEGINNING_OF_TIME;
+      lastSeenResults = new TreeMap<>();
+    }
+
+    /**
+     * Calculate the average bid price for each category for all winning bids
+     * which are strictly before {@code end}.
+     */
+    private void averages(Instant end) {
+      Map<Long, Long> counts = new TreeMap<>();
+      Map<Long, Long> totals = new TreeMap<>();
+      for (TimestampedValue<CategoryPrice> value : winningPricesByCategory) {
+        if (!value.getTimestamp().isBefore(end)) {
+          continue;
+        }
+        long category = value.getValue().category;
+        long price = value.getValue().price;
+        Long count = counts.get(category);
+        if (count == null) {
+          count = 1L;
+        } else {
+          count += 1;
+        }
+        counts.put(category, count);
+        Long total = totals.get(category);
+        if (total == null) {
+          total = price;
+        } else {
+          total += price;
+        }
+        totals.put(category, total);
+      }
+      for (long category : counts.keySet()) {
+        long count = counts.get(category);
+        long total = totals.get(category);
+        TimestampedValue<CategoryPrice> result = TimestampedValue.of(
+            new CategoryPrice(category, Math.round((double) total / count), true), lastTimestamp);
+        addIntermediateResult(result);
+        lastSeenResults.put(category, result);
+      }
+    }
+
+    /**
+     * Calculate averages for any windows which can now be retired. Also prune entries
+     * which can no longer contribute to any future window.
+     */
+    private void prune(Instant newWindowStart) {
+      while (!newWindowStart.equals(windowStart)) {
+        averages(windowStart.plus(Duration.standardSeconds(configuration.windowSizeSec)));
+        windowStart = windowStart.plus(Duration.standardSeconds(configuration.windowPeriodSec));
+        Iterator<TimestampedValue<CategoryPrice>> itr = winningPricesByCategory.iterator();
+        while (itr.hasNext()) {
+          if (itr.next().getTimestamp().isBefore(windowStart)) {
+            itr.remove();
+          }
+        }
+        if (winningPricesByCategory.isEmpty()) {
+          windowStart = newWindowStart;
+        }
+      }
+    }
+
+    /**
+     * Capture the winning bid.
+     */
+    private void captureWinningBid(Auction auction, Bid bid, Instant timestamp) {
+      winningPricesByCategory.add(
+          TimestampedValue.of(new CategoryPrice(auction.category, bid.price, false), timestamp));
+    }
+
+    @Override
+    protected void run() {
+      TimestampedValue<AuctionBid> timestampedWinningBid = nextInput();
+      if (timestampedWinningBid == null) {
+        prune(NexmarkUtils.END_OF_TIME);
+        for (TimestampedValue<CategoryPrice> result : lastSeenResults.values()) {
+          addResult(result);
+        }
+        allDone();
+        return;
+      }
+      lastTimestamp = timestampedWinningBid.getTimestamp();
+      Instant newWindowStart = windowStart(Duration.standardSeconds(configuration.windowSizeSec),
+          Duration.standardSeconds(configuration.windowPeriodSec), lastTimestamp);
+      prune(newWindowStart);
+      captureWinningBid(timestampedWinningBid.getValue().auction,
+          timestampedWinningBid.getValue().bid, lastTimestamp);
+    }
+  }
+
+  public Query4Model(NexmarkConfiguration configuration) {
+    super(configuration);
+  }
+
+  @Override
+  public AbstractSimulator<?, ?> simulator() {
+    return new Simulator(configuration);
+  }
+
+  @Override
+  protected Iterable<TimestampedValue<KnownSize>> relevantResults(
+      Iterable<TimestampedValue<KnownSize>> results) {
+    // Find the last (in processing time) reported average price for each category.
+    Map<Long, TimestampedValue<KnownSize>> finalAverages = new TreeMap<>();
+    for (TimestampedValue<KnownSize> obj : results) {
+      Assert.assertTrue("have CategoryPrice", obj.getValue() instanceof CategoryPrice);
+      CategoryPrice categoryPrice = (CategoryPrice) obj.getValue();
+      if (categoryPrice.isLast) {
+        finalAverages.put(
+            categoryPrice.category,
+            TimestampedValue.of((KnownSize) categoryPrice, obj.getTimestamp()));
+      }
+    }
+
+    return finalAverages.values();
+  }
+
+  @Override
+  protected <T> Collection<String> toCollection(Iterator<TimestampedValue<T>> itr) {
+    return toValue(itr);
+  }
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/a7f9f7d0/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query5.java
----------------------------------------------------------------------
diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query5.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query5.java
new file mode 100644
index 0000000..9f02ddb
--- /dev/null
+++ b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query5.java
@@ -0,0 +1,129 @@
+/*
+ * 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.beam.integration.nexmark.queries;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.List;
+import org.apache.beam.integration.nexmark.NexmarkConfiguration;
+import org.apache.beam.integration.nexmark.NexmarkQuery;
+import org.apache.beam.integration.nexmark.NexmarkUtils;
+import org.apache.beam.integration.nexmark.model.AuctionCount;
+import org.apache.beam.integration.nexmark.model.Bid;
+import org.apache.beam.integration.nexmark.model.Event;
+import org.apache.beam.integration.nexmark.model.KnownSize;
+import org.apache.beam.sdk.transforms.Combine;
+import org.apache.beam.sdk.transforms.Count;
+import org.apache.beam.sdk.transforms.DoFn;
+import org.apache.beam.sdk.transforms.ParDo;
+import org.apache.beam.sdk.transforms.windowing.SlidingWindows;
+import org.apache.beam.sdk.transforms.windowing.Window;
+import org.apache.beam.sdk.values.KV;
+import org.apache.beam.sdk.values.PCollection;
+import org.joda.time.Duration;
+
+/**
+ * Query 5, 'Hot Items'. Which auctions have seen the most bids in the last hour (updated every
+ * minute). In CQL syntax:
+ *
+ * <pre>{@code
+ * SELECT Rstream(auction)
+ * FROM (SELECT B1.auction, count(*) AS num
+ *       FROM Bid [RANGE 60 MINUTE SLIDE 1 MINUTE] B1
+ *       GROUP BY B1.auction)
+ * WHERE num >= ALL (SELECT count(*)
+ *                   FROM Bid [RANGE 60 MINUTE SLIDE 1 MINUTE] B2
+ *                   GROUP BY B2.auction);
+ * }</pre>
+ *
+ * <p>To make things a bit more dynamic and easier to test we use much shorter windows, and
+ * we'll also preserve the bid counts.
+ */
+public class Query5 extends NexmarkQuery {
+  public Query5(NexmarkConfiguration configuration) {
+    super(configuration, "Query5");
+  }
+
+  private PCollection<AuctionCount> applyTyped(PCollection<Event> events) {
+    return events
+        // Only want the bid events.
+        .apply(JUST_BIDS)
+        // Window the bids into sliding windows.
+        .apply(Window.<Bid>into(
+            SlidingWindows.of(Duration.standardSeconds(configuration.windowSizeSec))
+                .every(Duration.standardSeconds(configuration.windowPeriodSec))))
+        // Project just the auction id.
+        .apply("BidToAuction", BID_TO_AUCTION)
+
+        // Count the number of bids per auction id.
+        .apply(Count.<Long>perElement())
+
+      // We'll want to keep all auctions with the maximal number of bids.
+        // Start by lifting each into a singleton list.
+        .apply(name + ".ToSingletons",
+            ParDo.of(new DoFn<KV<Long, Long>, KV<List<Long>, Long>>() {
+                  @ProcessElement
+                  public void processElement(ProcessContext c) {
+                    c.output(KV.of(Arrays.asList(c.element().getKey()), c.element().getValue()));
+                  }
+                }))
+
+        // Keep only the auction ids with the most bids.
+        .apply(
+            Combine
+                .globally(new Combine.BinaryCombineFn<KV<List<Long>, Long>>() {
+                  @Override
+                  public KV<List<Long>, Long> apply(
+                      KV<List<Long>, Long> left, KV<List<Long>, Long> right) {
+                    List<Long> leftBestAuctions = left.getKey();
+                    long leftCount = left.getValue();
+                    List<Long> rightBestAuctions = right.getKey();
+                    long rightCount = right.getValue();
+                    if (leftCount > rightCount) {
+                      return left;
+                    } else if (leftCount < rightCount) {
+                      return right;
+                    } else {
+                      List<Long> newBestAuctions = new ArrayList<>();
+                      newBestAuctions.addAll(leftBestAuctions);
+                      newBestAuctions.addAll(rightBestAuctions);
+                      return KV.of(newBestAuctions, leftCount);
+                    }
+                  }
+                })
+                .withoutDefaults()
+                .withFanout(configuration.fanout))
+
+        // Project into result.
+        .apply(name + ".Select",
+            ParDo.of(new DoFn<KV<List<Long>, Long>, AuctionCount>() {
+                  @ProcessElement
+                  public void processElement(ProcessContext c) {
+                    long count = c.element().getValue();
+                    for (long auction : c.element().getKey()) {
+                      c.output(new AuctionCount(auction, count));
+                    }
+                  }
+                }));
+  }
+
+  @Override
+  protected PCollection<KnownSize> applyPrim(PCollection<Event> events) {
+    return NexmarkUtils.castToKnownSize(name, applyTyped(events));
+  }
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/a7f9f7d0/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query5Model.java
----------------------------------------------------------------------
diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query5Model.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query5Model.java
new file mode 100644
index 0000000..6bf65dc
--- /dev/null
+++ b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query5Model.java
@@ -0,0 +1,178 @@
+/*
+ * 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.beam.integration.nexmark.queries;
+
+import java.io.Serializable;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.TreeMap;
+import org.apache.beam.integration.nexmark.AbstractSimulator;
+import org.apache.beam.integration.nexmark.NexmarkConfiguration;
+import org.apache.beam.integration.nexmark.NexmarkQueryModel;
+import org.apache.beam.integration.nexmark.NexmarkUtils;
+import org.apache.beam.integration.nexmark.model.AuctionCount;
+import org.apache.beam.integration.nexmark.model.Bid;
+import org.apache.beam.integration.nexmark.model.Event;
+import org.apache.beam.sdk.values.TimestampedValue;
+import org.joda.time.Duration;
+import org.joda.time.Instant;
+
+/**
+ * A direct implementation of {@link Query5}.
+ */
+public class Query5Model extends NexmarkQueryModel implements Serializable {
+  /**
+   * Simulator for query 5.
+   */
+  private class Simulator extends AbstractSimulator<Event, AuctionCount> {
+    /** Time of bids still contributing to open windows, indexed by their auction id. */
+    private final Map<Long, List<Instant>> bids;
+
+    /** When oldest active window starts. */
+    private Instant windowStart;
+
+    public Simulator(NexmarkConfiguration configuration) {
+      super(NexmarkUtils.standardEventIterator(configuration));
+      bids = new TreeMap<>();
+      windowStart = NexmarkUtils.BEGINNING_OF_TIME;
+    }
+
+    /**
+     * Count bids per auction id for bids strictly before {@code end}. Add the auction ids with
+     * the maximum number of bids to results.
+     */
+    private void countBids(Instant end) {
+      Map<Long, Long> counts = new TreeMap<>();
+      long maxCount = 0L;
+      for (Map.Entry<Long, List<Instant>> entry : bids.entrySet()) {
+        long count = 0L;
+        long auction = entry.getKey();
+        for (Instant bid : entry.getValue()) {
+          if (bid.isBefore(end)) {
+            count++;
+          }
+        }
+        if (count > 0) {
+          counts.put(auction, count);
+          maxCount = Math.max(maxCount, count);
+        }
+      }
+      for (Map.Entry<Long, Long> entry : counts.entrySet()) {
+        long auction = entry.getKey();
+        long count = entry.getValue();
+        if (count == maxCount) {
+          AuctionCount result = new AuctionCount(auction, count);
+          addResult(TimestampedValue.of(result, end));
+        }
+      }
+    }
+
+    /**
+     * Retire bids which are strictly before {@code cutoff}. Return true if there are any bids
+     * remaining.
+     */
+    private boolean retireBids(Instant cutoff) {
+      boolean anyRemain = false;
+      for (Map.Entry<Long, List<Instant>> entry : bids.entrySet()) {
+        long auction = entry.getKey();
+        Iterator<Instant> itr = entry.getValue().iterator();
+        while (itr.hasNext()) {
+          Instant bid = itr.next();
+          if (bid.isBefore(cutoff)) {
+            NexmarkUtils.info("retire: %s for %s", bid, auction);
+            itr.remove();
+          } else {
+            anyRemain = true;
+          }
+        }
+      }
+      return anyRemain;
+    }
+
+    /**
+     * Retire active windows until we've reached {@code newWindowStart}.
+     */
+    private void retireWindows(Instant newWindowStart) {
+      while (!newWindowStart.equals(windowStart)) {
+        NexmarkUtils.info("retiring window %s, aiming for %s", windowStart, newWindowStart);
+        // Count bids in the window (windowStart, windowStart + size].
+        countBids(windowStart.plus(Duration.standardSeconds(configuration.windowSizeSec)));
+        // Advance the window.
+        windowStart = windowStart.plus(Duration.standardSeconds(configuration.windowPeriodSec));
+        // Retire bids which will never contribute to a future window.
+        if (!retireBids(windowStart)) {
+          // Can fast forward to latest window since no more outstanding bids.
+          windowStart = newWindowStart;
+        }
+      }
+    }
+
+    /**
+     * Add bid to state.
+     */
+    private void captureBid(Bid bid, Instant timestamp) {
+      List<Instant> existing = bids.get(bid.auction);
+      if (existing == null) {
+        existing = new ArrayList<>();
+        bids.put(bid.auction, existing);
+      }
+      existing.add(timestamp);
+    }
+
+    @Override
+    public void run() {
+      TimestampedValue<Event> timestampedEvent = nextInput();
+      if (timestampedEvent == null) {
+        // Drain the remaining windows.
+        retireWindows(NexmarkUtils.END_OF_TIME);
+        allDone();
+        return;
+      }
+
+      Event event = timestampedEvent.getValue();
+      if (event.bid == null) {
+        // Ignore non-bid events.
+        return;
+      }
+      Instant timestamp = timestampedEvent.getTimestamp();
+      Instant newWindowStart = windowStart(Duration.standardSeconds(configuration.windowSizeSec),
+          Duration.standardSeconds(configuration.windowPeriodSec), timestamp);
+      // Capture results from any windows we can now retire.
+      retireWindows(newWindowStart);
+      // Capture current bid.
+      captureBid(event.bid, timestamp);
+    }
+  }
+
+  public Query5Model(NexmarkConfiguration configuration) {
+    super(configuration);
+  }
+
+  @Override
+  public AbstractSimulator<?, ?> simulator() {
+    return new Simulator(configuration);
+  }
+
+  @Override
+  protected <T> Collection<String> toCollection(Iterator<TimestampedValue<T>> itr) {
+    return toValue(itr);
+  }
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/a7f9f7d0/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query6.java
----------------------------------------------------------------------
diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query6.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query6.java
new file mode 100644
index 0000000..2a5ab702
--- /dev/null
+++ b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query6.java
@@ -0,0 +1,159 @@
+/*
+ * 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.beam.integration.nexmark.queries;
+
+import com.google.common.collect.Lists;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+import org.apache.beam.integration.nexmark.NexmarkConfiguration;
+import org.apache.beam.integration.nexmark.NexmarkQuery;
+import org.apache.beam.integration.nexmark.NexmarkUtils;
+import org.apache.beam.integration.nexmark.WinningBids;
+import org.apache.beam.integration.nexmark.model.Auction;
+import org.apache.beam.integration.nexmark.model.AuctionBid;
+import org.apache.beam.integration.nexmark.model.Bid;
+import org.apache.beam.integration.nexmark.model.Event;
+import org.apache.beam.integration.nexmark.model.KnownSize;
+import org.apache.beam.integration.nexmark.model.SellerPrice;
+import org.apache.beam.sdk.transforms.Combine;
+import org.apache.beam.sdk.transforms.DoFn;
+import org.apache.beam.sdk.transforms.ParDo;
+import org.apache.beam.sdk.transforms.windowing.AfterPane;
+import org.apache.beam.sdk.transforms.windowing.GlobalWindows;
+import org.apache.beam.sdk.transforms.windowing.Repeatedly;
+import org.apache.beam.sdk.transforms.windowing.Window;
+import org.apache.beam.sdk.values.KV;
+import org.apache.beam.sdk.values.PCollection;
+import org.joda.time.Duration;
+
+/**
+ * Query 6, 'Average Selling Price by Seller'. Select the average selling price over the
+ * last 10 closed auctions by the same seller. In CQL syntax:
+ *
+ * <pre>{@code
+ * SELECT Istream(AVG(Q.final), Q.seller)
+ * FROM (SELECT Rstream(MAX(B.price) AS final, A.seller)
+ *       FROM Auction A [ROWS UNBOUNDED], Bid B [ROWS UNBOUNDED]
+ *       WHERE A.id=B.auction AND B.datetime < A.expires AND A.expires < CURRENT_TIME
+ *       GROUP BY A.id, A.seller) [PARTITION BY A.seller ROWS 10] Q
+ * GROUP BY Q.seller;
+ * }</pre>
+ *
+ * <p>We are a little more exact with selecting winning bids: see {@link WinningBids}.
+ */
+public class Query6 extends NexmarkQuery {
+  /**
+   * Combiner to keep track of up to {@code maxNumBids} of the most recent wining bids and calculate
+   * their average selling price.
+   */
+  private static class MovingMeanSellingPrice extends Combine.CombineFn<Bid, List<Bid>, Long> {
+    private final int maxNumBids;
+
+    public MovingMeanSellingPrice(int maxNumBids) {
+      this.maxNumBids = maxNumBids;
+    }
+
+    @Override
+    public List<Bid> createAccumulator() {
+      return new ArrayList<>();
+    }
+
+    @Override
+    public List<Bid> addInput(List<Bid> accumulator, Bid input) {
+      accumulator.add(input);
+      Collections.sort(accumulator, Bid.ASCENDING_TIME_THEN_PRICE);
+      if (accumulator.size() > maxNumBids) {
+        accumulator.remove(0);
+      }
+      return accumulator;
+    }
+
+    @Override
+    public List<Bid> mergeAccumulators(Iterable<List<Bid>> accumulators) {
+      List<Bid> result = new ArrayList<>();
+      for (List<Bid> accumulator : accumulators) {
+        for (Bid bid : accumulator) {
+          result.add(bid);
+        }
+      }
+      Collections.sort(result, Bid.ASCENDING_TIME_THEN_PRICE);
+      if (result.size() > maxNumBids) {
+        result = Lists.newArrayList(result.listIterator(result.size() - maxNumBids));
+      }
+      return result;
+    }
+
+    @Override
+    public Long extractOutput(List<Bid> accumulator) {
+      if (accumulator.isEmpty()) {
+        return 0L;
+      }
+      long sumOfPrice = 0;
+      for (Bid bid : accumulator) {
+        sumOfPrice += bid.price;
+      }
+      return Math.round((double) sumOfPrice / accumulator.size());
+    }
+  }
+
+  public Query6(NexmarkConfiguration configuration) {
+    super(configuration, "Query6");
+  }
+
+  private PCollection<SellerPrice> applyTyped(PCollection<Event> events) {
+    return events
+        // Find the winning bid for each closed auction.
+        .apply(new WinningBids(name + ".WinningBids", configuration))
+
+        // Key the winning bid by the seller id.
+        .apply(name + ".Rekey",
+            ParDo.of(new DoFn<AuctionBid, KV<Long, Bid>>() {
+                  @ProcessElement
+                  public void processElement(ProcessContext c) {
+                    Auction auction = c.element().auction;
+                    Bid bid = c.element().bid;
+                    c.output(KV.of(auction.seller, bid));
+                  }
+                }))
+
+        // Re-window to update on every wining bid.
+        .apply(
+            Window.<KV<Long, Bid>>into(new GlobalWindows())
+                .triggering(Repeatedly.forever(AfterPane.elementCountAtLeast(1)))
+                .accumulatingFiredPanes()
+                .withAllowedLateness(Duration.ZERO))
+
+        // Find the average of last 10 winning bids for each seller.
+        .apply(Combine.<Long, Bid, Long>perKey(new MovingMeanSellingPrice(10)))
+
+        // Project into our datatype.
+        .apply(name + ".Select",
+            ParDo.of(new DoFn<KV<Long, Long>, SellerPrice>() {
+                  @ProcessElement
+                  public void processElement(ProcessContext c) {
+                    c.output(new SellerPrice(c.element().getKey(), c.element().getValue()));
+                  }
+                }));
+  }
+
+  @Override
+  protected PCollection<KnownSize> applyPrim(PCollection<Event> events) {
+    return NexmarkUtils.castToKnownSize(name, applyTyped(events));
+  }
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/a7f9f7d0/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query6Model.java
----------------------------------------------------------------------
diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query6Model.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query6Model.java
new file mode 100644
index 0000000..4325337
--- /dev/null
+++ b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query6Model.java
@@ -0,0 +1,135 @@
+/*
+ * 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.beam.integration.nexmark.queries;
+
+import java.io.Serializable;
+import java.util.Collection;
+import java.util.Iterator;
+import java.util.Map;
+import java.util.TreeMap;
+import org.apache.beam.integration.nexmark.AbstractSimulator;
+import org.apache.beam.integration.nexmark.NexmarkConfiguration;
+import org.apache.beam.integration.nexmark.NexmarkQueryModel;
+import org.apache.beam.integration.nexmark.NexmarkUtils;
+import org.apache.beam.integration.nexmark.WinningBidsSimulator;
+import org.apache.beam.integration.nexmark.model.Auction;
+import org.apache.beam.integration.nexmark.model.AuctionBid;
+import org.apache.beam.integration.nexmark.model.Bid;
+import org.apache.beam.integration.nexmark.model.KnownSize;
+import org.apache.beam.integration.nexmark.model.SellerPrice;
+import org.apache.beam.sdk.transforms.windowing.BoundedWindow;
+import org.apache.beam.sdk.values.TimestampedValue;
+import org.joda.time.Instant;
+import org.junit.Assert;
+
+/**
+ * A direct implementation of {@link Query6}.
+ */
+public class Query6Model extends NexmarkQueryModel implements Serializable {
+  /**
+   * Simulator for query 6.
+   */
+  private static class Simulator extends AbstractSimulator<AuctionBid, SellerPrice> {
+    /** The cumulative count of winning bids, indexed by seller id. */
+    private final Map<Long, Long> numWinningBidsPerSeller;
+
+    /** The cumulative total of winning bid prices, indexed by seller id. */
+    private final Map<Long, Long> totalWinningBidPricesPerSeller;
+
+    private Instant lastTimestamp;
+
+    public Simulator(NexmarkConfiguration configuration) {
+      super(new WinningBidsSimulator(configuration).results());
+      numWinningBidsPerSeller = new TreeMap<>();
+      totalWinningBidPricesPerSeller = new TreeMap<>();
+      lastTimestamp = BoundedWindow.TIMESTAMP_MIN_VALUE;
+    }
+
+    /**
+     * Update the per-seller running counts/sums.
+     */
+    private void captureWinningBid(Auction auction, Bid bid, Instant timestamp) {
+      NexmarkUtils.info("winning auction, bid: %s, %s", auction, bid);
+      Long count = numWinningBidsPerSeller.get(auction.seller);
+      if (count == null) {
+        count = 1L;
+      } else {
+        count += 1;
+      }
+      numWinningBidsPerSeller.put(auction.seller, count);
+      Long total = totalWinningBidPricesPerSeller.get(auction.seller);
+      if (total == null) {
+        total = bid.price;
+      } else {
+        total += bid.price;
+      }
+      totalWinningBidPricesPerSeller.put(auction.seller, total);
+      TimestampedValue<SellerPrice> intermediateResult = TimestampedValue.of(
+          new SellerPrice(auction.seller, Math.round((double) total / count)), timestamp);
+      addIntermediateResult(intermediateResult);
+    }
+
+
+    @Override
+    protected void run() {
+      TimestampedValue<AuctionBid> timestampedWinningBid = nextInput();
+      if (timestampedWinningBid == null) {
+        for (long seller : numWinningBidsPerSeller.keySet()) {
+          long count = numWinningBidsPerSeller.get(seller);
+          long total = totalWinningBidPricesPerSeller.get(seller);
+          addResult(TimestampedValue.of(
+              new SellerPrice(seller, Math.round((double) total / count)), lastTimestamp));
+        }
+        allDone();
+        return;
+      }
+
+      lastTimestamp = timestampedWinningBid.getTimestamp();
+      captureWinningBid(timestampedWinningBid.getValue().auction,
+          timestampedWinningBid.getValue().bid, lastTimestamp);
+    }
+  }
+
+  public Query6Model(NexmarkConfiguration configuration) {
+    super(configuration);
+  }
+
+  @Override
+  public AbstractSimulator<?, ?> simulator() {
+    return new Simulator(configuration);
+  }
+
+  @Override
+  protected Iterable<TimestampedValue<KnownSize>> relevantResults(
+      Iterable<TimestampedValue<KnownSize>> results) {
+    // Find the last (in processing time) reported average price for each seller.
+    Map<Long, TimestampedValue<KnownSize>> finalAverages = new TreeMap<>();
+    for (TimestampedValue<KnownSize> obj : results) {
+      Assert.assertTrue("have SellerPrice", obj.getValue() instanceof SellerPrice);
+      SellerPrice sellerPrice = (SellerPrice) obj.getValue();
+      finalAverages.put(
+          sellerPrice.seller, TimestampedValue.of((KnownSize) sellerPrice, obj.getTimestamp()));
+    }
+    return finalAverages.values();
+  }
+
+  @Override
+  protected <T> Collection<String> toCollection(Iterator<TimestampedValue<T>> itr) {
+    return toValue(itr);
+  }
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/a7f9f7d0/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query7.java
----------------------------------------------------------------------
diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query7.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query7.java
new file mode 100644
index 0000000..2835737
--- /dev/null
+++ b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query7.java
@@ -0,0 +1,91 @@
+/*
+ * 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.beam.integration.nexmark.queries;
+
+import org.apache.beam.integration.nexmark.NexmarkConfiguration;
+import org.apache.beam.integration.nexmark.NexmarkQuery;
+import org.apache.beam.integration.nexmark.NexmarkUtils;
+import org.apache.beam.integration.nexmark.model.Bid;
+import org.apache.beam.integration.nexmark.model.Event;
+import org.apache.beam.integration.nexmark.model.KnownSize;
+import org.apache.beam.sdk.transforms.DoFn;
+import org.apache.beam.sdk.transforms.Max;
+import org.apache.beam.sdk.transforms.ParDo;
+import org.apache.beam.sdk.transforms.windowing.FixedWindows;
+import org.apache.beam.sdk.transforms.windowing.Window;
+import org.apache.beam.sdk.values.PCollection;
+import org.apache.beam.sdk.values.PCollectionView;
+
+import org.joda.time.Duration;
+
+/**
+ * Query 7, 'Highest Bid'. Select the bids with the highest bid
+ * price in the last minute. In CQL syntax:
+ *
+ * <pre>
+ * SELECT Rstream(B.auction, B.price, B.bidder)
+ * FROM Bid [RANGE 1 MINUTE SLIDE 1 MINUTE] B
+ * WHERE B.price = (SELECT MAX(B1.price)
+ *                  FROM BID [RANGE 1 MINUTE SLIDE 1 MINUTE] B1);
+ * </pre>
+ *
+ * <p>We will use a shorter window to help make testing easier. We'll also implement this using
+ * a side-input in order to exercise that functionality. (A combiner, as used in Query 5, is
+ * a more efficient approach.).
+ */
+public class Query7 extends NexmarkQuery {
+  public Query7(NexmarkConfiguration configuration) {
+    super(configuration, "Query7");
+  }
+
+  private PCollection<Bid> applyTyped(PCollection<Event> events) {
+    // Window the bids.
+    PCollection<Bid> slidingBids = events.apply(JUST_BIDS).apply(
+        Window.<Bid>into(FixedWindows.of(Duration.standardSeconds(configuration.windowSizeSec))));
+
+    // Find the largest price in all bids.
+    // NOTE: It would be more efficient to write this query much as we did for Query5, using
+    // a binary combiner to accumulate the bids with maximal price. As written this query
+    // requires an additional scan per window, with the associated cost of snapshotted state and
+    // its I/O. We'll keep this implementation since it illustrates the use of side inputs.
+    final PCollectionView<Long> maxPriceView =
+        slidingBids //
+            .apply("BidToPrice", BID_TO_PRICE)
+            .apply(Max.longsGlobally().withFanout(configuration.fanout).asSingletonView());
+
+    return slidingBids
+        // Select all bids which have that maximum price (there may be more than one).
+        .apply(name + ".Select",
+            ParDo.withSideInputs(maxPriceView)
+                .of(new DoFn<Bid, Bid>() {
+                  @ProcessElement
+                  public void processElement(ProcessContext c) {
+                    long maxPrice = c.sideInput(maxPriceView);
+                    Bid bid = c.element();
+                    if (bid.price == maxPrice) {
+                      c.output(bid);
+                    }
+                  }
+                }));
+  }
+
+  @Override
+  protected PCollection<KnownSize> applyPrim(PCollection<Event> events) {
+    return NexmarkUtils.castToKnownSize(name, applyTyped(events));
+  }
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/a7f9f7d0/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query7Model.java
----------------------------------------------------------------------
diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query7Model.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query7Model.java
new file mode 100644
index 0000000..0a80e59
--- /dev/null
+++ b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query7Model.java
@@ -0,0 +1,133 @@
+/*
+ * 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.beam.integration.nexmark.queries;
+
+import java.io.Serializable;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Iterator;
+import java.util.List;
+
+import org.apache.beam.integration.nexmark.AbstractSimulator;
+import org.apache.beam.integration.nexmark.NexmarkConfiguration;
+import org.apache.beam.integration.nexmark.NexmarkQueryModel;
+import org.apache.beam.integration.nexmark.NexmarkUtils;
+import org.apache.beam.integration.nexmark.model.Bid;
+import org.apache.beam.integration.nexmark.model.Event;
+import org.apache.beam.sdk.transforms.windowing.BoundedWindow;
+import org.apache.beam.sdk.values.TimestampedValue;
+import org.joda.time.Duration;
+import org.joda.time.Instant;
+
+/**
+ * A direct implementation of {@link Query7}.
+ */
+public class Query7Model extends NexmarkQueryModel implements Serializable {
+  /**
+   * Simulator for query 7.
+   */
+  private class Simulator extends AbstractSimulator<Event, Bid> {
+    /** Bids with highest bid price seen in the current window. */
+    private final List<Bid> highestBids;
+
+    /** When current window started. */
+    private Instant windowStart;
+
+    private Instant lastTimestamp;
+
+    public Simulator(NexmarkConfiguration configuration) {
+      super(NexmarkUtils.standardEventIterator(configuration));
+      highestBids = new ArrayList<>();
+      windowStart = NexmarkUtils.BEGINNING_OF_TIME;
+      lastTimestamp = BoundedWindow.TIMESTAMP_MIN_VALUE;
+    }
+
+    /**
+     * Transfer the currently winning bids into results and retire them.
+     */
+    private void retireWindow(Instant timestamp) {
+      for (Bid bid : highestBids) {
+        addResult(TimestampedValue.of(bid, timestamp));
+      }
+      highestBids.clear();
+    }
+
+    /**
+     * Keep just the highest price bid.
+     */
+    private void captureBid(Bid bid) {
+      Iterator<Bid> itr = highestBids.iterator();
+      boolean isWinning = true;
+      while (itr.hasNext()) {
+        Bid existingBid = itr.next();
+        if (existingBid.price > bid.price) {
+          isWinning = false;
+          break;
+        }
+        NexmarkUtils.info("smaller price: %s", existingBid);
+        itr.remove();
+      }
+      if (isWinning) {
+        NexmarkUtils.info("larger price: %s", bid);
+        highestBids.add(bid);
+      }
+    }
+
+    @Override
+    protected void run() {
+      TimestampedValue<Event> timestampedEvent = nextInput();
+      if (timestampedEvent == null) {
+        // Capture all remaining bids in results.
+        retireWindow(lastTimestamp);
+        allDone();
+        return;
+      }
+
+      Event event = timestampedEvent.getValue();
+      if (event.bid == null) {
+        // Ignore non-bid events.
+        return;
+      }
+      lastTimestamp = timestampedEvent.getTimestamp();
+      Instant newWindowStart = windowStart(Duration.standardSeconds(configuration.windowSizeSec),
+          Duration.standardSeconds(configuration.windowSizeSec), lastTimestamp);
+      if (!newWindowStart.equals(windowStart)) {
+        // Capture highest priced bids in current window and retire it.
+        retireWindow(lastTimestamp);
+        windowStart = newWindowStart;
+      }
+      // Keep only the highest bids.
+      captureBid(event.bid);
+      //TODO test fails because offset of some hundreds of ms between expect and actual
+    }
+  }
+
+  public Query7Model(NexmarkConfiguration configuration) {
+    super(configuration);
+  }
+
+  @Override
+  public AbstractSimulator<?, ?> simulator() {
+    return new Simulator(configuration);
+  }
+
+  @Override
+  protected <T> Collection<String> toCollection(Iterator<TimestampedValue<T>> itr) {
+    return toValueOrder(itr);
+  }
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/a7f9f7d0/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query8.java
----------------------------------------------------------------------
diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query8.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query8.java
new file mode 100644
index 0000000..e7daccd
--- /dev/null
+++ b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query8.java
@@ -0,0 +1,98 @@
+/*
+ * 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.beam.integration.nexmark.queries;
+
+import org.apache.beam.integration.nexmark.NexmarkConfiguration;
+import org.apache.beam.integration.nexmark.NexmarkQuery;
+import org.apache.beam.integration.nexmark.NexmarkUtils;
+import org.apache.beam.integration.nexmark.model.Auction;
+import org.apache.beam.integration.nexmark.model.Event;
+import org.apache.beam.integration.nexmark.model.IdNameReserve;
+import org.apache.beam.integration.nexmark.model.KnownSize;
+import org.apache.beam.integration.nexmark.model.Person;
+import org.apache.beam.sdk.transforms.DoFn;
+import org.apache.beam.sdk.transforms.ParDo;
+import org.apache.beam.sdk.transforms.join.CoGbkResult;
+import org.apache.beam.sdk.transforms.join.CoGroupByKey;
+import org.apache.beam.sdk.transforms.join.KeyedPCollectionTuple;
+import org.apache.beam.sdk.transforms.windowing.FixedWindows;
+import org.apache.beam.sdk.transforms.windowing.Window;
+import org.apache.beam.sdk.values.KV;
+import org.apache.beam.sdk.values.PCollection;
+import org.joda.time.Duration;
+
+/**
+ * Query 8, 'Monitor New Users'. Select people who have entered the system and created auctions
+ * in the last 12 hours, updated every 12 hours. In CQL syntax:
+ *
+ * <pre>
+ * SELECT Rstream(P.id, P.name, A.reserve)
+ * FROM Person [RANGE 12 HOUR] P, Auction [RANGE 12 HOUR] A
+ * WHERE P.id = A.seller;
+ * </pre>
+ *
+ * <p>To make things a bit more dynamic and easier to test we'll use a much shorter window.
+ */
+public class Query8 extends NexmarkQuery {
+  public Query8(NexmarkConfiguration configuration) {
+    super(configuration, "Query8");
+  }
+
+  private PCollection<IdNameReserve> applyTyped(PCollection<Event> events) {
+    // Window and key new people by their id.
+    PCollection<KV<Long, Person>> personsById =
+        events
+          .apply(JUST_NEW_PERSONS)
+          .apply("Query8.WindowPersons",
+            Window.<Person>into(
+              FixedWindows.of(Duration.standardSeconds(configuration.windowSizeSec))))
+            .apply("PersonById", PERSON_BY_ID);
+
+    // Window and key new auctions by their id.
+    PCollection<KV<Long, Auction>> auctionsBySeller =
+        events.apply(JUST_NEW_AUCTIONS)
+          .apply("Query8.WindowAuctions",
+            Window.<Auction>into(
+              FixedWindows.of(Duration.standardSeconds(configuration.windowSizeSec))))
+            .apply("AuctionBySeller", AUCTION_BY_SELLER);
+
+    // Join people and auctions and project the person id, name and auction reserve price.
+    return KeyedPCollectionTuple.of(PERSON_TAG, personsById)
+        .and(AUCTION_TAG, auctionsBySeller)
+        .apply(CoGroupByKey.<Long>create())
+        .apply(name + ".Select",
+            ParDo.of(new DoFn<KV<Long, CoGbkResult>, IdNameReserve>() {
+                  @ProcessElement
+                  public void processElement(ProcessContext c) {
+                    Person person = c.element().getValue().getOnly(PERSON_TAG, null);
+                    if (person == null) {
+                      // Person was not created in last window period.
+                      return;
+                    }
+                    for (Auction auction : c.element().getValue().getAll(AUCTION_TAG)) {
+                      c.output(new IdNameReserve(person.id, person.name, auction.reserve));
+                    }
+                  }
+                }));
+  }
+
+  @Override
+  protected PCollection<KnownSize> applyPrim(PCollection<Event> events) {
+    return NexmarkUtils.castToKnownSize(name, applyTyped(events));
+  }
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/a7f9f7d0/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query8Model.java
----------------------------------------------------------------------
diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query8Model.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query8Model.java
new file mode 100644
index 0000000..1161994
--- /dev/null
+++ b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query8Model.java
@@ -0,0 +1,150 @@
+/*
+ * 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.beam.integration.nexmark.queries;
+
+import com.google.common.collect.ArrayListMultimap;
+import com.google.common.collect.Multimap;
+import java.io.Serializable;
+import java.util.Collection;
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.Map;
+import org.apache.beam.integration.nexmark.AbstractSimulator;
+import org.apache.beam.integration.nexmark.NexmarkConfiguration;
+import org.apache.beam.integration.nexmark.NexmarkQueryModel;
+import org.apache.beam.integration.nexmark.NexmarkUtils;
+import org.apache.beam.integration.nexmark.model.Auction;
+import org.apache.beam.integration.nexmark.model.Event;
+import org.apache.beam.integration.nexmark.model.IdNameReserve;
+import org.apache.beam.integration.nexmark.model.Person;
+import org.apache.beam.sdk.values.TimestampedValue;
+import org.joda.time.Duration;
+import org.joda.time.Instant;
+
+/**
+ * A direct implementation of {@link Query8}.
+ */
+public class Query8Model extends NexmarkQueryModel implements Serializable {
+  /**
+   * Simulator for query 8.
+   */
+  private class Simulator extends AbstractSimulator<Event, IdNameReserve> {
+    /** New persons seen in the current window, indexed by id. */
+    private final Map<Long, Person> newPersons;
+
+    /** New auctions seen in the current window, indexed by seller id. */
+    private final Multimap<Long, Auction> newAuctions;
+
+    /** When did the current window start. */
+    private Instant windowStart;
+
+    public Simulator(NexmarkConfiguration configuration) {
+      super(NexmarkUtils.standardEventIterator(configuration));
+      newPersons = new HashMap<>();
+      newAuctions = ArrayListMultimap.create();
+      windowStart = NexmarkUtils.BEGINNING_OF_TIME;
+    }
+
+    /**
+     * Retire all persons added in last window.
+     */
+    private void retirePersons() {
+      for (Map.Entry<Long, Person> entry : newPersons.entrySet()) {
+        NexmarkUtils.info("retire: %s", entry.getValue());
+      }
+      newPersons.clear();
+    }
+
+    /**
+     * Retire all auctions added in last window.
+     */
+    private void retireAuctions() {
+      for (Map.Entry<Long, Auction> entry : newAuctions.entries()) {
+        NexmarkUtils.info("retire: %s", entry.getValue());
+      }
+      newAuctions.clear();
+    }
+
+    /**
+     * Capture new result.
+     */
+    private void addResult(Auction auction, Person person, Instant timestamp) {
+      addResult(TimestampedValue.of(
+          new IdNameReserve(person.id, person.name, auction.reserve), timestamp));
+    }
+
+    @Override
+    public void run() {
+      TimestampedValue<Event> timestampedEvent = nextInput();
+      if (timestampedEvent == null) {
+        allDone();
+        return;
+      }
+
+      Event event = timestampedEvent.getValue();
+      if (event.bid != null) {
+        // Ignore bid events.
+        // Keep looking for next events.
+        return;
+      }
+      Instant timestamp = timestampedEvent.getTimestamp();
+      Instant newWindowStart = windowStart(Duration.standardSeconds(configuration.windowSizeSec),
+          Duration.standardSeconds(configuration.windowSizeSec), timestamp);
+      if (!newWindowStart.equals(windowStart)) {
+        // Retire this window.
+        retirePersons();
+        retireAuctions();
+        windowStart = newWindowStart;
+      }
+
+      if (event.newAuction != null) {
+        // Join new auction with existing person, if any.
+        Person person = newPersons.get(event.newAuction.seller);
+        if (person != null) {
+          addResult(event.newAuction, person, timestamp);
+        } else {
+          // Remember auction for future new people.
+          newAuctions.put(event.newAuction.seller, event.newAuction);
+        }
+      } else { // event is not an auction, nor a bid, so it is a person
+        // Join new person with existing auctions.
+        for (Auction auction : newAuctions.get(event.newPerson.id)) {
+          addResult(auction, event.newPerson, timestamp);
+        }
+        // We'll never need these auctions again.
+        newAuctions.removeAll(event.newPerson.id);
+        // Remember person for future auctions.
+        newPersons.put(event.newPerson.id, event.newPerson);
+      }
+    }
+  }
+
+  public Query8Model(NexmarkConfiguration configuration) {
+    super(configuration);
+  }
+
+  @Override
+  public AbstractSimulator<?, ?> simulator() {
+    return new Simulator(configuration);
+  }
+
+  @Override
+  protected <T> Collection<String> toCollection(Iterator<TimestampedValue<T>> itr) {
+    return toValue(itr);
+  }
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/a7f9f7d0/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query9.java
----------------------------------------------------------------------
diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query9.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query9.java
new file mode 100644
index 0000000..aed827b
--- /dev/null
+++ b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query9.java
@@ -0,0 +1,46 @@
+/*
+ * 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.beam.integration.nexmark.queries;
+
+import org.apache.beam.integration.nexmark.NexmarkConfiguration;
+import org.apache.beam.integration.nexmark.NexmarkQuery;
+import org.apache.beam.integration.nexmark.NexmarkUtils;
+import org.apache.beam.integration.nexmark.WinningBids;
+import org.apache.beam.integration.nexmark.model.AuctionBid;
+import org.apache.beam.integration.nexmark.model.Event;
+import org.apache.beam.integration.nexmark.model.KnownSize;
+import org.apache.beam.sdk.values.PCollection;
+
+/**
+ * Query "9", 'Winning bids'. Select just the winning bids. Not in original NEXMark suite, but
+ * handy for testing. See {@link WinningBids} for the details.
+ */
+public class Query9 extends NexmarkQuery {
+  public Query9(NexmarkConfiguration configuration) {
+    super(configuration, "Query9");
+  }
+
+  private PCollection<AuctionBid> applyTyped(PCollection<Event> events) {
+    return events.apply(new WinningBids(name, configuration));
+  }
+
+  @Override
+  protected PCollection<KnownSize> applyPrim(PCollection<Event> events) {
+    return NexmarkUtils.castToKnownSize(name, applyTyped(events));
+  }
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/a7f9f7d0/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query9Model.java
----------------------------------------------------------------------
diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query9Model.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query9Model.java
new file mode 100644
index 0000000..b88d60a
--- /dev/null
+++ b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query9Model.java
@@ -0,0 +1,47 @@
+/*
+ * 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.beam.integration.nexmark.queries;
+
+import java.io.Serializable;
+import java.util.Collection;
+import java.util.Iterator;
+
+import org.apache.beam.integration.nexmark.AbstractSimulator;
+import org.apache.beam.integration.nexmark.NexmarkConfiguration;
+import org.apache.beam.integration.nexmark.NexmarkQueryModel;
+import org.apache.beam.integration.nexmark.WinningBidsSimulator;
+import org.apache.beam.sdk.values.TimestampedValue;
+
+/**
+ * A direct implementation of {@link Query9}.
+ */
+public class Query9Model extends NexmarkQueryModel implements Serializable {
+  public Query9Model(NexmarkConfiguration configuration) {
+    super(configuration);
+  }
+
+  @Override
+  public AbstractSimulator<?, ?> simulator() {
+    return new WinningBidsSimulator(configuration);
+  }
+
+  @Override
+  protected <T> Collection<String> toCollection(Iterator<TimestampedValue<T>> itr) {
+    return toValue(itr);
+  }
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/a7f9f7d0/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/package-info.java
----------------------------------------------------------------------
diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/package-info.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/package-info.java
new file mode 100644
index 0000000..7a56733
--- /dev/null
+++ b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/package-info.java
@@ -0,0 +1,22 @@
+/*
+ * 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.
+ */
+
+/**
+ * Nexmark Queries.
+ */
+package org.apache.beam.integration.nexmark.queries;


[31/55] [abbrv] beam git commit: Fix static analysis issues

Posted by ie...@apache.org.
Fix static analysis issues

Restrict access level on classes + other static analysis fixes

Fix findbugs issues (issue #33)
Fix compile after AvroIO, TextIO, PubsubIO and State refactor


Project: http://git-wip-us.apache.org/repos/asf/beam/repo
Commit: http://git-wip-us.apache.org/repos/asf/beam/commit/1541fad0
Tree: http://git-wip-us.apache.org/repos/asf/beam/tree/1541fad0
Diff: http://git-wip-us.apache.org/repos/asf/beam/diff/1541fad0

Branch: refs/heads/master
Commit: 1541fad077e47df1d47636fd186a72aa827bbc42
Parents: a39cb80
Author: Ismaël Mejía <ie...@apache.org>
Authored: Mon May 1 00:54:08 2017 +0200
Committer: Ismaël Mejía <ie...@gmail.com>
Committed: Wed Aug 23 19:07:28 2017 +0200

----------------------------------------------------------------------
 integration/java/nexmark/pom.xml                |   2 +-
 .../beam/integration/nexmark/Monitor.java       |   4 +-
 .../beam/integration/nexmark/NexmarkDriver.java |  12 +-
 .../beam/integration/nexmark/NexmarkRunner.java | 124 +++++++++++--------
 .../beam/integration/nexmark/NexmarkUtils.java  |  34 +++--
 .../beam/integration/nexmark/model/Auction.java |   8 +-
 .../integration/nexmark/model/AuctionCount.java |   6 +-
 .../integration/nexmark/model/AuctionPrice.java |   4 +-
 .../nexmark/model/BidsPerSession.java           |   4 +-
 .../beam/integration/nexmark/model/Done.java    |   2 +-
 .../beam/integration/nexmark/model/Event.java   |  13 --
 .../nexmark/model/IdNameReserve.java            |   6 +-
 .../nexmark/model/NameCityStateId.java          |   8 +-
 .../beam/integration/nexmark/model/Person.java  |   6 +-
 .../integration/nexmark/model/SellerPrice.java  |   2 +-
 .../nexmark/queries/AbstractSimulator.java      |  10 +-
 .../nexmark/queries/NexmarkQuery.java           |  34 ++---
 .../nexmark/queries/NexmarkQueryModel.java      |  17 +--
 .../nexmark/queries/Query0Model.java            |   2 +-
 .../integration/nexmark/queries/Query10.java    |   6 +-
 .../integration/nexmark/queries/Query11.java    |   3 +-
 .../nexmark/queries/Query1Model.java            |   2 +-
 .../integration/nexmark/queries/Query3.java     |  24 ++--
 .../nexmark/queries/Query3Model.java            |   2 +-
 .../nexmark/queries/Query4Model.java            |   5 +-
 .../integration/nexmark/queries/Query5.java     |   4 +-
 .../integration/nexmark/queries/Query6.java     |   4 +-
 .../nexmark/queries/Query6Model.java            |   5 +-
 .../nexmark/queries/WinningBids.java            |  30 +++--
 .../integration/nexmark/sources/Generator.java  |  11 +-
 .../nexmark/sources/GeneratorConfig.java        |  26 ++--
 .../nexmark/sources/UnboundedEventSource.java   |   2 +-
 .../sources/UnboundedEventSourceTest.java       |   5 +-
 integration/pom.xml                             |  14 +++
 34 files changed, 221 insertions(+), 220 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/beam/blob/1541fad0/integration/java/nexmark/pom.xml
----------------------------------------------------------------------
diff --git a/integration/java/nexmark/pom.xml b/integration/java/nexmark/pom.xml
index fb213e9..8a65c0f 100644
--- a/integration/java/nexmark/pom.xml
+++ b/integration/java/nexmark/pom.xml
@@ -210,7 +210,7 @@
 
     <dependency>
       <groupId>org.apache.beam</groupId>
-      <artifactId>beam-sdks-java-extensions-gcp-core</artifactId>
+      <artifactId>beam-sdks-java-extensions-google-cloud-platform-core</artifactId>
     </dependency>
 
     <dependency>

http://git-wip-us.apache.org/repos/asf/beam/blob/1541fad0/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/Monitor.java
----------------------------------------------------------------------
diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/Monitor.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/Monitor.java
index cb4d71c..2f0c56a 100644
--- a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/Monitor.java
+++ b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/Monitor.java
@@ -63,8 +63,8 @@ public class Monitor<T extends KnownSize> implements Serializable {
 
   public final String name;
   public final String prefix;
-  final MonitorDoFn doFn;
-  final PTransform<PCollection<? extends T>, PCollection<T>> transform;
+  private final MonitorDoFn doFn;
+  private final PTransform<PCollection<? extends T>, PCollection<T>> transform;
 
   public Monitor(String name, String prefix) {
     this.name = name;

http://git-wip-us.apache.org/repos/asf/beam/blob/1541fad0/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkDriver.java
----------------------------------------------------------------------
diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkDriver.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkDriver.java
index 7d532cc..a982a8d 100644
--- a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkDriver.java
+++ b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkDriver.java
@@ -57,7 +57,7 @@ public class NexmarkDriver<OptionT extends NexmarkOptions> {
   /**
    * Entry point.
    */
-  public void runAll(OptionT options, NexmarkRunner runner) {
+  void runAll(OptionT options, NexmarkRunner runner) {
     Instant start = Instant.now();
     Map<NexmarkConfiguration, NexmarkPerf> baseline = loadBaseline(options.getBaselineFilename());
     Map<NexmarkConfiguration, NexmarkPerf> actual = new LinkedHashMap<>();
@@ -87,7 +87,7 @@ public class NexmarkDriver<OptionT extends NexmarkOptions> {
     }
 
     if (!successful) {
-      System.exit(1);
+      throw new RuntimeException("Execution was not successful");
     }
   }
 
@@ -149,8 +149,6 @@ public class NexmarkDriver<OptionT extends NexmarkOptions> {
 
   /**
    * Print summary  of {@code actual} vs (if non-null) {@code baseline}.
-   *
-   * @throws IOException
    */
   private static void saveSummary(
       @Nullable String summaryFilename,
@@ -227,7 +225,7 @@ public class NexmarkDriver<OptionT extends NexmarkOptions> {
       if (actualPerf != null) {
         List<String> errors = actualPerf.errors;
         if (errors == null) {
-          errors = new ArrayList<String>();
+          errors = new ArrayList<>();
           errors.add("NexmarkGoogleRunner returned null errors list");
         }
         for (String error : errors) {
@@ -300,7 +298,7 @@ public class NexmarkDriver<OptionT extends NexmarkOptions> {
     NexmarkOptions options = PipelineOptionsFactory.fromArgs(args)
       .withValidation()
       .as(NexmarkOptions.class);
-    NexmarkRunner runner = new NexmarkRunner(options);
-    new NexmarkDriver().runAll(options, runner);
+    NexmarkRunner<NexmarkOptions> runner = new NexmarkRunner<>(options);
+    new NexmarkDriver<>().runAll(options, runner);
   }
 }

http://git-wip-us.apache.org/repos/asf/beam/blob/1541fad0/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkRunner.java
----------------------------------------------------------------------
diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkRunner.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkRunner.java
index a3c4d33..6df76f0 100644
--- a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkRunner.java
+++ b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkRunner.java
@@ -29,6 +29,7 @@ import java.io.IOException;
 import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.Collections;
+import java.util.HashMap;
 import java.util.List;
 import java.util.NoSuchElementException;
 import java.util.concurrent.ThreadLocalRandom;
@@ -65,10 +66,12 @@ import org.apache.beam.integration.nexmark.queries.Query9;
 import org.apache.beam.integration.nexmark.queries.Query9Model;
 import org.apache.beam.sdk.Pipeline;
 import org.apache.beam.sdk.PipelineResult;
+import org.apache.beam.sdk.coders.CoderException;
 import org.apache.beam.sdk.io.AvroIO;
 import org.apache.beam.sdk.io.TextIO;
 import org.apache.beam.sdk.io.gcp.bigquery.BigQueryIO;
 import org.apache.beam.sdk.io.gcp.pubsub.PubsubIO;
+import org.apache.beam.sdk.io.gcp.pubsub.PubsubMessage;
 import org.apache.beam.sdk.metrics.DistributionResult;
 import org.apache.beam.sdk.metrics.MetricNameFilter;
 import org.apache.beam.sdk.metrics.MetricQueryResults;
@@ -77,6 +80,7 @@ import org.apache.beam.sdk.metrics.MetricsFilter;
 import org.apache.beam.sdk.testing.PAssert;
 import org.apache.beam.sdk.transforms.DoFn;
 import org.apache.beam.sdk.transforms.ParDo;
+import org.apache.beam.sdk.util.CoderUtils;
 import org.apache.beam.sdk.values.PCollection;
 import org.apache.beam.sdk.values.PCollectionTuple;
 import org.apache.beam.sdk.values.TimestampedValue;
@@ -91,15 +95,15 @@ public class NexmarkRunner<OptionT extends NexmarkOptions> {
   /**
    * Minimum number of samples needed for 'stead-state' rate calculation.
    */
-  protected static final int MIN_SAMPLES = 9;
+  private static final int MIN_SAMPLES = 9;
   /**
    * Minimum length of time over which to consider samples for 'steady-state' rate calculation.
    */
-  protected static final Duration MIN_WINDOW = Duration.standardMinutes(2);
+  private static final Duration MIN_WINDOW = Duration.standardMinutes(2);
   /**
    * Delay between perf samples.
    */
-  protected static final Duration PERF_DELAY = Duration.standardSeconds(15);
+  private static final Duration PERF_DELAY = Duration.standardSeconds(15);
   /**
    * How long to let streaming pipeline run after all events have been generated and we've
    * seen no activity.
@@ -117,37 +121,37 @@ public class NexmarkRunner<OptionT extends NexmarkOptions> {
   /**
    * NexmarkOptions shared by all runs.
    */
-  protected final OptionT options;
+  private final OptionT options;
 
   /**
    * Which configuration we are running.
    */
   @Nullable
-  protected NexmarkConfiguration configuration;
+  private NexmarkConfiguration configuration;
 
   /**
    * If in --pubsubMode=COMBINED, the event monitor for the publisher pipeline. Otherwise null.
    */
   @Nullable
-  protected Monitor<Event> publisherMonitor;
+  private Monitor<Event> publisherMonitor;
 
   /**
    * If in --pubsubMode=COMBINED, the pipeline result for the publisher pipeline. Otherwise null.
    */
   @Nullable
-  protected PipelineResult publisherResult;
+  private PipelineResult publisherResult;
 
   /**
    * Result for the main pipeline.
    */
   @Nullable
-  protected PipelineResult mainResult;
+  private PipelineResult mainResult;
 
   /**
    * Query name we are running.
    */
   @Nullable
-  protected String queryName;
+  private String queryName;
 
   public NexmarkRunner(OptionT options) {
     this.options = options;
@@ -160,7 +164,7 @@ public class NexmarkRunner<OptionT extends NexmarkOptions> {
   /**
    * Is this query running in streaming mode?
    */
-  protected boolean isStreaming() {
+  private boolean isStreaming() {
     return options.isStreaming();
   }
 
@@ -174,7 +178,7 @@ public class NexmarkRunner<OptionT extends NexmarkOptions> {
   /**
    * Return maximum number of workers.
    */
-  protected int maxNumWorkers() {
+  private int maxNumWorkers() {
     return 5;
   }
 
@@ -182,7 +186,7 @@ public class NexmarkRunner<OptionT extends NexmarkOptions> {
    * Return the current value for a long counter, or a default value if can't be retrieved.
    * Note this uses only attempted metrics because some runners don't support committed metrics.
    */
-  protected long getCounterMetric(PipelineResult result, String namespace, String name,
+  private long getCounterMetric(PipelineResult result, String namespace, String name,
     long defaultValue) {
     //TODO Ismael calc this only once
     MetricQueryResults metrics = result.metrics().queryMetrics(
@@ -201,7 +205,7 @@ public class NexmarkRunner<OptionT extends NexmarkOptions> {
    * Return the current value for a long counter, or a default value if can't be retrieved.
    * Note this uses only attempted metrics because some runners don't support committed metrics.
    */
-  protected long getDistributionMetric(PipelineResult result, String namespace, String name,
+  private long getDistributionMetric(PipelineResult result, String namespace, String name,
       DistributionType distType, long defaultValue) {
     MetricQueryResults metrics = result.metrics().queryMetrics(
         MetricsFilter.builder().addNameFilter(MetricNameFilter.named(namespace, name)).build());
@@ -226,7 +230,7 @@ public class NexmarkRunner<OptionT extends NexmarkOptions> {
   /**
    * Return the current value for a time counter, or -1 if can't be retrieved.
    */
-  protected long getTimestampMetric(long now, long value) {
+  private long getTimestampMetric(long now, long value) {
     //TODO Ismael improve doc
     if (Math.abs(value - now) > Duration.standardDays(10000).getMillis()) {
       return -1;
@@ -238,8 +242,7 @@ public class NexmarkRunner<OptionT extends NexmarkOptions> {
    * Find a 'steady state' events/sec from {@code snapshots} and
    * store it in {@code perf} if found.
    */
-  protected void captureSteadyState(NexmarkPerf perf,
-                                    List<NexmarkPerf.ProgressSnapshot> snapshots) {
+  private void captureSteadyState(NexmarkPerf perf, List<NexmarkPerf.ProgressSnapshot> snapshots) {
     if (!options.isStreaming()) {
       return;
     }
@@ -426,7 +429,7 @@ public class NexmarkRunner<OptionT extends NexmarkOptions> {
     return perf;
   }
 
-  protected String getJobId(PipelineResult job) {
+  private String getJobId(PipelineResult job) {
     return "";
   }
 
@@ -528,15 +531,14 @@ public class NexmarkRunner<OptionT extends NexmarkOptions> {
   /**
    * Build and run a pipeline using specified options.
    */
-  protected interface PipelineBuilder<OptionT extends NexmarkOptions> {
+  interface PipelineBuilder<OptionT extends NexmarkOptions> {
     void build(OptionT publishOnlyOptions);
   }
 
   /**
    * Invoke the builder with options suitable for running a publish-only child pipeline.
    */
-  protected void invokeBuilderForPublishOnlyPipeline(
-      PipelineBuilder builder) {
+  private void invokeBuilderForPublishOnlyPipeline(PipelineBuilder<NexmarkOptions> builder) {
     builder.build(options);
 //    throw new UnsupportedOperationException(
 //        "Cannot use --pubSubMode=COMBINED with DirectRunner");
@@ -546,7 +548,7 @@ public class NexmarkRunner<OptionT extends NexmarkOptions> {
    * If monitoring, wait until the publisher pipeline has run long enough to establish
    * a backlog on the Pubsub topic. Otherwise, return immediately.
    */
-  protected void waitForPublisherPreload() {
+  private void waitForPublisherPreload() {
     throw new UnsupportedOperationException();
   }
 
@@ -555,7 +557,7 @@ public class NexmarkRunner<OptionT extends NexmarkOptions> {
    * it was measured.
    */
   @Nullable
-  protected NexmarkPerf monitor(NexmarkQuery query) {
+  private NexmarkPerf monitor(NexmarkQuery query) {
     if (!options.getMonitorJobs()) {
       return null;
     }
@@ -841,14 +843,28 @@ public class NexmarkRunner<OptionT extends NexmarkOptions> {
   private PCollection<Event> sourceEventsFromPubsub(Pipeline p, long now) {
     String shortSubscription = shortSubscription(now);
     NexmarkUtils.console("Reading events from Pubsub %s", shortSubscription);
-    PubsubIO.Read<Event> io =
-        PubsubIO.<Event>read().fromSubscription(shortSubscription)
-            .withIdAttribute(NexmarkUtils.PUBSUB_ID)
-            .withCoder(Event.CODER);
+
+    PubsubIO.Read<PubsubMessage> io =
+        PubsubIO.readPubsubMessagesWithAttributes().fromSubscription(shortSubscription)
+            .withIdAttribute(NexmarkUtils.PUBSUB_ID);
     if (!configuration.usePubsubPublishTime) {
       io = io.withTimestampAttribute(NexmarkUtils.PUBSUB_TIMESTAMP);
     }
-    return p.apply(queryName + ".ReadPubsubEvents", io);
+
+    return p
+      .apply(queryName + ".ReadPubsubEvents", io)
+      .apply(queryName + ".PubsubMessageToEvent", ParDo.of(new DoFn<PubsubMessage, Event>() {
+        @ProcessElement
+        public void processElement(ProcessContext c) {
+          byte[] payload = c.element().getPayload();
+          try {
+            Event event = CoderUtils.decodeFromByteArray(Event.CODER, payload);
+            c.output(event);
+          } catch (CoderException e) {
+            // TODO Log decoding Event error
+          }
+        }
+      }));
   }
 
   /**
@@ -861,9 +877,8 @@ public class NexmarkRunner<OptionT extends NexmarkOptions> {
     }
     NexmarkUtils.console("Reading events from Avro files at %s", filename);
     return p
-        .apply(queryName + ".ReadAvroEvents", AvroIO.Read
-                          .from(filename + "*.avro")
-                          .withSchema(Event.class))
+        .apply(queryName + ".ReadAvroEvents", AvroIO.read(Event.class)
+                          .from(filename + "*.avro"))
         .apply("OutputWithTimestamp", NexmarkQuery.EVENT_TIMESTAMP_FROM_DATA);
   }
 
@@ -873,14 +888,28 @@ public class NexmarkRunner<OptionT extends NexmarkOptions> {
   private void sinkEventsToPubsub(PCollection<Event> events, long now) {
     String shortTopic = shortTopic(now);
     NexmarkUtils.console("Writing events to Pubsub %s", shortTopic);
-    PubsubIO.Write<Event> io =
-        PubsubIO.<Event>write().to(shortTopic)
-                      .withIdAttribute(NexmarkUtils.PUBSUB_ID)
-                      .withCoder(Event.CODER);
+
+    PubsubIO.Write<PubsubMessage> io =
+        PubsubIO.writePubsubMessages().to(shortTopic)
+            .withIdAttribute(NexmarkUtils.PUBSUB_ID);
     if (!configuration.usePubsubPublishTime) {
       io = io.withTimestampAttribute(NexmarkUtils.PUBSUB_TIMESTAMP);
     }
-    events.apply(queryName + ".WritePubsubEvents", io);
+
+    events.apply(queryName + ".EventToPubsubMessage",
+            ParDo.of(new DoFn<Event, PubsubMessage>() {
+              @ProcessElement
+              public void processElement(ProcessContext c) {
+                try {
+                  byte[] payload = CoderUtils.encodeToByteArray(Event.CODER, c.element());
+                  c.output(new PubsubMessage(payload, new HashMap<String, String>()));
+                } catch (CoderException e1) {
+                  // TODO Log encoding Event error
+                }
+              }
+            })
+        )
+        .apply(queryName + ".WritePubsubEvents", io);
   }
 
   /**
@@ -890,7 +919,7 @@ public class NexmarkRunner<OptionT extends NexmarkOptions> {
     String shortTopic = shortTopic(now);
     NexmarkUtils.console("Writing results to Pubsub %s", shortTopic);
     PubsubIO.Write<String> io =
-        PubsubIO.<String>write().to(shortTopic)
+        PubsubIO.writeStrings().to(shortTopic)
             .withIdAttribute(NexmarkUtils.PUBSUB_ID);
     if (!configuration.usePubsubPublishTime) {
       io = io.withTimestampAttribute(NexmarkUtils.PUBSUB_TIMESTAMP);
@@ -917,18 +946,16 @@ public class NexmarkRunner<OptionT extends NexmarkOptions> {
     }
     NexmarkUtils.console("Writing events to Avro files at %s", filename);
     source.apply(queryName + ".WriteAvroEvents",
-            AvroIO.Write.to(filename + "/event").withSuffix(".avro").withSchema(Event.class));
+            AvroIO.write(Event.class).to(filename + "/event").withSuffix(".avro"));
     source.apply(NexmarkQuery.JUST_BIDS)
           .apply(queryName + ".WriteAvroBids",
-            AvroIO.Write.to(filename + "/bid").withSuffix(".avro").withSchema(Bid.class));
+            AvroIO.write(Bid.class).to(filename + "/bid").withSuffix(".avro"));
     source.apply(NexmarkQuery.JUST_NEW_AUCTIONS)
           .apply(queryName + ".WriteAvroAuctions",
-                  AvroIO.Write.to(filename + "/auction").withSuffix(".avro")
-                          .withSchema(Auction.class));
+            AvroIO.write(Auction.class).to(filename + "/auction").withSuffix(".avro"));
     source.apply(NexmarkQuery.JUST_NEW_PERSONS)
           .apply(queryName + ".WriteAvroPeople",
-                  AvroIO.Write.to(filename + "/person").withSuffix(".avro")
-                             .withSchema(Person.class));
+            AvroIO.write(Person.class).to(filename + "/person").withSuffix(".avro"));
   }
 
   /**
@@ -938,7 +965,7 @@ public class NexmarkRunner<OptionT extends NexmarkOptions> {
     String filename = textFilename(now);
     NexmarkUtils.console("Writing results to text files at %s", filename);
     formattedResults.apply(queryName + ".WriteTextResults",
-        TextIO.Write.to(filename));
+        TextIO.write().to(filename));
   }
 
   private static class StringToTableRow extends DoFn<String, TableRow> {
@@ -1010,12 +1037,12 @@ public class NexmarkRunner<OptionT extends NexmarkOptions> {
             // Send synthesized events to Pubsub in separate publisher job.
             // We won't start the main pipeline until the publisher has sent the pre-load events.
             // We'll shutdown the publisher job when we notice the main job has finished.
-            invokeBuilderForPublishOnlyPipeline(new PipelineBuilder() {
+            invokeBuilderForPublishOnlyPipeline(new PipelineBuilder<NexmarkOptions>() {
               @Override
               public void build(NexmarkOptions publishOnlyOptions) {
                 Pipeline sp = Pipeline.create(options);
                 NexmarkUtils.setupPipeline(configuration.coderStrategy, sp);
-                publisherMonitor = new Monitor<Event>(queryName, "publisher");
+                publisherMonitor = new Monitor<>(queryName, "publisher");
                 sinkEventsToPubsub(
                     sourceEventsFromSynthetic(sp)
                             .apply(queryName + ".Monitor", publisherMonitor.getTransform()),
@@ -1140,9 +1167,6 @@ public class NexmarkRunner<OptionT extends NexmarkOptions> {
     checkState(queryName == null);
     configuration = runConfiguration;
 
-    // GCS URI patterns to delete on exit.
-    List<String> pathsToDelete = new ArrayList<>();
-
     try {
       NexmarkUtils.console("Running %s", configuration.toShortString());
 
@@ -1220,9 +1244,6 @@ public class NexmarkRunner<OptionT extends NexmarkOptions> {
           }
           ((Query10) query).setOutputPath(path);
           ((Query10) query).setMaxNumWorkers(maxNumWorkers());
-          if (path != null && options.getManageResources()) {
-            pathsToDelete.add(path + "/**");
-          }
         }
 
         // Apply query.
@@ -1252,7 +1273,6 @@ public class NexmarkRunner<OptionT extends NexmarkOptions> {
     } finally {
       configuration = null;
       queryName = null;
-      // TODO: Cleanup pathsToDelete
     }
   }
 }

http://git-wip-us.apache.org/repos/asf/beam/blob/1541fad0/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkUtils.java
----------------------------------------------------------------------
diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkUtils.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkUtils.java
index 18589c4..f6215e9 100644
--- a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkUtils.java
+++ b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkUtils.java
@@ -55,6 +55,9 @@ import org.apache.beam.sdk.coders.SerializableCoder;
 import org.apache.beam.sdk.io.Read;
 import org.apache.beam.sdk.metrics.Counter;
 import org.apache.beam.sdk.metrics.Metrics;
+import org.apache.beam.sdk.state.StateSpec;
+import org.apache.beam.sdk.state.StateSpecs;
+import org.apache.beam.sdk.state.ValueState;
 import org.apache.beam.sdk.transforms.Combine;
 import org.apache.beam.sdk.transforms.DoFn;
 import org.apache.beam.sdk.transforms.PTransform;
@@ -63,9 +66,6 @@ import org.apache.beam.sdk.transforms.windowing.AfterPane;
 import org.apache.beam.sdk.transforms.windowing.BoundedWindow;
 import org.apache.beam.sdk.transforms.windowing.GlobalWindows;
 import org.apache.beam.sdk.transforms.windowing.Window;
-import org.apache.beam.sdk.util.state.StateSpec;
-import org.apache.beam.sdk.util.state.StateSpecs;
-import org.apache.beam.sdk.util.state.ValueState;
 import org.apache.beam.sdk.values.PBegin;
 import org.apache.beam.sdk.values.PCollection;
 import org.apache.beam.sdk.values.TimestampedValue;
@@ -178,7 +178,7 @@ public class NexmarkUtils {
     /** Names are suffixed with the query being run. */
     QUERY,
     /** Names are suffixed with the query being run and a random number. */
-    QUERY_AND_SALT;
+    QUERY_AND_SALT
   }
 
   /**
@@ -310,7 +310,7 @@ public class NexmarkUtils {
    * Log message to console. For client side only.
    */
   public static void console(String format, Object... args) {
-    System.out.printf("%s %s\n", Instant.now(), String.format(format, args));
+    System.out.printf("%s %s%n", Instant.now(), String.format(format, args));
   }
 
   /**
@@ -326,7 +326,7 @@ public class NexmarkUtils {
   /**
    * All events will be given a timestamp relative to this time (ms since epoch).
    */
-  public static final long BASE_TIME = Instant.parse("2015-07-15T00:00:00.000Z").getMillis();
+  private static final long BASE_TIME = Instant.parse("2015-07-15T00:00:00.000Z").getMillis();
 
   /**
    * Instants guaranteed to be strictly before and after all event timestamps, and which won't
@@ -377,7 +377,7 @@ public class NexmarkUtils {
   /**
    * Return a generator config to match the given {@code options}.
    */
-  public static GeneratorConfig standardGeneratorConfig(NexmarkConfiguration configuration) {
+  private static GeneratorConfig standardGeneratorConfig(NexmarkConfiguration configuration) {
     return new GeneratorConfig(configuration,
                                configuration.useWallclockEventTime ? System.currentTimeMillis()
                                                                    : BASE_TIME, 0,
@@ -558,15 +558,14 @@ public class NexmarkUtils {
                         }
                         p++;
                       }
-                      long next = System.currentTimeMillis();
-                      now = next;
+                      now = System.currentTimeMillis();
                     }
                     c.output(c.element());
                   }
                 });
   }
 
-  private static final StateSpec<Object, ValueState<byte[]>> DUMMY_TAG =
+  private static final StateSpec<ValueState<byte[]>> DUMMY_TAG =
           StateSpecs.value(ByteArrayCoder.of());
   private static final int MAX_BUFFER_SIZE = 1 << 24;
 
@@ -578,20 +577,19 @@ public class NexmarkUtils {
                   @ProcessElement
                   public void processElement(ProcessContext c) {
                     long remain = bytes;
-                    long start = System.currentTimeMillis();
-                    long now = start;
+//                    long now = System.currentTimeMillis();
                     while (remain > 0) {
+                      //TODO Ismael google on state
                       long thisBytes = Math.min(remain, MAX_BUFFER_SIZE);
                       remain -= thisBytes;
-                      byte[] arr = new byte[(int) thisBytes];
-                      for (int i = 0; i < thisBytes; i++) {
-                        arr[i] = (byte) now;
-                      }
-                      //TODO Ismael google on state
+//                      byte[] arr = new byte[(int) thisBytes];
+//                      for (int i = 0; i < thisBytes; i++) {
+//                        arr[i] = (byte) now;
+//                      }
 //                      ValueState<byte[]> state = c.windowingInternals().stateInternals().state(
 //                          StateNamespaces.global(), DUMMY_TAG);
 //                      state.write(arr);
-                      now = System.currentTimeMillis();
+//                      now = System.currentTimeMillis();
                     }
                     c.output(c.element());
                   }

http://git-wip-us.apache.org/repos/asf/beam/blob/1541fad0/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/model/Auction.java
----------------------------------------------------------------------
diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/model/Auction.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/model/Auction.java
index 4b1a848..5c018dc 100644
--- a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/model/Auction.java
+++ b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/model/Auction.java
@@ -81,14 +81,14 @@ public class Auction implements KnownSize, Serializable {
 
   /** Extra auction properties. */
   @JsonProperty
-  public final String itemName;
+  private final String itemName;
 
   @JsonProperty
-  public final String description;
+  private final String description;
 
   /** Initial bid price, in cents. */
   @JsonProperty
-  public final long initialBid;
+  private final long initialBid;
 
   /** Reserve price, in cents. */
   @JsonProperty
@@ -111,7 +111,7 @@ public class Auction implements KnownSize, Serializable {
 
   /** Additional arbitrary payload for performance testing. */
   @JsonProperty
-  public final String extra;
+  private final String extra;
 
 
   // For Avro only.

http://git-wip-us.apache.org/repos/asf/beam/blob/1541fad0/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/model/AuctionCount.java
----------------------------------------------------------------------
diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/model/AuctionCount.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/model/AuctionCount.java
index e6d3450..c83a455 100644
--- a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/model/AuctionCount.java
+++ b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/model/AuctionCount.java
@@ -54,11 +54,9 @@ public class AuctionCount implements KnownSize, Serializable {
     }
   };
 
-  @JsonProperty
-  public final long auction;
+  @JsonProperty private final long auction;
 
-  @JsonProperty
-  public final long count;
+  @JsonProperty private final long count;
 
   // For Avro only.
   @SuppressWarnings("unused")

http://git-wip-us.apache.org/repos/asf/beam/blob/1541fad0/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/model/AuctionPrice.java
----------------------------------------------------------------------
diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/model/AuctionPrice.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/model/AuctionPrice.java
index cb971e2..43d0b27 100644
--- a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/model/AuctionPrice.java
+++ b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/model/AuctionPrice.java
@@ -55,11 +55,11 @@ public class AuctionPrice implements KnownSize, Serializable {
   };
 
   @JsonProperty
-  public final long auction;
+  private final long auction;
 
   /** Price in cents. */
   @JsonProperty
-  public final long price;
+  private final long price;
 
   // For Avro only.
   @SuppressWarnings("unused")

http://git-wip-us.apache.org/repos/asf/beam/blob/1541fad0/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/model/BidsPerSession.java
----------------------------------------------------------------------
diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/model/BidsPerSession.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/model/BidsPerSession.java
index 26b6a41..6dddf34 100644
--- a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/model/BidsPerSession.java
+++ b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/model/BidsPerSession.java
@@ -56,10 +56,10 @@ public class BidsPerSession implements KnownSize, Serializable {
   };
 
   @JsonProperty
-  public final long personId;
+  private final long personId;
 
   @JsonProperty
-  public final long bidsPerSession;
+  private final long bidsPerSession;
 
   public BidsPerSession() {
     personId = 0;

http://git-wip-us.apache.org/repos/asf/beam/blob/1541fad0/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/model/Done.java
----------------------------------------------------------------------
diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/model/Done.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/model/Done.java
index 42999cd..0c14e8f 100644
--- a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/model/Done.java
+++ b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/model/Done.java
@@ -54,7 +54,7 @@ public class Done implements KnownSize, Serializable {
   };
 
   @JsonProperty
-  public final String message;
+  private final String message;
 
   // For Avro only.
   @SuppressWarnings("unused")

http://git-wip-us.apache.org/repos/asf/beam/blob/1541fad0/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/model/Event.java
----------------------------------------------------------------------
diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/model/Event.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/model/Event.java
index e2130c9..1f1f096 100644
--- a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/model/Event.java
+++ b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/model/Event.java
@@ -138,19 +138,6 @@ public class Event implements KnownSize, Serializable {
     }
   }
 
-  /**
-   * Remove {@code annotation} from event. (Used for debugging.)
-   */
-  public Event withoutAnnotation(String annotation) {
-    if (newPerson != null) {
-      return new Event(newPerson.withoutAnnotation(annotation));
-    } else if (newAuction != null) {
-      return new Event(newAuction.withoutAnnotation(annotation));
-    } else {
-      return new Event(bid.withoutAnnotation(annotation));
-    }
-  }
-
   @Override
   public long sizeInBytes() {
     if (newPerson != null) {

http://git-wip-us.apache.org/repos/asf/beam/blob/1541fad0/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/model/IdNameReserve.java
----------------------------------------------------------------------
diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/model/IdNameReserve.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/model/IdNameReserve.java
index cf1e571..17b8c4a 100644
--- a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/model/IdNameReserve.java
+++ b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/model/IdNameReserve.java
@@ -60,14 +60,14 @@ public class IdNameReserve implements KnownSize, Serializable {
   };
 
   @JsonProperty
-  public final long id;
+  private final long id;
 
   @JsonProperty
-  public final String name;
+  private final String name;
 
   /** Reserve price in cents. */
   @JsonProperty
-  public final long reserve;
+  private final long reserve;
 
   // For Avro only.
   @SuppressWarnings("unused")

http://git-wip-us.apache.org/repos/asf/beam/blob/1541fad0/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/model/NameCityStateId.java
----------------------------------------------------------------------
diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/model/NameCityStateId.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/model/NameCityStateId.java
index 86d1738..28f25cd 100644
--- a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/model/NameCityStateId.java
+++ b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/model/NameCityStateId.java
@@ -62,16 +62,16 @@ public class NameCityStateId implements KnownSize, Serializable {
   };
 
   @JsonProperty
-  public final String name;
+  private final String name;
 
   @JsonProperty
-  public final String city;
+  private final String city;
 
   @JsonProperty
-  public final String state;
+  private final String state;
 
   @JsonProperty
-  public final long id;
+  private final long id;
 
   // For Avro only.
   @SuppressWarnings("unused")

http://git-wip-us.apache.org/repos/asf/beam/blob/1541fad0/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/model/Person.java
----------------------------------------------------------------------
diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/model/Person.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/model/Person.java
index 906df94..c690fd4 100644
--- a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/model/Person.java
+++ b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/model/Person.java
@@ -77,10 +77,10 @@ public class Person implements KnownSize, Serializable {
   public final String name;
 
   @JsonProperty
-  public final String emailAddress;
+  private final String emailAddress;
 
   @JsonProperty
-  public final String creditCard;
+  private final String creditCard;
 
   @JsonProperty
   public final String city;
@@ -93,7 +93,7 @@ public class Person implements KnownSize, Serializable {
 
   /** Additional arbitrary payload for performance testing. */
   @JsonProperty
-  public final String extra;
+  private final String extra;
 
   // For Avro only.
   @SuppressWarnings("unused")

http://git-wip-us.apache.org/repos/asf/beam/blob/1541fad0/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/model/SellerPrice.java
----------------------------------------------------------------------
diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/model/SellerPrice.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/model/SellerPrice.java
index 68f2697..52ff540 100644
--- a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/model/SellerPrice.java
+++ b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/model/SellerPrice.java
@@ -60,7 +60,7 @@ public class SellerPrice implements KnownSize, Serializable {
 
   /** Price in cents. */
   @JsonProperty
-  public final long price;
+  private final long price;
 
   // For Avro only.
   @SuppressWarnings("unused")

http://git-wip-us.apache.org/repos/asf/beam/blob/1541fad0/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/AbstractSimulator.java
----------------------------------------------------------------------
diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/AbstractSimulator.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/AbstractSimulator.java
index 270b5c3..1395182 100644
--- a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/AbstractSimulator.java
+++ b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/AbstractSimulator.java
@@ -37,7 +37,7 @@ import org.joda.time.Instant;
  */
 public abstract class AbstractSimulator<InputT, OutputT> {
   /** Window size for action bucket sampling. */
-  public static final Duration WINDOW_SIZE = Duration.standardMinutes(1);
+  private static final Duration WINDOW_SIZE = Duration.standardMinutes(1);
 
   /** Input event stream we should draw from. */
   private final Iterator<TimestampedValue<InputT>> input;
@@ -77,7 +77,7 @@ public abstract class AbstractSimulator<InputT, OutputT> {
 
   /** Called by implementors of {@link #run}: Fetch the next input element. */
   @Nullable
-  protected TimestampedValue<InputT> nextInput() {
+  TimestampedValue<InputT> nextInput() {
     if (!input.hasNext()) {
       return null;
     }
@@ -90,7 +90,7 @@ public abstract class AbstractSimulator<InputT, OutputT> {
    * Called by implementors of {@link #run}:  Capture an intermediate result, for the purpose of
    * recording the expected activity of the query over time.
    */
-  protected void addIntermediateResult(TimestampedValue<OutputT> result) {
+  void addIntermediateResult(TimestampedValue<OutputT> result) {
     NexmarkUtils.info("intermediate result: %s", result);
     updateCounts(result.getTimestamp());
   }
@@ -99,7 +99,7 @@ public abstract class AbstractSimulator<InputT, OutputT> {
    * Called by implementors of {@link #run}: Capture a final result, for the purpose of checking
    * semantic correctness.
    */
-  protected void addResult(TimestampedValue<OutputT> result) {
+  void addResult(TimestampedValue<OutputT> result) {
     NexmarkUtils.info("result: %s", result);
     pendingResults.add(result);
     updateCounts(result.getTimestamp());
@@ -121,7 +121,7 @@ public abstract class AbstractSimulator<InputT, OutputT> {
   }
 
   /** Called by implementors of {@link #run}: Record that no more results will be emitted. */
-  protected void allDone() {
+  void allDone() {
     isDone = true;
   }
 

http://git-wip-us.apache.org/repos/asf/beam/blob/1541fad0/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/NexmarkQuery.java
----------------------------------------------------------------------
diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/NexmarkQuery.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/NexmarkQuery.java
index 0796ce5..09415c0 100644
--- a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/NexmarkQuery.java
+++ b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/NexmarkQuery.java
@@ -46,10 +46,10 @@ public abstract class NexmarkQuery
     extends PTransform<PCollection<Event>, PCollection<TimestampedValue<KnownSize>>> {
   public static final TupleTag<Auction> AUCTION_TAG = new TupleTag<>("auctions");
   public static final TupleTag<Bid> BID_TAG = new TupleTag<>("bids");
-  protected static final TupleTag<Person> PERSON_TAG = new TupleTag<>("person");
+  static final TupleTag<Person> PERSON_TAG = new TupleTag<>("person");
 
   /** Predicate to detect a new person event. */
-  protected static final SerializableFunction<Event, Boolean> IS_NEW_PERSON =
+  private static final SerializableFunction<Event, Boolean> IS_NEW_PERSON =
       new SerializableFunction<Event, Boolean>() {
         @Override
         public Boolean apply(Event event) {
@@ -58,7 +58,7 @@ public abstract class NexmarkQuery
       };
 
   /** DoFn to convert a new person event to a person. */
-  protected static final DoFn<Event, Person> AS_PERSON = new DoFn<Event, Person>() {
+  private static final DoFn<Event, Person> AS_PERSON = new DoFn<Event, Person>() {
     @ProcessElement
     public void processElement(ProcessContext c) {
       c.output(c.element().newPerson);
@@ -66,7 +66,7 @@ public abstract class NexmarkQuery
   };
 
   /** Predicate to detect a new auction event. */
-  protected static final SerializableFunction<Event, Boolean> IS_NEW_AUCTION =
+  private static final SerializableFunction<Event, Boolean> IS_NEW_AUCTION =
       new SerializableFunction<Event, Boolean>() {
         @Override
         public Boolean apply(Event event) {
@@ -75,7 +75,7 @@ public abstract class NexmarkQuery
       };
 
   /** DoFn to convert a new auction event to an auction. */
-  protected static final DoFn<Event, Auction> AS_AUCTION = new DoFn<Event, Auction>() {
+  private static final DoFn<Event, Auction> AS_AUCTION = new DoFn<Event, Auction>() {
     @ProcessElement
     public void processElement(ProcessContext c) {
       c.output(c.element().newAuction);
@@ -83,7 +83,7 @@ public abstract class NexmarkQuery
   };
 
   /** Predicate to detect a new bid event. */
-  protected static final SerializableFunction<Event, Boolean> IS_BID =
+  private static final SerializableFunction<Event, Boolean> IS_BID =
       new SerializableFunction<Event, Boolean>() {
         @Override
         public Boolean apply(Event event) {
@@ -92,7 +92,7 @@ public abstract class NexmarkQuery
       };
 
   /** DoFn to convert a bid event to a bid. */
-  protected static final DoFn<Event, Bid> AS_BID = new DoFn<Event, Bid>() {
+  private static final DoFn<Event, Bid> AS_BID = new DoFn<Event, Bid>() {
     @ProcessElement
     public void processElement(ProcessContext c) {
       c.output(c.element().bid);
@@ -100,7 +100,7 @@ public abstract class NexmarkQuery
   };
 
   /** Transform to key each person by their id. */
-  protected static final ParDo.SingleOutput<Person, KV<Long, Person>> PERSON_BY_ID =
+  static final ParDo.SingleOutput<Person, KV<Long, Person>> PERSON_BY_ID =
       ParDo.of(new DoFn<Person, KV<Long, Person>>() {
              @ProcessElement
              public void processElement(ProcessContext c) {
@@ -109,7 +109,7 @@ public abstract class NexmarkQuery
            });
 
   /** Transform to key each auction by its id. */
-  protected static final ParDo.SingleOutput<Auction, KV<Long, Auction>> AUCTION_BY_ID =
+  static final ParDo.SingleOutput<Auction, KV<Long, Auction>> AUCTION_BY_ID =
       ParDo.of(new DoFn<Auction, KV<Long, Auction>>() {
              @ProcessElement
              public void processElement(ProcessContext c) {
@@ -118,7 +118,7 @@ public abstract class NexmarkQuery
            });
 
   /** Transform to key each auction by its seller id. */
-  protected static final ParDo.SingleOutput<Auction, KV<Long, Auction>> AUCTION_BY_SELLER =
+  static final ParDo.SingleOutput<Auction, KV<Long, Auction>> AUCTION_BY_SELLER =
       ParDo.of(new DoFn<Auction, KV<Long, Auction>>() {
              @ProcessElement
              public void processElement(ProcessContext c) {
@@ -127,7 +127,7 @@ public abstract class NexmarkQuery
            });
 
   /** Transform to key each bid by it's auction id. */
-  protected static final ParDo.SingleOutput<Bid, KV<Long, Bid>> BID_BY_AUCTION =
+  static final ParDo.SingleOutput<Bid, KV<Long, Bid>> BID_BY_AUCTION =
       ParDo.of(new DoFn<Bid, KV<Long, Bid>>() {
              @ProcessElement
              public void processElement(ProcessContext c) {
@@ -136,7 +136,7 @@ public abstract class NexmarkQuery
            });
 
   /** Transform to project the auction id from each bid. */
-  protected static final ParDo.SingleOutput<Bid, Long> BID_TO_AUCTION =
+  static final ParDo.SingleOutput<Bid, Long> BID_TO_AUCTION =
       ParDo.of(new DoFn<Bid, Long>() {
              @ProcessElement
              public void processElement(ProcessContext c) {
@@ -145,7 +145,7 @@ public abstract class NexmarkQuery
            });
 
   /** Transform to project the price from each bid. */
-  protected static final ParDo.SingleOutput<Bid, Long> BID_TO_PRICE =
+  static final ParDo.SingleOutput<Bid, Long> BID_TO_PRICE =
       ParDo.of(new DoFn<Bid, Long>() {
              @ProcessElement
              public void processElement(ProcessContext c) {
@@ -205,13 +205,13 @@ public abstract class NexmarkQuery
         }
       };
 
-  protected final NexmarkConfiguration configuration;
+  final NexmarkConfiguration configuration;
   public final Monitor<Event> eventMonitor;
   public final Monitor<KnownSize> resultMonitor;
-  public final Monitor<Event> endOfStreamMonitor;
-  protected final Counter fatalCounter;
+  private final Monitor<Event> endOfStreamMonitor;
+  private final Counter fatalCounter;
 
-  protected NexmarkQuery(NexmarkConfiguration configuration, String name) {
+  NexmarkQuery(NexmarkConfiguration configuration, String name) {
     super(name);
     this.configuration = configuration;
     if (configuration.debug) {

http://git-wip-us.apache.org/repos/asf/beam/blob/1541fad0/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/NexmarkQueryModel.java
----------------------------------------------------------------------
diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/NexmarkQueryModel.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/NexmarkQueryModel.java
index 1ad9099..bfa668b 100644
--- a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/NexmarkQueryModel.java
+++ b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/NexmarkQueryModel.java
@@ -43,7 +43,7 @@ import org.junit.Assert;
 public abstract class NexmarkQueryModel implements Serializable {
   public final NexmarkConfiguration configuration;
 
-  public NexmarkQueryModel(NexmarkConfiguration configuration) {
+  NexmarkQueryModel(NexmarkConfiguration configuration) {
     this.configuration = configuration;
   }
 
@@ -51,7 +51,7 @@ public abstract class NexmarkQueryModel implements Serializable {
    * Return the start of the most recent window of {@code size} and {@code period} which ends
    * strictly before {@code timestamp}.
    */
-  public static Instant windowStart(Duration size, Duration period, Instant timestamp) {
+  static Instant windowStart(Duration size, Duration period, Instant timestamp) {
     long ts = timestamp.getMillis();
     long p = period.getMillis();
     long lim = ts - ts % p;
@@ -60,7 +60,7 @@ public abstract class NexmarkQueryModel implements Serializable {
   }
 
   /** Convert {@code itr} to strings capturing values, timestamps and order. */
-  protected static <T> List<String> toValueTimestampOrder(Iterator<TimestampedValue<T>> itr) {
+  static <T> List<String> toValueTimestampOrder(Iterator<TimestampedValue<T>> itr) {
     List<String> strings = new ArrayList<>();
     while (itr.hasNext()) {
       strings.add(itr.next().toString());
@@ -69,7 +69,7 @@ public abstract class NexmarkQueryModel implements Serializable {
   }
 
   /** Convert {@code itr} to strings capturing values and order. */
-  protected static <T> List<String> toValueOrder(Iterator<TimestampedValue<T>> itr) {
+  static <T> List<String> toValueOrder(Iterator<TimestampedValue<T>> itr) {
     List<String> strings = new ArrayList<>();
     while (itr.hasNext()) {
       strings.add(itr.next().getValue().toString());
@@ -78,7 +78,7 @@ public abstract class NexmarkQueryModel implements Serializable {
   }
 
   /** Convert {@code itr} to strings capturing values only. */
-  protected static <T> Set<String> toValue(Iterator<TimestampedValue<T>> itr) {
+  static <T> Set<String> toValue(Iterator<TimestampedValue<T>> itr) {
     Set<String> strings = new HashSet<>();
     while (itr.hasNext()) {
       strings.add(itr.next().getValue().toString());
@@ -90,7 +90,7 @@ public abstract class NexmarkQueryModel implements Serializable {
   public abstract AbstractSimulator<?, ?> simulator();
 
   /** Return sub-sequence of results which are significant for model. */
-  protected Iterable<TimestampedValue<KnownSize>> relevantResults(
+  Iterable<TimestampedValue<KnownSize>> relevantResults(
       Iterable<TimestampedValue<KnownSize>> results) {
     return results;
   }
@@ -104,8 +104,6 @@ public abstract class NexmarkQueryModel implements Serializable {
   /** Return assertion to use on results of pipeline for this query. */
   public SerializableFunction<Iterable<TimestampedValue<KnownSize>>, Void> assertionFor() {
     final Collection<String> expectedStrings = toCollection(simulator().results());
-    final String[] expectedStringsArray =
-      expectedStrings.toArray(new String[expectedStrings.size()]);
 
     return new SerializableFunction<Iterable<TimestampedValue<KnownSize>>, Void>() {
       @Override
@@ -113,9 +111,6 @@ public abstract class NexmarkQueryModel implements Serializable {
       Collection<String> actualStrings = toCollection(relevantResults(actual).iterator());
         Assert.assertThat("wrong pipeline output", actualStrings,
           IsEqual.equalTo(expectedStrings));
-//compare without order
-//      Assert.assertThat("wrong pipeline output", actualStrings,
-//        IsIterableContainingInAnyOrder.containsInAnyOrder(expectedStringsArray));
         return null;
       }
     };

http://git-wip-us.apache.org/repos/asf/beam/blob/1541fad0/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query0Model.java
----------------------------------------------------------------------
diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query0Model.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query0Model.java
index 6fb6613..8e65591 100644
--- a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query0Model.java
+++ b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query0Model.java
@@ -32,7 +32,7 @@ public class Query0Model extends NexmarkQueryModel {
   /**
    * Simulator for query 0.
    */
-  private class Simulator extends AbstractSimulator<Event, Event> {
+  private static class Simulator extends AbstractSimulator<Event, Event> {
     public Simulator(NexmarkConfiguration configuration) {
       super(NexmarkUtils.standardEventIterator(configuration));
     }

http://git-wip-us.apache.org/repos/asf/beam/blob/1541fad0/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query10.java
----------------------------------------------------------------------
diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query10.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query10.java
index c919691..516dab1 100644
--- a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query10.java
+++ b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query10.java
@@ -35,7 +35,7 @@ import org.apache.beam.integration.nexmark.model.KnownSize;
 import org.apache.beam.sdk.coders.Coder;
 import org.apache.beam.sdk.metrics.Counter;
 import org.apache.beam.sdk.metrics.Metrics;
-import org.apache.beam.sdk.options.GcsOptions;
+import org.apache.beam.sdk.extensions.gcp.options.GcsOptions;
 import org.apache.beam.sdk.transforms.DoFn;
 import org.apache.beam.sdk.transforms.GroupByKey;
 import org.apache.beam.sdk.transforms.ParDo;
@@ -101,7 +101,7 @@ public class Query10 extends NexmarkQuery {
 
     @Override
     public String toString() {
-      return String.format("%s %s %d %s %s\n", maxTimestamp, shard, index, timing, filename);
+      return String.format("%s %s %d %s %s%n", maxTimestamp, shard, index, timing, filename);
     }
   }
 
@@ -130,8 +130,6 @@ public class Query10 extends NexmarkQuery {
 
   /**
    * Return channel for writing bytes to GCS.
-   *
-   * @throws IOException
    */
   private WritableByteChannel openWritableGcsFile(GcsOptions options, String filename)
       throws IOException {

http://git-wip-us.apache.org/repos/asf/beam/blob/1541fad0/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query11.java
----------------------------------------------------------------------
diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query11.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query11.java
index fd936a9..6db9bcf 100644
--- a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query11.java
+++ b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query11.java
@@ -63,14 +63,13 @@ public class Query11 extends NexmarkQuery {
                 Repeatedly.forever(AfterPane.elementCountAtLeast(configuration.maxLogEvents)))
             .discardingFiredPanes()
             .withAllowedLateness(Duration.standardSeconds(configuration.occasionalDelaySec / 2)));
-    PCollection<BidsPerSession> bidsPerSession = biddersWindowed.apply(Count.<Long>perElement())
+    return biddersWindowed.apply(Count.<Long>perElement())
         .apply(name + ".ToResult", ParDo.of(new DoFn<KV<Long, Long>, BidsPerSession>() {
 
           @ProcessElement public void processElement(ProcessContext c) {
             c.output(new BidsPerSession(c.element().getKey(), c.element().getValue()));
           }
         }));
-    return bidsPerSession;
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/beam/blob/1541fad0/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query1Model.java
----------------------------------------------------------------------
diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query1Model.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query1Model.java
index 0388687..5d4de45 100644
--- a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query1Model.java
+++ b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query1Model.java
@@ -34,7 +34,7 @@ public class Query1Model extends NexmarkQueryModel implements Serializable {
   /**
    * Simulator for query 1.
    */
-  private class Simulator extends AbstractSimulator<Event, Bid> {
+  private static class Simulator extends AbstractSimulator<Event, Bid> {
     public Simulator(NexmarkConfiguration configuration) {
       super(NexmarkUtils.standardEventIterator(configuration));
     }

http://git-wip-us.apache.org/repos/asf/beam/blob/1541fad0/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query3.java
----------------------------------------------------------------------
diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query3.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query3.java
index 71364ba..f74b78d 100644
--- a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query3.java
+++ b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query3.java
@@ -17,7 +17,6 @@
  */
 package org.apache.beam.integration.nexmark.queries;
 
-import java.io.IOException;
 import java.util.ArrayList;
 import java.util.List;
 import org.apache.beam.integration.nexmark.NexmarkConfiguration;
@@ -30,6 +29,13 @@ import org.apache.beam.integration.nexmark.model.Person;
 import org.apache.beam.sdk.coders.ListCoder;
 import org.apache.beam.sdk.metrics.Counter;
 import org.apache.beam.sdk.metrics.Metrics;
+import org.apache.beam.sdk.state.TimeDomain;
+import org.apache.beam.sdk.state.Timer;
+import org.apache.beam.sdk.state.TimerSpec;
+import org.apache.beam.sdk.state.TimerSpecs;
+import org.apache.beam.sdk.state.ValueState;
+import org.apache.beam.sdk.state.StateSpec;
+import org.apache.beam.sdk.state.StateSpecs;
 import org.apache.beam.sdk.transforms.DoFn;
 import org.apache.beam.sdk.transforms.Filter;
 import org.apache.beam.sdk.transforms.ParDo;
@@ -41,13 +47,6 @@ import org.apache.beam.sdk.transforms.windowing.AfterPane;
 import org.apache.beam.sdk.transforms.windowing.GlobalWindows;
 import org.apache.beam.sdk.transforms.windowing.Repeatedly;
 import org.apache.beam.sdk.transforms.windowing.Window;
-import org.apache.beam.sdk.util.TimeDomain;
-import org.apache.beam.sdk.util.Timer;
-import org.apache.beam.sdk.util.TimerSpec;
-import org.apache.beam.sdk.util.TimerSpecs;
-import org.apache.beam.sdk.util.state.StateSpec;
-import org.apache.beam.sdk.util.state.StateSpecs;
-import org.apache.beam.sdk.util.state.ValueState;
 import org.apache.beam.sdk.values.KV;
 import org.apache.beam.sdk.values.PCollection;
 import org.joda.time.Duration;
@@ -176,18 +175,18 @@ public class Query3 extends NexmarkQuery {
    */
   private static class JoinDoFn extends DoFn<KV<Long, CoGbkResult>, KV<Auction, Person>> {
 
-    private int maxAuctionsWaitingTime;
+    private final int maxAuctionsWaitingTime;
     private static final String AUCTIONS = "auctions";
     private static final String PERSON = "person";
 
     @StateId(PERSON)
-    private static final StateSpec<Object, ValueState<Person>> personSpec =
+    private static final StateSpec<ValueState<Person>> personSpec =
         StateSpecs.value(Person.CODER);
 
     private static final String PERSON_STATE_EXPIRING = "personStateExpiring";
 
     @StateId(AUCTIONS)
-    private final StateSpec<Object, ValueState<List<Auction>>> auctionsSpec =
+    private final StateSpec<ValueState<List<Auction>>> auctionsSpec =
         StateSpecs.value(ListCoder.of(Auction.CODER));
 
     @TimerId(PERSON_STATE_EXPIRING)
@@ -219,8 +218,7 @@ public class Query3 extends NexmarkQuery {
         ProcessContext c,
         @TimerId(PERSON_STATE_EXPIRING) Timer timer,
         @StateId(PERSON) ValueState<Person> personState,
-        @StateId(AUCTIONS) ValueState<List<Auction>> auctionsState)
-        throws IOException {
+        @StateId(AUCTIONS) ValueState<List<Auction>> auctionsState) {
       // We would *almost* implement this by  rewindowing into the global window and
       // running a combiner over the result. The combiner's accumulator would be the
       // state we use below. However, combiners cannot emit intermediate results, thus

http://git-wip-us.apache.org/repos/asf/beam/blob/1541fad0/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query3Model.java
----------------------------------------------------------------------
diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query3Model.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query3Model.java
index 6b98e2a..f415709 100644
--- a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query3Model.java
+++ b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query3Model.java
@@ -42,7 +42,7 @@ public class Query3Model extends NexmarkQueryModel implements Serializable {
   /**
    * Simulator for query 3.
    */
-  private class Simulator extends AbstractSimulator<Event, NameCityStateId> {
+  private static class Simulator extends AbstractSimulator<Event, NameCityStateId> {
     /** Auctions, indexed by seller id. */
     private final Multimap<Long, Auction> newAuctions;
 

http://git-wip-us.apache.org/repos/asf/beam/blob/1541fad0/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query4Model.java
----------------------------------------------------------------------
diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query4Model.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query4Model.java
index 634a58e..269e47a 100644
--- a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query4Model.java
+++ b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query4Model.java
@@ -93,8 +93,9 @@ public class Query4Model extends NexmarkQueryModel implements Serializable {
         }
         totals.put(category, total);
       }
-      for (long category : counts.keySet()) {
-        long count = counts.get(category);
+      for (Map.Entry<Long, Long> entry : counts.entrySet()) {
+        long category = entry.getKey();
+        long count = entry.getValue();
         long total = totals.get(category);
         TimestampedValue<CategoryPrice> result = TimestampedValue.of(
             new CategoryPrice(category, Math.round((double) total / count), true), lastTimestamp);

http://git-wip-us.apache.org/repos/asf/beam/blob/1541fad0/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query5.java
----------------------------------------------------------------------
diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query5.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query5.java
index 18ce578..1944330 100644
--- a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query5.java
+++ b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query5.java
@@ -18,7 +18,7 @@
 package org.apache.beam.integration.nexmark.queries;
 
 import java.util.ArrayList;
-import java.util.Arrays;
+import java.util.Collections;
 import java.util.List;
 import org.apache.beam.integration.nexmark.NexmarkConfiguration;
 import org.apache.beam.integration.nexmark.NexmarkUtils;
@@ -80,7 +80,7 @@ public class Query5 extends NexmarkQuery {
             ParDo.of(new DoFn<KV<Long, Long>, KV<List<Long>, Long>>() {
                   @ProcessElement
                   public void processElement(ProcessContext c) {
-                    c.output(KV.of(Arrays.asList(c.element().getKey()), c.element().getValue()));
+                    c.output(KV.of(Collections.singletonList(c.element().getKey()), c.element().getValue()));
                   }
                 }))
 

http://git-wip-us.apache.org/repos/asf/beam/blob/1541fad0/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query6.java
----------------------------------------------------------------------
diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query6.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query6.java
index 65789ab..ea39ede 100644
--- a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query6.java
+++ b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query6.java
@@ -86,9 +86,7 @@ public class Query6 extends NexmarkQuery {
     public List<Bid> mergeAccumulators(Iterable<List<Bid>> accumulators) {
       List<Bid> result = new ArrayList<>();
       for (List<Bid> accumulator : accumulators) {
-        for (Bid bid : accumulator) {
-          result.add(bid);
-        }
+        result.addAll(accumulator);
       }
       Collections.sort(result, Bid.ASCENDING_TIME_THEN_PRICE);
       if (result.size() > maxNumBids) {

http://git-wip-us.apache.org/repos/asf/beam/blob/1541fad0/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query6Model.java
----------------------------------------------------------------------
diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query6Model.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query6Model.java
index 0691714..9cb8b3d 100644
--- a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query6Model.java
+++ b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query6Model.java
@@ -86,8 +86,9 @@ public class Query6Model extends NexmarkQueryModel implements Serializable {
     protected void run() {
       TimestampedValue<AuctionBid> timestampedWinningBid = nextInput();
       if (timestampedWinningBid == null) {
-        for (long seller : numWinningBidsPerSeller.keySet()) {
-          long count = numWinningBidsPerSeller.get(seller);
+        for (Map.Entry<Long, Long> entry : numWinningBidsPerSeller.entrySet()) {
+          long seller = entry.getKey();
+          long count = entry.getValue();
           long total = totalWinningBidPricesPerSeller.get(seller);
           addResult(TimestampedValue.of(
               new SellerPrice(seller, Math.round((double) total / count)), lastTimestamp));

http://git-wip-us.apache.org/repos/asf/beam/blob/1541fad0/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/WinningBids.java
----------------------------------------------------------------------
diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/WinningBids.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/WinningBids.java
index 11a4d38..52891a7 100644
--- a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/WinningBids.java
+++ b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/WinningBids.java
@@ -25,8 +25,8 @@ import java.io.InputStream;
 import java.io.OutputStream;
 import java.io.Serializable;
 import java.util.ArrayList;
-import java.util.Arrays;
 import java.util.Collection;
+import java.util.Collections;
 import java.util.List;
 import java.util.Map;
 import java.util.TreeMap;
@@ -77,7 +77,7 @@ import org.joda.time.Instant;
  */
 public class WinningBids extends PTransform<PCollection<Event>, PCollection<AuctionBid>> {
   /** Windows for open auctions and bids. */
-  private static class AuctionOrBidWindow extends IntervalWindow implements Serializable {
+  private static class AuctionOrBidWindow extends IntervalWindow {
     /** Id of auction this window is for. */
     public final long auction;
 
@@ -104,9 +104,7 @@ public class WinningBids extends PTransform<PCollection<Event>, PCollection<Auct
 
     /** Return an auction window for {@code auction}. */
     public static AuctionOrBidWindow forAuction(Instant timestamp, Auction auction) {
-      AuctionOrBidWindow result =
-          new AuctionOrBidWindow(timestamp, new Instant(auction.expires), auction.id, true);
-      return result;
+      return new AuctionOrBidWindow(timestamp, new Instant(auction.expires), auction.id, true);
     }
 
     /**
@@ -127,9 +125,8 @@ public class WinningBids extends PTransform<PCollection<Event>, PCollection<Auct
       // Instead, we will just give the bid a finite window which expires at
       // the upper bound of auctions assuming the auction starts at the same time as the bid,
       // and assuming the system is running at its lowest event rate (as per interEventDelayUs).
-      AuctionOrBidWindow result = new AuctionOrBidWindow(
+      return new AuctionOrBidWindow(
           timestamp, timestamp.plus(expectedAuctionDurationMs * 2), bid.auction, false);
-      return result;
     }
 
     /** Is this an auction window? */
@@ -171,8 +168,7 @@ public class WinningBids extends PTransform<PCollection<Event>, PCollection<Auct
         throws IOException, CoderException {
       IntervalWindow superWindow = SUPER_CODER.decode(inStream, Coder.Context.NESTED);
       long auction = ID_CODER.decode(inStream, Coder.Context.NESTED);
-      boolean isAuctionWindow =
-          INT_CODER.decode(inStream, Coder.Context.NESTED) == 0 ? false : true;
+      boolean isAuctionWindow = INT_CODER.decode(inStream, Context.NESTED) != 0;
       return new AuctionOrBidWindow(
           superWindow.start(), superWindow.end(), auction, isAuctionWindow);
     }
@@ -194,15 +190,16 @@ public class WinningBids extends PTransform<PCollection<Event>, PCollection<Auct
       Event event = c.element();
       if (event.newAuction != null) {
         // Assign auctions to an auction window which expires at the auction's close.
-        return Arrays.asList(AuctionOrBidWindow.forAuction(c.timestamp(), event.newAuction));
+        return Collections
+            .singletonList(AuctionOrBidWindow.forAuction(c.timestamp(), event.newAuction));
       } else if (event.bid != null) {
         // Assign bids to a temporary bid window which will later be merged into the appropriate
         // auction window.
-        return Arrays.asList(
+        return Collections.singletonList(
             AuctionOrBidWindow.forBid(expectedAuctionDurationMs, c.timestamp(), event.bid));
       } else {
         // Don't assign people to any window. They will thus be dropped.
-        return Arrays.asList();
+        return Collections.emptyList();
       }
     }
 
@@ -226,8 +223,9 @@ public class WinningBids extends PTransform<PCollection<Event>, PCollection<Auct
 
       // Merge all 'bid' windows into their corresponding 'auction' window, provided the
       // auction has not expired.
-      for (long auction : idToTrueAuctionWindow.keySet()) {
-        AuctionOrBidWindow auctionWindow = idToTrueAuctionWindow.get(auction);
+      for (Map.Entry<Long, AuctionOrBidWindow> entry : idToTrueAuctionWindow.entrySet()) {
+        long auction = entry.getKey();
+        AuctionOrBidWindow auctionWindow = entry.getValue();
         List<AuctionOrBidWindow> bidWindows = idToBidAuctionWindows.get(auction);
         if (bidWindows != null) {
           List<AuctionOrBidWindow> toBeMerged = new ArrayList<>();
@@ -296,8 +294,8 @@ public class WinningBids extends PTransform<PCollection<Event>, PCollection<Auct
         configuration.firstEventRate, configuration.nextEventRate,
         configuration.rateUnit, configuration.numEventGenerators);
     long longestDelayUs = 0;
-    for (int i = 0; i < interEventDelayUs.length; i++) {
-      longestDelayUs = Math.max(longestDelayUs, interEventDelayUs[i]);
+    for (long interEventDelayU : interEventDelayUs) {
+      longestDelayUs = Math.max(longestDelayUs, interEventDelayU);
     }
     // Adjust for proportion of auction events amongst all events.
     longestDelayUs =

http://git-wip-us.apache.org/repos/asf/beam/blob/1541fad0/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/sources/Generator.java
----------------------------------------------------------------------
diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/sources/Generator.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/sources/Generator.java
index 012d4e6..2a2732b 100644
--- a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/sources/Generator.java
+++ b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/sources/Generator.java
@@ -123,8 +123,8 @@ public class Generator implements Iterator<TimestampedValue<Event>>, Serializabl
           @Override public void verifyDeterministic() throws NonDeterministicException {}
         };
 
-    private long numEvents;
-    private long wallclockBaseTime;
+    private final long numEvents;
+    private final long wallclockBaseTime;
 
     private Checkpoint(long numEvents, long wallclockBaseTime) {
       this.numEvents = numEvents;
@@ -403,8 +403,8 @@ public class Generator implements Iterator<TimestampedValue<Event>>, Serializabl
     if (n < Integer.MAX_VALUE) {
       return random.nextInt((int) n);
     } else {
-      // TODO: Very skewed distribution! Bad!
-      return Math.abs(random.nextLong()) % n;
+      // WARNING: Very skewed distribution! Bad!
+      return Math.abs(random.nextLong() % n);
     }
   }
 
@@ -470,14 +470,13 @@ public class Generator implements Iterator<TimestampedValue<Event>>, Serializabl
 
     long category = GeneratorConfig.FIRST_CATEGORY_ID + random.nextInt(NUM_CATEGORIES);
     long initialBid = nextPrice(random);
-    long dateTime = timestamp;
     long expires = timestamp + nextAuctionLengthMs(random, timestamp);
     String name = nextString(random, 20);
     String desc = nextString(random, 100);
     long reserve = initialBid + nextPrice(random);
     int currentSize = 8 + name.length() + desc.length() + 8 + 8 + 8 + 8 + 8;
     String extra = nextExtra(random, currentSize, config.configuration.avgAuctionByteSize);
-    return new Auction(id, name, desc, initialBid, reserve, dateTime, expires, seller, category,
+    return new Auction(id, name, desc, initialBid, reserve, timestamp, expires, seller, category,
         extra);
   }
 

http://git-wip-us.apache.org/repos/asf/beam/blob/1541fad0/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/sources/GeneratorConfig.java
----------------------------------------------------------------------
diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/sources/GeneratorConfig.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/sources/GeneratorConfig.java
index 3caaf51..5799bb2 100644
--- a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/sources/GeneratorConfig.java
+++ b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/sources/GeneratorConfig.java
@@ -42,7 +42,7 @@ public class GeneratorConfig implements Serializable {
    */
   public static final int PERSON_PROPORTION = 1;
   public static final int AUCTION_PROPORTION = 3;
-  public static final int BID_PROPORTION = 46;
+  private static final int BID_PROPORTION = 46;
   public static final int PROPORTION_DENOMINATOR =
       PERSON_PROPORTION + AUCTION_PROPORTION + BID_PROPORTION;
 
@@ -55,12 +55,12 @@ public class GeneratorConfig implements Serializable {
    * Delay between events, in microseconds. If the array has more than one entry then
    * the rate is changed every {@link #stepLengthSec}, and wraps around.
    */
-  public final long[] interEventDelayUs;
+  private final long[] interEventDelayUs;
 
   /**
    * Delay before changing the current inter-event delay.
    */
-  public final long stepLengthSec;
+  private final long stepLengthSec;
 
   /**
    * Time for first event (ms since epoch).
@@ -88,13 +88,13 @@ public class GeneratorConfig implements Serializable {
    * True period of epoch in milliseconds. Derived from above.
    * (Ie time to run through cycle for all interEventDelayUs entries).
    */
-  public final long epochPeriodMs;
+  private final long epochPeriodMs;
 
   /**
    * Number of events per epoch. Derived from above.
    * (Ie number of events to run through cycle for all interEventDelayUs entries).
    */
-  public final long eventsPerEpoch;
+  private final long eventsPerEpoch;
 
   public GeneratorConfig(
       NexmarkConfiguration configuration, long baseTime, long firstEventId,
@@ -121,10 +121,10 @@ public class GeneratorConfig implements Serializable {
     long eventsPerEpoch = 0;
     long epochPeriodMs = 0;
     if (interEventDelayUs.length > 1) {
-      for (int i = 0; i < interEventDelayUs.length; i++) {
-        long numEventsForThisCycle = (stepLengthSec * 1_000_000L) / interEventDelayUs[i];
+      for (long interEventDelayU : interEventDelayUs) {
+        long numEventsForThisCycle = (stepLengthSec * 1_000_000L) / interEventDelayU;
         eventsPerEpoch += numEventsForThisCycle;
-        epochPeriodMs += (numEventsForThisCycle * interEventDelayUs[i]) / 1000L;
+        epochPeriodMs += (numEventsForThisCycle * interEventDelayU) / 1000L;
       }
     }
     this.eventsPerEpoch = eventsPerEpoch;
@@ -248,16 +248,16 @@ public class GeneratorConfig implements Serializable {
     long epoch = eventNumber / eventsPerEpoch;
     long n = eventNumber % eventsPerEpoch;
     long offsetInEpochMs = 0;
-    for (int i = 0; i < interEventDelayUs.length; i++) {
-      long numEventsForThisCycle = (stepLengthSec * 1_000_000L) / interEventDelayUs[i];
+    for (long interEventDelayU : interEventDelayUs) {
+      long numEventsForThisCycle = (stepLengthSec * 1_000_000L) / interEventDelayU;
       if (n < numEventsForThisCycle) {
-        long offsetInCycleUs = n * interEventDelayUs[i];
+        long offsetInCycleUs = n * interEventDelayU;
         long timestamp =
             baseTime + epoch * epochPeriodMs + offsetInEpochMs + (offsetInCycleUs / 1000L);
-        return KV.of(timestamp, interEventDelayUs[i]);
+        return KV.of(timestamp, interEventDelayU);
       }
       n -= numEventsForThisCycle;
-      offsetInEpochMs += (numEventsForThisCycle * interEventDelayUs[i]) / 1000L;
+      offsetInEpochMs += (numEventsForThisCycle * interEventDelayU) / 1000L;
     }
     throw new RuntimeException("internal eventsPerEpoch incorrect"); // can't reach
   }

http://git-wip-us.apache.org/repos/asf/beam/blob/1541fad0/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/sources/UnboundedEventSource.java
----------------------------------------------------------------------
diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/sources/UnboundedEventSource.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/sources/UnboundedEventSource.java
index c3c6eb0..09d945d 100644
--- a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/sources/UnboundedEventSource.java
+++ b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/sources/UnboundedEventSource.java
@@ -116,7 +116,7 @@ public class UnboundedEventSource extends UnboundedSource<Event, Generator.Check
     private TimestampedValue<Event> currentEvent;
 
     /** Events which have been held back so as to force them to be late. */
-    private Queue<Generator.NextEvent> heldBackEvents = new PriorityQueue<>();
+    private final Queue<Generator.NextEvent> heldBackEvents = new PriorityQueue<>();
 
     public EventReader(Generator generator) {
       this.generator = generator;

http://git-wip-us.apache.org/repos/asf/beam/blob/1541fad0/integration/java/nexmark/src/test/java/org/apache/beam/integration/nexmark/sources/UnboundedEventSourceTest.java
----------------------------------------------------------------------
diff --git a/integration/java/nexmark/src/test/java/org/apache/beam/integration/nexmark/sources/UnboundedEventSourceTest.java b/integration/java/nexmark/src/test/java/org/apache/beam/integration/nexmark/sources/UnboundedEventSourceTest.java
index 15e17a8..1d04e2a 100644
--- a/integration/java/nexmark/src/test/java/org/apache/beam/integration/nexmark/sources/UnboundedEventSourceTest.java
+++ b/integration/java/nexmark/src/test/java/org/apache/beam/integration/nexmark/sources/UnboundedEventSourceTest.java
@@ -53,8 +53,8 @@ public class UnboundedEventSourceTest {
    * confirming reading events match the model events.
    */
   private static class EventIdChecker {
-    private Set<Long> seenPersonIds = new HashSet<>();
-    private Set<Long> seenAuctionIds = new HashSet<>();
+    private final Set<Long> seenPersonIds = new HashSet<>();
+    private final Set<Long> seenAuctionIds = new HashSet<>();
 
     public void add(Event event) {
       if (event.newAuction != null) {
@@ -90,7 +90,6 @@ public class UnboundedEventSourceTest {
 
     EventIdChecker checker = new EventIdChecker();
     PipelineOptions options = TestPipeline.testingPipelineOptions();
-    Pipeline p = TestPipeline.create(options);
     UnboundedEventSource source = new UnboundedEventSource(config, 1, 0, false);
     UnboundedReader<Event> reader = source.createReader(options, null);
 

http://git-wip-us.apache.org/repos/asf/beam/blob/1541fad0/integration/pom.xml
----------------------------------------------------------------------
diff --git a/integration/pom.xml b/integration/pom.xml
index 4839da5..31f293e 100644
--- a/integration/pom.xml
+++ b/integration/pom.xml
@@ -30,6 +30,20 @@
   <packaging>pom</packaging>
   <name>Apache Beam :: Integration Tests</name>
 
+  <profiles>
+    <profile>
+      <id>release</id>
+      <build>
+        <plugins>
+          <plugin>
+            <groupId>org.codehaus.mojo</groupId>
+            <artifactId>findbugs-maven-plugin</artifactId>
+          </plugin>
+        </plugins>
+      </build>
+    </profile>
+  </profiles>
+
   <modules>
     <module>java</module>
   </modules>


[55/55] [abbrv] beam git commit: This closes #3114

Posted by ie...@apache.org.
This closes #3114


Project: http://git-wip-us.apache.org/repos/asf/beam/repo
Commit: http://git-wip-us.apache.org/repos/asf/beam/commit/64ff21f3
Tree: http://git-wip-us.apache.org/repos/asf/beam/tree/64ff21f3
Diff: http://git-wip-us.apache.org/repos/asf/beam/diff/64ff21f3

Branch: refs/heads/master
Commit: 64ff21f35ee2946d38645fb0a51678628e49e62a
Parents: f0ce31b 0acd720
Author: Ismaël Mejía <ie...@gmail.com>
Authored: Wed Aug 23 19:08:32 2017 +0200
Committer: Ismaël Mejía <ie...@gmail.com>
Committed: Wed Aug 23 19:08:32 2017 +0200

----------------------------------------------------------------------
 sdks/java/javadoc/ant.xml                       |    1 +
 sdks/java/nexmark/README.md                     |  340 +++++
 sdks/java/nexmark/pom.xml                       |  261 ++++
 .../java/org/apache/beam/sdk/nexmark/Main.java  |  303 +++++
 .../org/apache/beam/sdk/nexmark/Monitor.java    |   78 ++
 .../beam/sdk/nexmark/NexmarkConfiguration.java  |  721 +++++++++++
 .../beam/sdk/nexmark/NexmarkLauncher.java       | 1157 ++++++++++++++++++
 .../apache/beam/sdk/nexmark/NexmarkOptions.java |  403 ++++++
 .../apache/beam/sdk/nexmark/NexmarkPerf.java    |  207 ++++
 .../apache/beam/sdk/nexmark/NexmarkSuite.java   |  112 ++
 .../apache/beam/sdk/nexmark/NexmarkUtils.java   |  674 ++++++++++
 .../apache/beam/sdk/nexmark/model/Auction.java  |  187 +++
 .../beam/sdk/nexmark/model/AuctionBid.java      |   85 ++
 .../beam/sdk/nexmark/model/AuctionCount.java    |   84 ++
 .../beam/sdk/nexmark/model/AuctionPrice.java    |   88 ++
 .../org/apache/beam/sdk/nexmark/model/Bid.java  |  177 +++
 .../beam/sdk/nexmark/model/BidsPerSession.java  |   87 ++
 .../beam/sdk/nexmark/model/CategoryPrice.java   |   97 ++
 .../org/apache/beam/sdk/nexmark/model/Done.java |   80 ++
 .../apache/beam/sdk/nexmark/model/Event.java    |  171 +++
 .../beam/sdk/nexmark/model/IdNameReserve.java   |   98 ++
 .../beam/sdk/nexmark/model/KnownSize.java       |   26 +
 .../beam/sdk/nexmark/model/NameCityStateId.java |  103 ++
 .../apache/beam/sdk/nexmark/model/Person.java   |  163 +++
 .../beam/sdk/nexmark/model/SellerPrice.java     |   89 ++
 .../beam/sdk/nexmark/model/package-info.java    |   22 +
 .../apache/beam/sdk/nexmark/package-info.java   |   21 +
 .../sdk/nexmark/queries/AbstractSimulator.java  |  211 ++++
 .../beam/sdk/nexmark/queries/NexmarkQuery.java  |  270 ++++
 .../sdk/nexmark/queries/NexmarkQueryModel.java  |  118 ++
 .../apache/beam/sdk/nexmark/queries/Query0.java |   70 ++
 .../beam/sdk/nexmark/queries/Query0Model.java   |   64 +
 .../apache/beam/sdk/nexmark/queries/Query1.java |   67 +
 .../beam/sdk/nexmark/queries/Query10.java       |  367 ++++++
 .../beam/sdk/nexmark/queries/Query11.java       |   79 ++
 .../beam/sdk/nexmark/queries/Query12.java       |   80 ++
 .../beam/sdk/nexmark/queries/Query1Model.java   |   76 ++
 .../apache/beam/sdk/nexmark/queries/Query2.java |   79 ++
 .../beam/sdk/nexmark/queries/Query2Model.java   |   80 ++
 .../apache/beam/sdk/nexmark/queries/Query3.java |  301 +++++
 .../beam/sdk/nexmark/queries/Query3Model.java   |  124 ++
 .../apache/beam/sdk/nexmark/queries/Query4.java |  116 ++
 .../beam/sdk/nexmark/queries/Query4Model.java   |  186 +++
 .../apache/beam/sdk/nexmark/queries/Query5.java |  138 +++
 .../beam/sdk/nexmark/queries/Query5Model.java   |  176 +++
 .../apache/beam/sdk/nexmark/queries/Query6.java |  155 +++
 .../beam/sdk/nexmark/queries/Query6Model.java   |  133 ++
 .../apache/beam/sdk/nexmark/queries/Query7.java |   90 ++
 .../beam/sdk/nexmark/queries/Query7Model.java   |  130 ++
 .../apache/beam/sdk/nexmark/queries/Query8.java |   97 ++
 .../beam/sdk/nexmark/queries/Query8Model.java   |  148 +++
 .../apache/beam/sdk/nexmark/queries/Query9.java |   44 +
 .../beam/sdk/nexmark/queries/Query9Model.java   |   44 +
 .../beam/sdk/nexmark/queries/WinningBids.java   |  412 +++++++
 .../nexmark/queries/WinningBidsSimulator.java   |  206 ++++
 .../beam/sdk/nexmark/queries/package-info.java  |   22 +
 .../sdk/nexmark/sources/BoundedEventSource.java |  190 +++
 .../beam/sdk/nexmark/sources/Generator.java     |  609 +++++++++
 .../sdk/nexmark/sources/GeneratorConfig.java    |  298 +++++
 .../nexmark/sources/UnboundedEventSource.java   |  329 +++++
 .../beam/sdk/nexmark/sources/package-info.java  |   22 +
 .../nexmark/src/main/resources/log4j.properties |   55 +
 .../beam/sdk/nexmark/queries/QueryTest.java     |  185 +++
 .../nexmark/sources/BoundedEventSourceTest.java |   70 ++
 .../beam/sdk/nexmark/sources/GeneratorTest.java |  110 ++
 .../sources/UnboundedEventSourceTest.java       |  105 ++
 sdks/java/pom.xml                               |    1 +
 67 files changed, 11892 insertions(+)
----------------------------------------------------------------------



[18/55] [abbrv] beam git commit: Fix Queries tests

Posted by ie...@apache.org.
Fix Queries tests

Workaround for issue #22 + extra cleaning

Replace junit asserts by hamcrest asserts
Set numEvents in test to the minimum number that makes the tests pass
issue #15

comments, improve asserts (hamcrest), reformat

For now make generate monothreaded


Project: http://git-wip-us.apache.org/repos/asf/beam/repo
Commit: http://git-wip-us.apache.org/repos/asf/beam/commit/1bd57351
Tree: http://git-wip-us.apache.org/repos/asf/beam/tree/1bd57351
Diff: http://git-wip-us.apache.org/repos/asf/beam/diff/1bd57351

Branch: refs/heads/master
Commit: 1bd57351f1db9b932b253c36d08098cf57ce652b
Parents: a1fe33b
Author: Etienne Chauchot <ec...@gmail.com>
Authored: Thu Mar 16 11:38:08 2017 +0100
Committer: Ismaël Mejía <ie...@gmail.com>
Committed: Wed Aug 23 19:07:27 2017 +0200

----------------------------------------------------------------------
 integration/java/nexmark/pom.xml                |  8 +++-
 .../integration/nexmark/NexmarkQueryModel.java  | 49 ++++++++++----------
 .../beam/integration/nexmark/NexmarkUtils.java  |  3 +-
 .../beam/integration/nexmark/Query0Model.java   |  1 +
 .../beam/integration/nexmark/Query1Model.java   |  1 +
 .../beam/integration/nexmark/Query7Model.java   |  1 +
 .../beam/integration/nexmark/Query8Model.java   |  2 +-
 .../nexmark/WinningBidsSimulator.java           |  1 +
 .../beam/integration/nexmark/QueryTest.java     | 13 +++---
 9 files changed, 45 insertions(+), 34 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/beam/blob/1bd57351/integration/java/nexmark/pom.xml
----------------------------------------------------------------------
diff --git a/integration/java/nexmark/pom.xml b/integration/java/nexmark/pom.xml
index febd96d..27abb0e 100644
--- a/integration/java/nexmark/pom.xml
+++ b/integration/java/nexmark/pom.xml
@@ -264,7 +264,13 @@
     <dependency>
       <groupId>org.hamcrest</groupId>
       <artifactId>hamcrest-all</artifactId>
-      <scope>test</scope>
+      <version>${hamcrest.version}</version>
+    </dependency>
+
+    <dependency>
+      <groupId>org.hamcrest</groupId>
+      <artifactId>hamcrest-library</artifactId>
+      <version>${hamcrest.version}</version>
     </dependency>
 
     <dependency>

http://git-wip-us.apache.org/repos/asf/beam/blob/1bd57351/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkQueryModel.java
----------------------------------------------------------------------
diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkQueryModel.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkQueryModel.java
index a23f82b..f265e0d 100644
--- a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkQueryModel.java
+++ b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkQueryModel.java
@@ -17,6 +17,11 @@
  */
 package org.apache.beam.integration.nexmark;
 
+import static org.hamcrest.CoreMatchers.containsString;
+import static org.hamcrest.CoreMatchers.equalTo;
+import static org.hamcrest.CoreMatchers.hasItems;
+import org.hamcrest.collection.IsIterableContainingInAnyOrder;
+
 import java.io.Serializable;
 import java.util.ArrayList;
 import java.util.Collection;
@@ -28,16 +33,23 @@ import java.util.Set;
 import org.apache.beam.sdk.transforms.SerializableFunction;
 import org.apache.beam.sdk.values.TimestampedValue;
 
+import org.hamcrest.core.IsCollectionContaining;
+import org.hamcrest.core.IsEqual;
 import org.joda.time.Duration;
 import org.joda.time.Instant;
 import org.junit.Assert;
 
 /**
- * Base class for models of the eight NEXMark queries. Provides an assertion
- * function which can be applied against the actual query results to check their consistency
- * with the model.
+ * Base class for models of the eight NEXMark queries. Provides an assertion function which can be
+ * applied against the actual query results to check their consistency with the model.
  */
 public abstract class NexmarkQueryModel implements Serializable {
+  protected final NexmarkConfiguration configuration;
+
+  public NexmarkQueryModel(NexmarkConfiguration configuration) {
+    this.configuration = configuration;
+  }
+
   /**
    * Return the start of the most recent window of {@code size} and {@code period} which ends
    * strictly before {@code timestamp}.
@@ -50,15 +62,7 @@ public abstract class NexmarkQueryModel implements Serializable {
     return new Instant(lim - s);
   }
 
-  protected final NexmarkConfiguration configuration;
-
-  public NexmarkQueryModel(NexmarkConfiguration configuration) {
-    this.configuration = configuration;
-  }
-
-  /**
-   * Convert {@code itr} to strings capturing values, timestamps and order.
-   */
+  /** Convert {@code itr} to strings capturing values, timestamps and order. */
   protected static <T> List<String> toValueTimestampOrder(Iterator<TimestampedValue<T>> itr) {
     List<String> strings = new ArrayList<>();
     while (itr.hasNext()) {
@@ -67,9 +71,7 @@ public abstract class NexmarkQueryModel implements Serializable {
     return strings;
   }
 
-  /**
-   * Convert {@code itr} to strings capturing values and order.
-   */
+  /** Convert {@code itr} to strings capturing values and order. */
   protected static <T> List<String> toValueOrder(Iterator<TimestampedValue<T>> itr) {
     List<String> strings = new ArrayList<>();
     while (itr.hasNext()) {
@@ -78,9 +80,7 @@ public abstract class NexmarkQueryModel implements Serializable {
     return strings;
   }
 
-  /**
-   * Convert {@code itr} to strings capturing values only.
-   */
+  /** Convert {@code itr} to strings capturing values only. */
   protected static <T> Set<String> toValue(Iterator<TimestampedValue<T>> itr) {
     Set<String> strings = new HashSet<>();
     while (itr.hasNext()) {
@@ -99,22 +99,23 @@ public abstract class NexmarkQueryModel implements Serializable {
   }
 
   /**
-   * Convert iterator of elements to collection of strings to use when testing coherence
-   * of model against actual query results.
+   * Convert iterator of elements to collection of strings to use when testing coherence of model
+   * against actual query results.
    */
   protected abstract <T> Collection<String> toCollection(Iterator<TimestampedValue<T>> itr);
 
-  /**
-   * Return assertion to use on results of pipeline for this query.
-   */
+  /** Return assertion to use on results of pipeline for this query. */
   public SerializableFunction<Iterable<TimestampedValue<KnownSize>>, Void> assertionFor() {
     final Collection<String> expectedStrings = toCollection(simulator().results());
+    final String[] expectedStringsArray = expectedStrings.toArray(new String[expectedStrings.size()]);
 
     return new SerializableFunction<Iterable<TimestampedValue<KnownSize>>, Void>() {
       @Override
       public Void apply(Iterable<TimestampedValue<KnownSize>> actual) {
         Collection<String> actualStrings = toCollection(relevantResults(actual).iterator());
-        Assert.assertEquals(expectedStrings, actualStrings);
+                Assert.assertThat("wrong pipeline output", actualStrings, IsEqual.equalTo(expectedStrings));
+//compare without order
+//        Assert.assertThat("wrong pipeline output", actualStrings, IsIterableContainingInAnyOrder.containsInAnyOrder(expectedStringsArray));
         return null;
       }
     };

http://git-wip-us.apache.org/repos/asf/beam/blob/1bd57351/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkUtils.java
----------------------------------------------------------------------
diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkUtils.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkUtils.java
index 8f4cb22..f7417d3 100644
--- a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkUtils.java
+++ b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkUtils.java
@@ -382,8 +382,7 @@ public class NexmarkUtils {
    */
   public static PTransform<PBegin, PCollection<Event>> batchEventsSource(
           NexmarkConfiguration configuration) {
-    return Read.from(new BoundedEventSource(
-            NexmarkUtils.standardGeneratorConfig(configuration), configuration.numEventGenerators));
+    return Read.from(new BoundedEventSource(standardGeneratorConfig(configuration), configuration.numEventGenerators));
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/beam/blob/1bd57351/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/Query0Model.java
----------------------------------------------------------------------
diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/Query0Model.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/Query0Model.java
index b7cdf1c..37e3f93 100644
--- a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/Query0Model.java
+++ b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/Query0Model.java
@@ -42,6 +42,7 @@ public class Query0Model extends NexmarkQueryModel {
         return;
       }
       addResult(timestampedEvent);
+      //TODO test fails because offset of some hundreds of ms beween expect and actual
     }
   }
 

http://git-wip-us.apache.org/repos/asf/beam/blob/1bd57351/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/Query1Model.java
----------------------------------------------------------------------
diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/Query1Model.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/Query1Model.java
index ace6f7e..16287e6 100644
--- a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/Query1Model.java
+++ b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/Query1Model.java
@@ -53,6 +53,7 @@ public class Query1Model extends NexmarkQueryModel implements Serializable {
       TimestampedValue<Bid> result =
           TimestampedValue.of(resultBid, timestampedEvent.getTimestamp());
       addResult(result);
+      //TODO test fails because offset of some hundreds of ms beween expect and actual
     }
   }
 

http://git-wip-us.apache.org/repos/asf/beam/blob/1bd57351/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/Query7Model.java
----------------------------------------------------------------------
diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/Query7Model.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/Query7Model.java
index 73e96e2..0033c68 100644
--- a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/Query7Model.java
+++ b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/Query7Model.java
@@ -107,6 +107,7 @@ public class Query7Model extends NexmarkQueryModel implements Serializable {
       }
       // Keep only the highest bids.
       captureBid(event.bid);
+      //TODO test fails because offset of some hundreds of ms between expect and actual
     }
   }
 

http://git-wip-us.apache.org/repos/asf/beam/blob/1bd57351/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/Query8Model.java
----------------------------------------------------------------------
diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/Query8Model.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/Query8Model.java
index fdd2a35..261e383 100644
--- a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/Query8Model.java
+++ b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/Query8Model.java
@@ -115,7 +115,7 @@ public class Query8Model extends NexmarkQueryModel implements Serializable {
           // Remember auction for future new people.
           newAuctions.put(event.newAuction.seller, event.newAuction);
         }
-      } else {
+      } else { // event is not an auction, nor a bid, so it is a person
         // Join new person with existing auctions.
         for (Auction auction : newAuctions.get(event.newPerson.id)) {
           addResult(auction, event.newPerson, timestamp);

http://git-wip-us.apache.org/repos/asf/beam/blob/1bd57351/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/WinningBidsSimulator.java
----------------------------------------------------------------------
diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/WinningBidsSimulator.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/WinningBidsSimulator.java
index 5970556..dc8094b 100644
--- a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/WinningBidsSimulator.java
+++ b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/WinningBidsSimulator.java
@@ -175,6 +175,7 @@ public class WinningBidsSimulator extends AbstractSimulator<Event, AuctionBid> {
         return;
       }
       addResult(result);
+      //TODO test fails because offset of some hundreds of ms beween expect and actual
       return;
     }
 

http://git-wip-us.apache.org/repos/asf/beam/blob/1bd57351/integration/java/nexmark/src/test/java/org/apache/beam/integration/nexmark/QueryTest.java
----------------------------------------------------------------------
diff --git a/integration/java/nexmark/src/test/java/org/apache/beam/integration/nexmark/QueryTest.java b/integration/java/nexmark/src/test/java/org/apache/beam/integration/nexmark/QueryTest.java
index d4d51f1..e481eac 100644
--- a/integration/java/nexmark/src/test/java/org/apache/beam/integration/nexmark/QueryTest.java
+++ b/integration/java/nexmark/src/test/java/org/apache/beam/integration/nexmark/QueryTest.java
@@ -25,6 +25,7 @@ import org.apache.beam.sdk.values.PCollection;
 import org.apache.beam.sdk.values.TimestampedValue;
 
 import org.junit.Ignore;
+import org.junit.Rule;
 import org.junit.Test;
 import org.junit.runner.RunWith;
 import org.junit.runners.JUnit4;
@@ -33,23 +34,23 @@ import org.junit.runners.JUnit4;
  * Test the various NEXMark queries yield results coherent with their models.
  */
 @RunWith(JUnit4.class)
-@Ignore
-//TODO Ismael
 public class QueryTest {
   private static final NexmarkConfiguration CONFIG = NexmarkConfiguration.DEFAULT.clone();
+  @Rule
+  public TestPipeline p = TestPipeline.create();
 
   static {
-    CONFIG.numEvents = 2000;
+    //careful, results of tests are linked to numEvents value
+    CONFIG.numEvents = 100;
   }
 
   /** Test {@code query} matches {@code model}. */
-  private static void queryMatchesModel(String name, NexmarkQuery query, NexmarkQueryModel model) {
-    Pipeline p = TestPipeline.create();
+  private void queryMatchesModel(String name, NexmarkQuery query, NexmarkQueryModel model) {
     NexmarkUtils.setupPipeline(NexmarkUtils.CoderStrategy.HAND, p);
     PCollection<TimestampedValue<KnownSize>> results =
         p.apply(name + ".ReadBounded", NexmarkUtils.batchEventsSource(CONFIG)).apply(query);
     //TODO Ismael this should not be called explicitly
-//    results.setIsBoundedInternal(IsBounded.BOUNDED);
+    results.setIsBoundedInternal(PCollection.IsBounded.BOUNDED);
     PAssert.that(results).satisfies(model.assertionFor());
     p.run().waitUntilFinish();
   }


[16/55] [abbrv] beam git commit: Refactor classes into packages

Posted by ie...@apache.org.
Refactor classes into packages

The new hierarchy has logically based packages for:
- drivers
- io
- model
- queries
- sources


Project: http://git-wip-us.apache.org/repos/asf/beam/repo
Commit: http://git-wip-us.apache.org/repos/asf/beam/commit/a7f9f7d0
Tree: http://git-wip-us.apache.org/repos/asf/beam/tree/a7f9f7d0
Diff: http://git-wip-us.apache.org/repos/asf/beam/diff/a7f9f7d0

Branch: refs/heads/master
Commit: a7f9f7d0784d9ba1f53ac4a0b49d2d81700720d0
Parents: 9ce9bf0
Author: Ismaël Mejía <ie...@apache.org>
Authored: Thu Mar 23 19:32:45 2017 +0100
Committer: Ismaël Mejía <ie...@gmail.com>
Committed: Wed Aug 23 19:07:27 2017 +0200

----------------------------------------------------------------------
 .../java/nexmark/BEAM_ON_FLINK_ON_GCP.md        |   2 +-
 integration/java/nexmark/README.md              |  88 +--
 integration/java/nexmark/pom.xml                |  22 +-
 .../integration/nexmark/AbstractSimulator.java  |   2 +-
 .../beam/integration/nexmark/Auction.java       | 189 ------
 .../beam/integration/nexmark/AuctionBid.java    |  86 ---
 .../beam/integration/nexmark/AuctionCount.java  |  89 ---
 .../beam/integration/nexmark/AuctionPrice.java  |  90 ---
 .../apache/beam/integration/nexmark/Bid.java    | 177 ------
 .../integration/nexmark/BidsPerSession.java     |  88 ---
 .../integration/nexmark/BoundedEventSource.java | 189 ------
 .../beam/integration/nexmark/CategoryPrice.java |  99 ----
 .../apache/beam/integration/nexmark/Done.java   |  82 ---
 .../apache/beam/integration/nexmark/Event.java  | 179 ------
 .../beam/integration/nexmark/Generator.java     | 589 ------------------
 .../integration/nexmark/GeneratorConfig.java    | 294 ---------
 .../beam/integration/nexmark/IdNameReserve.java |  99 ----
 .../beam/integration/nexmark/KnownSize.java     |  26 -
 .../beam/integration/nexmark/Monitor.java       |   1 +
 .../integration/nexmark/NameCityStateId.java    | 105 ----
 .../integration/nexmark/NexmarkApexDriver.java  |  48 --
 .../integration/nexmark/NexmarkApexRunner.java  |  61 --
 .../nexmark/NexmarkConfiguration.java           |   6 +-
 .../nexmark/NexmarkDirectDriver.java            |  47 --
 .../nexmark/NexmarkDirectRunner.java            |  58 --
 .../beam/integration/nexmark/NexmarkDriver.java |   5 +-
 .../integration/nexmark/NexmarkFlinkDriver.java |  48 --
 .../integration/nexmark/NexmarkFlinkRunner.java |  53 --
 .../nexmark/NexmarkGoogleDriver.java            |  62 --
 .../nexmark/NexmarkGoogleRunner.java            | 159 -----
 .../integration/nexmark/NexmarkOptions.java     | 386 ++++++++++++
 .../beam/integration/nexmark/NexmarkPerf.java   |   4 +-
 .../beam/integration/nexmark/NexmarkQuery.java  |   7 +-
 .../integration/nexmark/NexmarkQueryModel.java  |  19 +-
 .../beam/integration/nexmark/NexmarkRunner.java |  40 +-
 .../integration/nexmark/NexmarkSparkDriver.java |  46 --
 .../integration/nexmark/NexmarkSparkRunner.java |  54 --
 .../beam/integration/nexmark/NexmarkSuite.java  |   2 +-
 .../beam/integration/nexmark/NexmarkUtils.java  |  25 +-
 .../beam/integration/nexmark/Options.java       | 386 ------------
 .../apache/beam/integration/nexmark/Person.java | 165 ------
 .../beam/integration/nexmark/PubsubHelper.java  | 216 -------
 .../apache/beam/integration/nexmark/Query0.java |  67 ---
 .../beam/integration/nexmark/Query0Model.java   |  62 --
 .../apache/beam/integration/nexmark/Query1.java |  62 --
 .../beam/integration/nexmark/Query10.java       | 380 ------------
 .../beam/integration/nexmark/Query11.java       |  73 ---
 .../beam/integration/nexmark/Query12.java       |  77 ---
 .../beam/integration/nexmark/Query1Model.java   |  73 ---
 .../apache/beam/integration/nexmark/Query2.java |  73 ---
 .../beam/integration/nexmark/Query2Model.java   |  75 ---
 .../apache/beam/integration/nexmark/Query3.java | 249 --------
 .../beam/integration/nexmark/Query3Model.java   | 118 ----
 .../apache/beam/integration/nexmark/Query4.java | 107 ----
 .../beam/integration/nexmark/Query4Model.java   | 179 ------
 .../apache/beam/integration/nexmark/Query5.java | 123 ----
 .../beam/integration/nexmark/Query5Model.java   | 172 ------
 .../apache/beam/integration/nexmark/Query6.java | 151 -----
 .../beam/integration/nexmark/Query6Model.java   | 126 ----
 .../apache/beam/integration/nexmark/Query7.java |  85 ---
 .../beam/integration/nexmark/Query7Model.java   | 127 ----
 .../apache/beam/integration/nexmark/Query8.java |  91 ---
 .../beam/integration/nexmark/Query8Model.java   | 144 -----
 .../apache/beam/integration/nexmark/Query9.java |  39 --
 .../beam/integration/nexmark/Query9Model.java   |  43 --
 .../beam/integration/nexmark/SellerPrice.java   |  90 ---
 .../nexmark/UnboundedEventSource.java           | 328 ----------
 .../beam/integration/nexmark/WinningBids.java   |  11 +-
 .../nexmark/WinningBidsSimulator.java           |   4 +
 .../nexmark/drivers/NexmarkApexDriver.java      |  50 ++
 .../nexmark/drivers/NexmarkApexRunner.java      |  65 ++
 .../nexmark/drivers/NexmarkDirectDriver.java    |  49 ++
 .../nexmark/drivers/NexmarkDirectRunner.java    |  60 ++
 .../nexmark/drivers/NexmarkFlinkDriver.java     |  50 ++
 .../nexmark/drivers/NexmarkFlinkRunner.java     |  55 ++
 .../nexmark/drivers/NexmarkGoogleDriver.java    |  67 +++
 .../nexmark/drivers/NexmarkGoogleRunner.java    | 163 +++++
 .../nexmark/drivers/NexmarkSparkDriver.java     |  48 ++
 .../nexmark/drivers/NexmarkSparkRunner.java     |  56 ++
 .../nexmark/drivers/package-info.java           |  22 +
 .../integration/nexmark/io/PubsubHelper.java    | 217 +++++++
 .../integration/nexmark/io/package-info.java    |  22 +
 .../beam/integration/nexmark/model/Auction.java | 190 ++++++
 .../integration/nexmark/model/AuctionBid.java   |  88 +++
 .../integration/nexmark/model/AuctionCount.java |  90 +++
 .../integration/nexmark/model/AuctionPrice.java |  91 +++
 .../beam/integration/nexmark/model/Bid.java     | 178 ++++++
 .../nexmark/model/BidsPerSession.java           |  89 +++
 .../nexmark/model/CategoryPrice.java            | 100 ++++
 .../beam/integration/nexmark/model/Done.java    |  83 +++
 .../beam/integration/nexmark/model/Event.java   | 179 ++++++
 .../nexmark/model/IdNameReserve.java            | 100 ++++
 .../integration/nexmark/model/KnownSize.java    |  26 +
 .../nexmark/model/NameCityStateId.java          | 106 ++++
 .../beam/integration/nexmark/model/Person.java  | 166 ++++++
 .../integration/nexmark/model/SellerPrice.java  |  91 +++
 .../integration/nexmark/model/package-info.java |  22 +
 .../beam/integration/nexmark/package-info.java  |   2 +-
 .../integration/nexmark/queries/Query0.java     |  72 +++
 .../nexmark/queries/Query0Model.java            |  67 +++
 .../integration/nexmark/queries/Query1.java     |  68 +++
 .../integration/nexmark/queries/Query10.java    | 384 ++++++++++++
 .../integration/nexmark/queries/Query11.java    |  80 +++
 .../integration/nexmark/queries/Query12.java    |  84 +++
 .../nexmark/queries/Query1Model.java            |  79 +++
 .../integration/nexmark/queries/Query2.java     |  80 +++
 .../nexmark/queries/Query2Model.java            |  82 +++
 .../integration/nexmark/queries/Query3.java     | 256 ++++++++
 .../nexmark/queries/Query3Model.java            | 126 ++++
 .../integration/nexmark/queries/Query4.java     | 118 ++++
 .../nexmark/queries/Query4Model.java            | 188 ++++++
 .../integration/nexmark/queries/Query5.java     | 129 ++++
 .../nexmark/queries/Query5Model.java            | 178 ++++++
 .../integration/nexmark/queries/Query6.java     | 159 +++++
 .../nexmark/queries/Query6Model.java            | 135 +++++
 .../integration/nexmark/queries/Query7.java     |  91 +++
 .../nexmark/queries/Query7Model.java            | 133 +++++
 .../integration/nexmark/queries/Query8.java     |  98 +++
 .../nexmark/queries/Query8Model.java            | 150 +++++
 .../integration/nexmark/queries/Query9.java     |  46 ++
 .../nexmark/queries/Query9Model.java            |  47 ++
 .../nexmark/queries/package-info.java           |  22 +
 .../nexmark/sources/BoundedEventSource.java     | 190 ++++++
 .../integration/nexmark/sources/Generator.java  | 593 +++++++++++++++++++
 .../nexmark/sources/GeneratorConfig.java        | 296 +++++++++
 .../nexmark/sources/UnboundedEventSource.java   | 330 +++++++++++
 .../nexmark/sources/package-info.java           |  22 +
 .../nexmark/src/main/resources/log4j.properties |   4 +
 .../nexmark/BoundedEventSourceTest.java         |  70 ---
 .../beam/integration/nexmark/GeneratorTest.java | 110 ----
 .../beam/integration/nexmark/QueryTest.java     | 107 ----
 .../nexmark/UnboundedEventSourceTest.java       | 108 ----
 .../integration/nexmark/queries/QueryTest.java  | 111 ++++
 .../nexmark/sources/BoundedEventSourceTest.java |  71 +++
 .../nexmark/sources/GeneratorTest.java          | 111 ++++
 .../sources/UnboundedEventSourceTest.java       | 110 ++++
 136 files changed, 7768 insertions(+), 7384 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/beam/blob/a7f9f7d0/integration/java/nexmark/BEAM_ON_FLINK_ON_GCP.md
----------------------------------------------------------------------
diff --git a/integration/java/nexmark/BEAM_ON_FLINK_ON_GCP.md b/integration/java/nexmark/BEAM_ON_FLINK_ON_GCP.md
index d1b51e8..6a7fd34 100644
--- a/integration/java/nexmark/BEAM_ON_FLINK_ON_GCP.md
+++ b/integration/java/nexmark/BEAM_ON_FLINK_ON_GCP.md
@@ -243,7 +243,7 @@ $GCLOUD compute ssh \
   --zone=$ZONE \
   $MASTER \
   --command "~/$FLINK_VER/bin/flink run \
-  -c org.apache.beam.integration.nexmark.NexmarkFlinkDriver \
+  -c org.apache.beam.integration.nexmark.drivers.NexmarkFlinkDriver \
   ~/$FLINK_VER/lib/java-integration-all-bundled-0.2.0-incubating-SNAPSHOT.jar \
   --project=$PROJECT \
   --streaming=true \

http://git-wip-us.apache.org/repos/asf/beam/blob/a7f9f7d0/integration/java/nexmark/README.md
----------------------------------------------------------------------
diff --git a/integration/java/nexmark/README.md b/integration/java/nexmark/README.md
index 7a91ab2..a3549f4 100644
--- a/integration/java/nexmark/README.md
+++ b/integration/java/nexmark/README.md
@@ -74,14 +74,15 @@ We have augmented the original queries with five more:
 The queries can be executed using a 'Driver' for a given backend.
 Currently the supported drivers are:
 
+* **NexmarkApexDriver** for running via the Apex runner.
 * **NexmarkDirectDriver** for running locally on a single machine.
-* **NexmarkGoogleDriver** for running on the Google Cloud Dataflow
-  service. Requires a Google Cloud account.
+* **NexmarkGoogleDriver** for running on the Google Cloud Dataflow service.
+  Requires a Google Cloud account.
 * **NexmarkFlinkDriver** for running on a Flink cluster. Requires the
   cluster to be established and the Nexmark jar to be distributed to
   each worker.
 * **NexmarkSparkDriver** for running on a Spark cluster.
-  
+
 Other drivers are straightforward.
 
 Test data is deterministically synthesized on demand. The test
@@ -103,9 +104,21 @@ the Google Cloud Dataflow driver.
 
 # Configuration
 
-Common configuration parameters:
+## Common configuration parameters
+
+Decide if batch or streaming:
+
+    --streaming=true
+
+Number of events generators
+
+    --numEventGenerators=4
+
+Run query N
 
-Available Suites:
+    --query=N
+
+## Available Suites
 
 - DEFAULT: Test default configuration with query 0.
 - SMOKE: Run the 12 default configurations.
@@ -114,32 +127,39 @@ Available Suites:
 
         --suite=SMOKE
 
-Decide if batch or streaming:
-
-    --streaming=true
+### Apex specific configuration
 
-Number of events generators
+    --suite=SMOKE --manageResources=false --monitorJobs=true
 
-    --numEventGenerators=4
+### Dataflow specific configuration
 
-## Apex specific configuration
+    --query=0 --suite=SMOKE --manageResources=false --monitorJobs=true \
+    --enforceEncodability=false --enforceImmutability=false
+    --project=<your project> \
+    --zone=<your zone> \
+    --workerMachineType=n1-highmem-8 \
+    --stagingLocation=<a gs path for staging>
 
---suite=SMOKE --manageResources=false --monitorJobs=true
+    --runner=BlockingDataflowRunner \
+    --tempLocation=gs://talend-imejia/nexmark/temp/ \
+    --stagingLocation=gs://talend-imejia/nexmark/temp/staging/ \
+    --filesToStage=target/beam-integration-java-0.7.0-SNAPSHOT.jar
 
-## Direct specific configuration
+### Direct specific configuration
 
---suite=SMOKE --manageResources=false --monitorJobs=true \
---enforceEncodability=false --enforceImmutability=false
+    --suite=SMOKE --manageResources=false --monitorJobs=true \
+    --enforceEncodability=false --enforceImmutability=false
 
-## Flink specific configuration
+### Flink specific configuration
 
---suite=SMOKE --manageResources=false --monitorJobs=true \
---flinkMaster=local
+    --suite=SMOKE --manageResources=false --monitorJobs=true \
+    --flinkMaster=[local] --parallelism=#numcores
 
-## Spark specific configuration
+### Spark specific configuration
 
---suite=SMOKE --manageResources=false --monitorJobs=true --sparkMaster=local \
--Dspark.ui.enabled=false -DSPARK_LOCAL_IP=localhost -Dsun.io.serialization.extendedDebugInfo=true
+    --suite=SMOKE --manageResources=false --monitorJobs=true \
+    --sparkMaster=local \
+    -Dspark.ui.enabled=false -DSPARK_LOCAL_IP=localhost -Dsun.io.serialization.extendedDebugInfo=true
 
 # Current Status
 
@@ -149,19 +169,19 @@ Open issues are tracked [here](https://github.com../../../../../issues):
 
 | Query | Direct                         | Spark                          | Flink                          | Apex                            |
 | ----: | ------------------------------ | ------------------------------ | ------------------------------ | ------------------------------- |
-|     0 | ok                             | [#1](../../../../../issues/1)  | ok                             | ok                              |
-|     1 | ok                             | [#1](../../../../../issues/1)  | ok                             | ok                              |
-|     2 | ok                             | [#1](../../../../../issues/1)  | ok                             | ok                              |
+|     0 | ok                             | ok                             | ok                             | ok                              |
+|     1 | ok                             | ok                             | ok                             | ok                              |
+|     2 | ok                             | ok                             | ok                             | ok                              |
 |     3 | [#7](../../../../../issues/7)  | [#7](../../../../../issues/7)  | [#7](../../../../../issues/7)  | [#7](../../../../../issues/7)   |
 |     4 | ok                             | ok                             | [#2](../../../../../issues/2)  | ok                              |
-|     5 | ok                             | [#3](../../../../../issues/3)  | ok                             | ok                              |
+|     5 | ok                             | ok                             | ok                             | ok                              |
 |     6 | ok                             | ok                             | [#2](../../../../../issues/2)  | ok                              |
-|     7 | ok                             | [#1](../../../../../issues/1)  | ok                             | [#24](../../../../../issues/24) |
-|     8 | ok                             | [#1](../../../../../issues/1)  | ok                             | ok                              |
+|     7 | ok                             | ok                             | ok                             | [#24](../../../../../issues/24) |
+|     8 | ok                             | ok                             | ok                             | ok                              |
 |     9 | ok                             | ok                             | [#2](../../../../../issues/2)  | ok                              |
-|    10 | [#5](../../../../../issues/5)  | [#4](../../../../../issues/4)  | ok                             | ok                              |
-|    11 | ok                             | [#1](../../../../../issues/1)  | ok                             | ok                              |
-|    12 | ok                             | [#1](../../../../../issues/1)  | ok                             | ok                              |
+|    10 | [#5](../../../../../issues/5)  | ok                             | ok                             | ok                              |
+|    11 | ok                             | ok                             | ok                             | ok                              |
+|    12 | ok                             | ok                             | ok                             | ok                              |
 
 ## Streaming / Synthetic / Local
 
@@ -205,11 +225,11 @@ Batch Mode
 
 -Dexec.classpathScope="test"
 
-    mvn exec:java -Dexec.mainClass=org.apache.beam.integration.nexmark.NexmarkDirectDriver -Dexec.args="--suite=SMOKE --streaming=false --numEventGenerators=4 --manageResources=false --monitorJobs=false --enforceEncodability=false --enforceImmutability=false"
+    mvn exec:java -Dexec.mainClass=org.apache.beam.integration.nexmark.drivers.NexmarkDirectDriver -Dexec.args="--suite=SMOKE --streaming=false --numEventGenerators=4 --manageResources=false --monitorJobs=false --enforceEncodability=false --enforceImmutability=false"
 
 Streaming Mode
 
-    mvn exec:java -Dexec.mainClass=org.apache.beam.integration.nexmark.NexmarkDirectDriver -Dexec.args="--suite=SMOKE --streaming=true --numEventGenerators=4 --manageResources=false --monitorJobs=false --enforceEncodability=false --enforceImmutability=false"
+    mvn exec:java -Dexec.mainClass=org.apache.beam.integration.nexmark.drivers.NexmarkDirectDriver -Dexec.args="--suite=SMOKE --streaming=true --numEventGenerators=4 --manageResources=false --monitorJobs=false --enforceEncodability=false --enforceImmutability=false"
 
 ## Running on Google Cloud Dataflow
 
@@ -218,7 +238,7 @@ service.
 
 ```
 java -cp integration/java/target/java-integration-all-bundled-0.2.0-incubating-SNAPSHOT.jar \
-  org.apache.beam.integration.nexmark.NexmarkGoogleDriver \
+  org.apache.beam.integration.nexmark.drivers.NexmarkGoogleDriver \
   --project=<your project> \
   --zone=<your zone> \
   --workerMachineType=n1-highmem-8 \
@@ -251,7 +271,7 @@ java -cp integration/java/target/java-integration-all-bundled-0.2.0-incubating-S
 
 ```
 java -cp integration/java/target/java-integration-all-bundled-0.2.0-incubating-SNAPSHOT.jar \
-  org.apache.beam.integration.nexmark.NexmarkGoogleDriver \
+  org.apache.beam.integration.nexmark.drivers.NexmarkGoogleDriver \
   --project=<your project> \
   --zone=<your zone> \
   --workerMachineType=n1-highmem-8 \

http://git-wip-us.apache.org/repos/asf/beam/blob/a7f9f7d0/integration/java/nexmark/pom.xml
----------------------------------------------------------------------
diff --git a/integration/java/nexmark/pom.xml b/integration/java/nexmark/pom.xml
index 0ecc298..7cd7d39 100644
--- a/integration/java/nexmark/pom.xml
+++ b/integration/java/nexmark/pom.xml
@@ -61,11 +61,6 @@
 
       <plugin>
         <groupId>org.apache.maven.plugins</groupId>
-        <artifactId>maven-dependency-plugin</artifactId>
-      </plugin>
-
-      <plugin>
-        <groupId>org.apache.maven.plugins</groupId>
         <artifactId>maven-checkstyle-plugin</artifactId>
       </plugin>
 
@@ -139,7 +134,6 @@
       <plugin>
         <groupId>org.apache.maven.plugins</groupId>
         <artifactId>maven-dependency-plugin</artifactId>
-        <version>2.10</version>
         <executions>
           <execution>
             <goals><goal>analyze-only</goal></goals>
@@ -196,11 +190,13 @@
       <groupId>org.apache.spark</groupId>
       <artifactId>spark-core_2.10</artifactId>
       <version>${spark.version}</version>
+      <scope>runtime</scope>
     </dependency>
     <dependency>
       <groupId>org.apache.spark</groupId>
       <artifactId>spark-streaming_2.10</artifactId>
       <version>${spark.version}</version>
+      <scope>runtime</scope>
     </dependency>
 
     <!-- Apex runner -->
@@ -215,12 +211,6 @@
       <scope>runtime</scope>
     </dependency>
     <dependency>
-      <groupId>com.fasterxml.jackson.core</groupId>
-      <artifactId>jackson-databind</artifactId>
-      <version>${jackson.version}</version>
-      <scope>runtime</scope>
-    </dependency>
-    <dependency>
       <groupId>org.codehaus.jackson</groupId>
       <artifactId>jackson-mapper-asl</artifactId>
       <version>${apex.codehaus.jackson.version}</version>
@@ -244,6 +234,7 @@
       <groupId>com.google.apis</groupId>
       <artifactId>google-api-services-dataflow</artifactId>
       <version>${dataflow.version}</version>
+      <scope>runtime</scope>
     </dependency>
 
     <dependency>
@@ -289,13 +280,6 @@
     <dependency>
       <groupId>org.hamcrest</groupId>
       <artifactId>hamcrest-all</artifactId>
-      <version>${hamcrest.version}</version>
-    </dependency>
-
-    <dependency>
-      <groupId>org.hamcrest</groupId>
-      <artifactId>hamcrest-library</artifactId>
-      <version>${hamcrest.version}</version>
     </dependency>
 
     <dependency>

http://git-wip-us.apache.org/repos/asf/beam/blob/a7f9f7d0/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/AbstractSimulator.java
----------------------------------------------------------------------
diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/AbstractSimulator.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/AbstractSimulator.java
index c08cdd3..b012842 100644
--- a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/AbstractSimulator.java
+++ b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/AbstractSimulator.java
@@ -34,7 +34,7 @@ import org.joda.time.Instant;
  * @param <InputT> Type of input elements.
  * @param <OutputT> Type of output elements.
  */
-abstract class AbstractSimulator<InputT, OutputT> {
+public abstract class AbstractSimulator<InputT, OutputT> {
   /** Window size for action bucket sampling. */
   public static final Duration WINDOW_SIZE = Duration.standardMinutes(1);
 

http://git-wip-us.apache.org/repos/asf/beam/blob/a7f9f7d0/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/Auction.java
----------------------------------------------------------------------
diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/Auction.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/Auction.java
deleted file mode 100644
index 16c28aa..0000000
--- a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/Auction.java
+++ /dev/null
@@ -1,189 +0,0 @@
-/*
- * 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.beam.integration.nexmark;
-
-import com.fasterxml.jackson.annotation.JsonProperty;
-import com.fasterxml.jackson.core.JsonProcessingException;
-
-import java.io.IOException;
-import java.io.InputStream;
-import java.io.OutputStream;
-import java.io.Serializable;
-
-import org.apache.beam.sdk.coders.AtomicCoder;
-import org.apache.beam.sdk.coders.Coder;
-import org.apache.beam.sdk.coders.CoderException;
-import org.apache.beam.sdk.coders.StringUtf8Coder;
-import org.apache.beam.sdk.coders.VarLongCoder;
-
-/**
- * An auction submitted by a person.
- */
-public class Auction implements KnownSize, Serializable {
-  private static final Coder<Long> LONG_CODER = VarLongCoder.of();
-  private static final Coder<String> STRING_CODER = StringUtf8Coder.of();
-
-  public static final Coder<Auction> CODER = new AtomicCoder<Auction>() {
-    @Override
-    public void encode(Auction value, OutputStream outStream,
-        Coder.Context context)
-        throws CoderException, IOException {
-      LONG_CODER.encode(value.id, outStream, Context.NESTED);
-      STRING_CODER.encode(value.itemName, outStream, Context.NESTED);
-      STRING_CODER.encode(value.description, outStream, Context.NESTED);
-      LONG_CODER.encode(value.initialBid, outStream, Context.NESTED);
-      LONG_CODER.encode(value.reserve, outStream, Context.NESTED);
-      LONG_CODER.encode(value.dateTime, outStream, Context.NESTED);
-      LONG_CODER.encode(value.expires, outStream, Context.NESTED);
-      LONG_CODER.encode(value.seller, outStream, Context.NESTED);
-      LONG_CODER.encode(value.category, outStream, Context.NESTED);
-      STRING_CODER.encode(value.extra, outStream, Context.NESTED);
-    }
-
-    @Override
-    public Auction decode(
-        InputStream inStream, Coder.Context context)
-        throws CoderException, IOException {
-      long id = LONG_CODER.decode(inStream, Context.NESTED);
-      String itemName = STRING_CODER.decode(inStream, Context.NESTED);
-      String description = STRING_CODER.decode(inStream, Context.NESTED);
-      long initialBid = LONG_CODER.decode(inStream, Context.NESTED);
-      long reserve = LONG_CODER.decode(inStream, Context.NESTED);
-      long dateTime = LONG_CODER.decode(inStream, Context.NESTED);
-      long expires = LONG_CODER.decode(inStream, Context.NESTED);
-      long seller = LONG_CODER.decode(inStream, Context.NESTED);
-      long category = LONG_CODER.decode(inStream, Context.NESTED);
-      String extra = STRING_CODER.decode(inStream, Context.NESTED);
-      return new Auction(
-          id, itemName, description, initialBid, reserve, dateTime, expires, seller, category,
-          extra);
-    }
-  };
-
-
-  /** Id of auction. */
-  @JsonProperty
-  public final long id; // primary key
-
-  /** Extra auction properties. */
-  @JsonProperty
-  public final String itemName;
-
-  @JsonProperty
-  public final String description;
-
-  /** Initial bid price, in cents. */
-  @JsonProperty
-  public final long initialBid;
-
-  /** Reserve price, in cents. */
-  @JsonProperty
-  public final long reserve;
-
-  @JsonProperty
-  public final long dateTime;
-
-  /** When does auction expire? (ms since epoch). Bids at or after this time are ignored. */
-  @JsonProperty
-  public final long expires;
-
-  /** Id of person who instigated auction. */
-  @JsonProperty
-  public final long seller; // foreign key: Person.id
-
-  /** Id of category auction is listed under. */
-  @JsonProperty
-  public final long category; // foreign key: Category.id
-
-  /** Additional arbitrary payload for performance testing. */
-  @JsonProperty
-  public final String extra;
-
-
-  // For Avro only.
-  @SuppressWarnings("unused")
-  private Auction() {
-    id = 0;
-    itemName = null;
-    description = null;
-    initialBid = 0;
-    reserve = 0;
-    dateTime = 0;
-    expires = 0;
-    seller = 0;
-    category = 0;
-    extra = null;
-  }
-
-  public Auction(long id, String itemName, String description, long initialBid, long reserve,
-      long dateTime, long expires, long seller, long category, String extra) {
-    this.id = id;
-    this.itemName = itemName;
-    this.description = description;
-    this.initialBid = initialBid;
-    this.reserve = reserve;
-    this.dateTime = dateTime;
-    this.expires = expires;
-    this.seller = seller;
-    this.category = category;
-    this.extra = extra;
-  }
-
-  /**
-   * Return a copy of auction which capture the given annotation.
-   * (Used for debugging).
-   */
-  public Auction withAnnotation(String annotation) {
-    return new Auction(id, itemName, description, initialBid, reserve, dateTime, expires, seller,
-        category, annotation + ": " + extra);
-  }
-
-  /**
-   * Does auction have {@code annotation}? (Used for debugging.)
-   */
-  public boolean hasAnnotation(String annotation) {
-    return extra.startsWith(annotation + ": ");
-  }
-
-  /**
-   * Remove {@code annotation} from auction. (Used for debugging.)
-   */
-  public Auction withoutAnnotation(String annotation) {
-    if (hasAnnotation(annotation)) {
-      return new Auction(id, itemName, description, initialBid, reserve, dateTime, expires, seller,
-          category, extra.substring(annotation.length() + 2));
-    } else {
-      return this;
-    }
-  }
-
-  @Override
-  public long sizeInBytes() {
-    return 8 + itemName.length() + 1 + description.length() + 1 + 8 + 8 + 8 + 8 + 8 + 8
-        + extra.length() + 1;
-  }
-
-  @Override
-  public String toString() {
-    try {
-      return NexmarkUtils.MAPPER.writeValueAsString(this);
-    } catch (JsonProcessingException e) {
-      throw new RuntimeException(e);
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/a7f9f7d0/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/AuctionBid.java
----------------------------------------------------------------------
diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/AuctionBid.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/AuctionBid.java
deleted file mode 100644
index cd52b02..0000000
--- a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/AuctionBid.java
+++ /dev/null
@@ -1,86 +0,0 @@
-/*
- * 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.beam.integration.nexmark;
-
-import com.fasterxml.jackson.annotation.JsonProperty;
-import com.fasterxml.jackson.core.JsonProcessingException;
-
-import java.io.IOException;
-import java.io.InputStream;
-import java.io.OutputStream;
-import java.io.Serializable;
-
-import org.apache.beam.sdk.coders.AtomicCoder;
-import org.apache.beam.sdk.coders.Coder;
-import org.apache.beam.sdk.coders.CoderException;
-
-/**
- * Result of {@link WinningBids} transform.
- */
-public class AuctionBid implements KnownSize, Serializable {
-  public static final Coder<AuctionBid> CODER = new AtomicCoder<AuctionBid>() {
-    @Override
-    public void encode(AuctionBid value, OutputStream outStream,
-        Coder.Context context)
-        throws CoderException, IOException {
-      Auction.CODER.encode(value.auction, outStream, Context.NESTED);
-      Bid.CODER.encode(value.bid, outStream, Context.NESTED);
-    }
-
-    @Override
-    public AuctionBid decode(
-        InputStream inStream, Coder.Context context)
-        throws CoderException, IOException {
-      Auction auction = Auction.CODER.decode(inStream, Context.NESTED);
-      Bid bid = Bid.CODER.decode(inStream, Context.NESTED);
-      return new AuctionBid(auction, bid);
-    }
-  };
-
-  @JsonProperty
-  public final Auction auction;
-
-  @JsonProperty
-  public final Bid bid;
-
-  // For Avro only.
-  @SuppressWarnings("unused")
-  private AuctionBid() {
-    auction = null;
-    bid = null;
-  }
-
-  public AuctionBid(Auction auction, Bid bid) {
-    this.auction = auction;
-    this.bid = bid;
-  }
-
-  @Override
-  public long sizeInBytes() {
-    return auction.sizeInBytes() + bid.sizeInBytes();
-  }
-
-  @Override
-  public String toString() {
-    try {
-      return NexmarkUtils.MAPPER.writeValueAsString(this);
-    } catch (JsonProcessingException e) {
-      throw new RuntimeException(e);
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/a7f9f7d0/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/AuctionCount.java
----------------------------------------------------------------------
diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/AuctionCount.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/AuctionCount.java
deleted file mode 100644
index ac1f080..0000000
--- a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/AuctionCount.java
+++ /dev/null
@@ -1,89 +0,0 @@
-/*
- * 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.beam.integration.nexmark;
-
-import com.fasterxml.jackson.annotation.JsonProperty;
-import com.fasterxml.jackson.core.JsonProcessingException;
-
-import java.io.IOException;
-import java.io.InputStream;
-import java.io.OutputStream;
-import java.io.Serializable;
-
-import org.apache.beam.sdk.coders.AtomicCoder;
-import org.apache.beam.sdk.coders.Coder;
-import org.apache.beam.sdk.coders.CoderException;
-import org.apache.beam.sdk.coders.VarLongCoder;
-
-/**
- * Result of {@link Query5}.
- */
-public class AuctionCount implements KnownSize, Serializable {
-  private static final Coder<Long> LONG_CODER = VarLongCoder.of();
-
-  public static final Coder<AuctionCount> CODER = new AtomicCoder<AuctionCount>() {
-    @Override
-    public void encode(AuctionCount value, OutputStream outStream,
-        Coder.Context context)
-        throws CoderException, IOException {
-      LONG_CODER.encode(value.auction, outStream, Context.NESTED);
-      LONG_CODER.encode(value.count, outStream, Context.NESTED);
-    }
-
-    @Override
-    public AuctionCount decode(
-        InputStream inStream, Coder.Context context)
-        throws CoderException, IOException {
-      long auction = LONG_CODER.decode(inStream, Context.NESTED);
-      long count = LONG_CODER.decode(inStream, Context.NESTED);
-      return new AuctionCount(auction, count);
-    }
-  };
-
-  @JsonProperty
-  public final long auction;
-
-  @JsonProperty
-  public final long count;
-
-  // For Avro only.
-  @SuppressWarnings("unused")
-  private AuctionCount() {
-    auction = 0;
-    count = 0;
-  }
-
-  public AuctionCount(long auction, long count) {
-    this.auction = auction;
-    this.count = count;
-  }
-
-  @Override
-  public long sizeInBytes() {
-    return 8 + 8;
-  }
-
-  @Override
-  public String toString() {
-    try {
-      return NexmarkUtils.MAPPER.writeValueAsString(this);
-    } catch (JsonProcessingException e) {
-      throw new RuntimeException(e);
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/a7f9f7d0/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/AuctionPrice.java
----------------------------------------------------------------------
diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/AuctionPrice.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/AuctionPrice.java
deleted file mode 100644
index 9bdf11c..0000000
--- a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/AuctionPrice.java
+++ /dev/null
@@ -1,90 +0,0 @@
-/*
- * 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.beam.integration.nexmark;
-
-import com.fasterxml.jackson.annotation.JsonProperty;
-import com.fasterxml.jackson.core.JsonProcessingException;
-
-import java.io.IOException;
-import java.io.InputStream;
-import java.io.OutputStream;
-import java.io.Serializable;
-
-import org.apache.beam.sdk.coders.AtomicCoder;
-import org.apache.beam.sdk.coders.Coder;
-import org.apache.beam.sdk.coders.CoderException;
-import org.apache.beam.sdk.coders.VarLongCoder;
-
-/**
- * Result of {@link Query2}.
- */
-public class AuctionPrice implements KnownSize, Serializable {
-  private static final Coder<Long> LONG_CODER = VarLongCoder.of();
-
-  public static final Coder<AuctionPrice> CODER = new AtomicCoder<AuctionPrice>() {
-    @Override
-    public void encode(AuctionPrice value, OutputStream outStream,
-        Coder.Context context)
-        throws CoderException, IOException {
-      LONG_CODER.encode(value.auction, outStream, Context.NESTED);
-      LONG_CODER.encode(value.price, outStream, Context.NESTED);
-    }
-
-    @Override
-    public AuctionPrice decode(
-        InputStream inStream, Coder.Context context)
-        throws CoderException, IOException {
-      long auction = LONG_CODER.decode(inStream, Context.NESTED);
-      long price = LONG_CODER.decode(inStream, Context.NESTED);
-      return new AuctionPrice(auction, price);
-    }
-  };
-
-  @JsonProperty
-  public final long auction;
-
-  /** Price in cents. */
-  @JsonProperty
-  public final long price;
-
-  // For Avro only.
-  @SuppressWarnings("unused")
-  private AuctionPrice() {
-    auction = 0;
-    price = 0;
-  }
-
-  public AuctionPrice(long auction, long price) {
-    this.auction = auction;
-    this.price = price;
-  }
-
-  @Override
-  public long sizeInBytes() {
-    return 8 + 8;
-  }
-
-  @Override
-  public String toString() {
-    try {
-      return NexmarkUtils.MAPPER.writeValueAsString(this);
-    } catch (JsonProcessingException e) {
-      throw new RuntimeException(e);
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/a7f9f7d0/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/Bid.java
----------------------------------------------------------------------
diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/Bid.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/Bid.java
deleted file mode 100644
index 04fcfdd..0000000
--- a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/Bid.java
+++ /dev/null
@@ -1,177 +0,0 @@
-/*
- * 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.beam.integration.nexmark;
-
-import com.fasterxml.jackson.annotation.JsonProperty;
-import com.fasterxml.jackson.core.JsonProcessingException;
-
-import java.io.IOException;
-import java.io.InputStream;
-import java.io.OutputStream;
-import java.io.Serializable;
-import java.util.Comparator;
-
-import org.apache.beam.sdk.coders.AtomicCoder;
-import org.apache.beam.sdk.coders.Coder;
-import org.apache.beam.sdk.coders.CoderException;
-import org.apache.beam.sdk.coders.StringUtf8Coder;
-import org.apache.beam.sdk.coders.VarLongCoder;
-
-/**
- * A bid for an item on auction.
- */
-public class Bid implements KnownSize, Serializable {
-  private static final Coder<Long> LONG_CODER = VarLongCoder.of();
-  private static final Coder<String> STRING_CODER = StringUtf8Coder.of();
-
-  public static final Coder<Bid> CODER = new AtomicCoder<Bid>() {
-    @Override
-    public void encode(Bid value, OutputStream outStream,
-        Coder.Context context)
-        throws CoderException, IOException {
-      LONG_CODER.encode(value.auction, outStream, Context.NESTED);
-      LONG_CODER.encode(value.bidder, outStream, Context.NESTED);
-      LONG_CODER.encode(value.price, outStream, Context.NESTED);
-      LONG_CODER.encode(value.dateTime, outStream, Context.NESTED);
-      STRING_CODER.encode(value.extra, outStream, Context.NESTED);
-    }
-
-    @Override
-    public Bid decode(
-        InputStream inStream, Coder.Context context)
-        throws CoderException, IOException {
-      long auction = LONG_CODER.decode(inStream, Context.NESTED);
-      long bidder = LONG_CODER.decode(inStream, Context.NESTED);
-      long price = LONG_CODER.decode(inStream, Context.NESTED);
-      long dateTime = LONG_CODER.decode(inStream, Context.NESTED);
-      String extra = STRING_CODER.decode(inStream, Context.NESTED);
-      return new Bid(auction, bidder, price, dateTime, extra);
-    }
-  };
-
-  /**
-   * Comparator to order bids by ascending price then descending time
-   * (for finding winning bids).
-   */
-  public static final Comparator<Bid> PRICE_THEN_DESCENDING_TIME = new Comparator<Bid>() {
-    @Override
-    public int compare(Bid left, Bid right) {
-      int i = Double.compare(left.price, right.price);
-      if (i != 0) {
-        return i;
-      }
-      return Long.compare(right.dateTime, left.dateTime);
-    }
-  };
-
-  /**
-   * Comparator to order bids by ascending time then ascending price.
-   * (for finding most recent bids).
-   */
-  public static final Comparator<Bid> ASCENDING_TIME_THEN_PRICE = new Comparator<Bid>() {
-    @Override
-    public int compare(Bid left, Bid right) {
-      int i = Long.compare(left.dateTime, right.dateTime);
-      if (i != 0) {
-        return i;
-      }
-      return Double.compare(left.price, right.price);
-    }
-  };
-
-  /** Id of auction this bid is for. */
-  @JsonProperty
-  public final long auction; // foreign key: Auction.id
-
-  /** Id of person bidding in auction. */
-  @JsonProperty
-  public final long bidder; // foreign key: Person.id
-
-  /** Price of bid, in cents. */
-  @JsonProperty
-  public final long price;
-
-  /**
-   * Instant at which bid was made (ms since epoch).
-   * NOTE: This may be earlier than the system's event time.
-   */
-  @JsonProperty
-  public final long dateTime;
-
-  /** Additional arbitrary payload for performance testing. */
-  @JsonProperty
-  public final String extra;
-
-  // For Avro only.
-  @SuppressWarnings("unused")
-  private Bid() {
-    auction = 0;
-    bidder = 0;
-    price = 0;
-    dateTime = 0;
-    extra = null;
-  }
-
-  public Bid(long auction, long bidder, long price, long dateTime, String extra) {
-    this.auction = auction;
-    this.bidder = bidder;
-    this.price = price;
-    this.dateTime = dateTime;
-    this.extra = extra;
-  }
-
-  /**
-   * Return a copy of bid which capture the given annotation.
-   * (Used for debugging).
-   */
-  public Bid withAnnotation(String annotation) {
-    return new Bid(auction, bidder, price, dateTime, annotation + ": " + extra);
-  }
-
-  /**
-   * Does bid have {@code annotation}? (Used for debugging.)
-   */
-  public boolean hasAnnotation(String annotation) {
-    return extra.startsWith(annotation + ": ");
-  }
-
-  /**
-   * Remove {@code annotation} from bid. (Used for debugging.)
-   */
-  public Bid withoutAnnotation(String annotation) {
-    if (hasAnnotation(annotation)) {
-      return new Bid(auction, bidder, price, dateTime, extra.substring(annotation.length() + 2));
-    } else {
-      return this;
-    }
-  }
-
-  @Override
-  public long sizeInBytes() {
-    return 8 + 8 + 8 + 8 + extra.length() + 1;
-  }
-
-  @Override
-  public String toString() {
-    try {
-      return NexmarkUtils.MAPPER.writeValueAsString(this);
-    } catch (JsonProcessingException e) {
-      throw new RuntimeException(e);
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/a7f9f7d0/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/BidsPerSession.java
----------------------------------------------------------------------
diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/BidsPerSession.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/BidsPerSession.java
deleted file mode 100644
index c6b0fe3..0000000
--- a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/BidsPerSession.java
+++ /dev/null
@@ -1,88 +0,0 @@
-/*
- * 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.beam.integration.nexmark;
-
-import com.fasterxml.jackson.annotation.JsonProperty;
-import com.fasterxml.jackson.core.JsonProcessingException;
-
-import java.io.IOException;
-import java.io.InputStream;
-import java.io.OutputStream;
-import java.io.Serializable;
-
-import org.apache.beam.sdk.coders.AtomicCoder;
-import org.apache.beam.sdk.coders.Coder;
-import org.apache.beam.sdk.coders.CoderException;
-import org.apache.beam.sdk.coders.VarLongCoder;
-
-/**
- * Result of query 11.
- */
-public class BidsPerSession implements KnownSize, Serializable {
-  private static final Coder<Long> LONG_CODER = VarLongCoder.of();
-
-  public static final Coder<BidsPerSession> CODER = new AtomicCoder<BidsPerSession>() {
-    @Override
-    public void encode(BidsPerSession value, OutputStream outStream,
-        Coder.Context context)
-        throws CoderException, IOException {
-      LONG_CODER.encode(value.personId, outStream, Context.NESTED);
-      LONG_CODER.encode(value.bidsPerSession, outStream, Context.NESTED);
-    }
-
-    @Override
-    public BidsPerSession decode(
-        InputStream inStream, Coder.Context context)
-        throws CoderException, IOException {
-      long personId = LONG_CODER.decode(inStream, Context.NESTED);
-      long bidsPerSession = LONG_CODER.decode(inStream, Context.NESTED);
-      return new BidsPerSession(personId, bidsPerSession);
-    }
-  };
-
-  @JsonProperty
-  public final long personId;
-
-  @JsonProperty
-  public final long bidsPerSession;
-
-  public BidsPerSession() {
-    personId = 0;
-    bidsPerSession = 0;
-  }
-
-  public BidsPerSession(long personId, long bidsPerSession) {
-    this.personId = personId;
-    this.bidsPerSession = bidsPerSession;
-  }
-
-  @Override
-  public long sizeInBytes() {
-    // Two longs.
-    return 8 + 8;
-  }
-
-  @Override
-  public String toString() {
-    try {
-      return NexmarkUtils.MAPPER.writeValueAsString(this);
-    } catch (JsonProcessingException e) {
-      throw new RuntimeException(e);
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/a7f9f7d0/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/BoundedEventSource.java
----------------------------------------------------------------------
diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/BoundedEventSource.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/BoundedEventSource.java
deleted file mode 100644
index 7dc1bcc..0000000
--- a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/BoundedEventSource.java
+++ /dev/null
@@ -1,189 +0,0 @@
-/*
- * 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.beam.integration.nexmark;
-
-import java.io.IOException;
-import java.util.ArrayList;
-import java.util.List;
-import java.util.NoSuchElementException;
-import javax.annotation.Nullable;
-
-import org.apache.beam.sdk.coders.Coder;
-import org.apache.beam.sdk.io.BoundedSource;
-import org.apache.beam.sdk.options.PipelineOptions;
-import org.apache.beam.sdk.values.TimestampedValue;
-import org.joda.time.Instant;
-
-/**
- * A custom, bounded source of event records.
- */
-class BoundedEventSource extends BoundedSource<Event> {
-  /** Configuration we generate events against. */
-  private final GeneratorConfig config;
-
-  /** How many bounded sources to create. */
-  private final int numEventGenerators;
-
-  public BoundedEventSource(GeneratorConfig config, int numEventGenerators) {
-    this.config = config;
-    this.numEventGenerators = numEventGenerators;
-  }
-
-  /** A reader to pull events from the generator. */
-  private static class EventReader extends BoundedReader<Event> {
-    /**
-     * Event source we purporting to be reading from.
-     * (We can't use Java's capture-outer-class pointer since we must update
-     * this field on calls to splitAtFraction.)
-     */
-    private BoundedEventSource source;
-
-    /** Generator we are reading from. */
-    private final Generator generator;
-
-    private boolean reportedStop;
-
-    @Nullable
-    private TimestampedValue<Event> currentEvent;
-
-    public EventReader(BoundedEventSource source, GeneratorConfig config) {
-      this.source = source;
-      generator = new Generator(config);
-      reportedStop = false;
-    }
-
-    @Override
-    public synchronized boolean start() {
-      NexmarkUtils.info("starting bounded generator %s", generator);
-      return advance();
-    }
-
-    @Override
-    public synchronized boolean advance() {
-      if (!generator.hasNext()) {
-        // No more events.
-        if (!reportedStop) {
-          reportedStop = true;
-          NexmarkUtils.info("stopped bounded generator %s", generator);
-        }
-        return false;
-      }
-      currentEvent = generator.next();
-      return true;
-    }
-
-    @Override
-    public synchronized Event getCurrent() throws NoSuchElementException {
-      if (currentEvent == null) {
-        throw new NoSuchElementException();
-      }
-      return currentEvent.getValue();
-    }
-
-    @Override
-    public synchronized Instant getCurrentTimestamp() throws NoSuchElementException {
-      if (currentEvent == null) {
-        throw new NoSuchElementException();
-      }
-      return currentEvent.getTimestamp();
-    }
-
-    @Override
-    public void close() throws IOException {
-      // Nothing to close.
-    }
-
-    @Override
-    public synchronized Double getFractionConsumed() {
-      return generator.getFractionConsumed();
-    }
-
-    @Override
-    public synchronized BoundedSource<Event> getCurrentSource() {
-      return source;
-    }
-
-    @Override
-    @Nullable
-    public synchronized BoundedEventSource splitAtFraction(double fraction) {
-      long startId = generator.getCurrentConfig().getStartEventId();
-      long stopId = generator.getCurrentConfig().getStopEventId();
-      long size = stopId - startId;
-      long splitEventId = startId + Math.min((int) (size * fraction), size);
-      if (splitEventId <= generator.getNextEventId() || splitEventId == stopId) {
-        // Already passed this position or split results in left or right being empty.
-        NexmarkUtils.info("split failed for bounded generator %s at %f", generator, fraction);
-        return null;
-      }
-
-      NexmarkUtils.info("about to split bounded generator %s at %d", generator, splitEventId);
-
-      // Scale back the event space of the current generator, and return a generator config
-      // representing the event space we just 'stole' from the current generator.
-      GeneratorConfig remainingConfig = generator.splitAtEventId(splitEventId);
-
-      NexmarkUtils.info("split bounded generator into %s and %s", generator, remainingConfig);
-
-      // At this point
-      //   generator.events() ++ new Generator(remainingConfig).events()
-      //   == originalGenerator.events()
-
-      // We need a new source to represent the now smaller key space for this reader, so
-      // that we can maintain the invariant that
-      //   this.getCurrentSource().createReader(...)
-      // will yield the same output as this.
-      source = new BoundedEventSource(generator.getCurrentConfig(), source.numEventGenerators);
-
-      // Return a source from which we may read the 'stolen' event space.
-      return new BoundedEventSource(remainingConfig, source.numEventGenerators);
-    }
-  }
-
-  @Override
-  public List<BoundedEventSource> splitIntoBundles(
-      long desiredBundleSizeBytes, PipelineOptions options) {
-    NexmarkUtils.info("slitting bounded source %s into %d sub-sources", config, numEventGenerators);
-    List<BoundedEventSource> results = new ArrayList<>();
-    // Ignore desiredBundleSizeBytes and use numEventGenerators instead.
-    for (GeneratorConfig subConfig : config.split(numEventGenerators)) {
-      results.add(new BoundedEventSource(subConfig, 1));
-    }
-    return results;
-  }
-
-  @Override
-  public long getEstimatedSizeBytes(PipelineOptions options) {
-    return config.getEstimatedSizeBytes();
-  }
-
-  @Override
-  public EventReader createReader(PipelineOptions options) {
-    NexmarkUtils.info("creating initial bounded reader for %s", config);
-    return new EventReader(this, config);
-  }
-
-  @Override
-  public void validate() {
-    // Nothing to validate.
-  }
-
-  @Override
-  public Coder<Event> getDefaultOutputCoder() {
-    return Event.CODER;
-  }
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/a7f9f7d0/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/CategoryPrice.java
----------------------------------------------------------------------
diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/CategoryPrice.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/CategoryPrice.java
deleted file mode 100644
index c83fb17..0000000
--- a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/CategoryPrice.java
+++ /dev/null
@@ -1,99 +0,0 @@
-/*
- * 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.beam.integration.nexmark;
-
-import com.fasterxml.jackson.annotation.JsonProperty;
-import com.fasterxml.jackson.core.JsonProcessingException;
-
-import java.io.IOException;
-import java.io.InputStream;
-import java.io.OutputStream;
-import java.io.Serializable;
-
-import org.apache.beam.sdk.coders.AtomicCoder;
-import org.apache.beam.sdk.coders.Coder;
-import org.apache.beam.sdk.coders.CoderException;
-import org.apache.beam.sdk.coders.VarIntCoder;
-import org.apache.beam.sdk.coders.VarLongCoder;
-
-/**
- * Result of {@link Query4}.
- */
-public class CategoryPrice implements KnownSize, Serializable {
-  private static final Coder<Long> LONG_CODER = VarLongCoder.of();
-  private static final Coder<Integer> INT_CODER = VarIntCoder.of();
-
-  public static final Coder<CategoryPrice> CODER = new AtomicCoder<CategoryPrice>() {
-    @Override
-    public void encode(CategoryPrice value, OutputStream outStream,
-        Coder.Context context)
-        throws CoderException, IOException {
-      LONG_CODER.encode(value.category, outStream, Context.NESTED);
-      LONG_CODER.encode(value.price, outStream, Context.NESTED);
-      INT_CODER.encode(value.isLast ? 1 : 0, outStream, Context.NESTED);
-    }
-
-    @Override
-    public CategoryPrice decode(
-        InputStream inStream, Coder.Context context)
-        throws CoderException, IOException {
-      long category = LONG_CODER.decode(inStream, Context.NESTED);
-      long price = LONG_CODER.decode(inStream, Context.NESTED);
-      boolean isLast = INT_CODER.decode(inStream, context) != 0;
-      return new CategoryPrice(category, price, isLast);
-    }
-  };
-
-  @JsonProperty
-  public final long category;
-
-  /** Price in cents. */
-  @JsonProperty
-  public final long price;
-
-  @JsonProperty
-  public final boolean isLast;
-
-  // For Avro only.
-  @SuppressWarnings("unused")
-  private CategoryPrice() {
-    category = 0;
-    price = 0;
-    isLast = false;
-  }
-
-  public CategoryPrice(long category, long price, boolean isLast) {
-    this.category = category;
-    this.price = price;
-    this.isLast = isLast;
-  }
-
-  @Override
-  public long sizeInBytes() {
-    return 8 + 8 + 1;
-  }
-
-  @Override
-  public String toString() {
-    try {
-      return NexmarkUtils.MAPPER.writeValueAsString(this);
-    } catch (JsonProcessingException e) {
-      throw new RuntimeException(e);
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/a7f9f7d0/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/Done.java
----------------------------------------------------------------------
diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/Done.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/Done.java
deleted file mode 100644
index 3a045f9..0000000
--- a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/Done.java
+++ /dev/null
@@ -1,82 +0,0 @@
-/*
- * 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.beam.integration.nexmark;
-
-import com.fasterxml.jackson.annotation.JsonProperty;
-import com.fasterxml.jackson.core.JsonProcessingException;
-
-import java.io.IOException;
-import java.io.InputStream;
-import java.io.OutputStream;
-import java.io.Serializable;
-
-import org.apache.beam.sdk.coders.AtomicCoder;
-import org.apache.beam.sdk.coders.Coder;
-import org.apache.beam.sdk.coders.CoderException;
-import org.apache.beam.sdk.coders.StringUtf8Coder;
-
-/**
- * Result of query 10.
- */
-public class Done implements KnownSize, Serializable {
-  private static final Coder<String> STRING_CODER = StringUtf8Coder.of();
-
-  public static final Coder<Done> CODER = new AtomicCoder<Done>() {
-    @Override
-    public void encode(Done value, OutputStream outStream,
-        Coder.Context context)
-        throws CoderException, IOException {
-      STRING_CODER.encode(value.message, outStream, Context.NESTED);
-    }
-
-    @Override
-    public Done decode(
-        InputStream inStream, Coder.Context context)
-        throws CoderException, IOException {
-      String message = STRING_CODER.decode(inStream, Context.NESTED);
-      return new Done(message);
-    }
-  };
-
-  @JsonProperty
-  public final String message;
-
-  // For Avro only.
-  @SuppressWarnings("unused")
-  public Done() {
-    message = null;
-  }
-
-  public Done(String message) {
-    this.message = message;
-  }
-
-  @Override
-  public long sizeInBytes() {
-    return message.length();
-  }
-
-  @Override
-  public String toString() {
-    try {
-      return NexmarkUtils.MAPPER.writeValueAsString(this);
-    } catch (JsonProcessingException e) {
-      throw new RuntimeException(e);
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/a7f9f7d0/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/Event.java
----------------------------------------------------------------------
diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/Event.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/Event.java
deleted file mode 100644
index 769cedd..0000000
--- a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/Event.java
+++ /dev/null
@@ -1,179 +0,0 @@
-/*
- * 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.beam.integration.nexmark;
-
-import java.io.IOException;
-import java.io.InputStream;
-import java.io.OutputStream;
-import java.io.Serializable;
-import javax.annotation.Nullable;
-
-import org.apache.beam.sdk.coders.AtomicCoder;
-import org.apache.beam.sdk.coders.Coder;
-import org.apache.beam.sdk.coders.CoderException;
-import org.apache.beam.sdk.coders.VarIntCoder;
-
-/**
- * An event in the auction system, either a (new) {@link Person}, a (new) {@link Auction},
- * or a {@link Bid}.
- */
-public class Event implements KnownSize, Serializable {
-  private static final Coder<Integer> INT_CODER = VarIntCoder.of();
-
-  public static final Coder<Event> CODER = new AtomicCoder<Event>() {
-    @Override
-    public void encode(Event value, OutputStream outStream, Coder.Context context)
-        throws CoderException, IOException {
-      if (value.newPerson != null) {
-        INT_CODER.encode(0, outStream, Context.NESTED);
-        Person.CODER.encode(value.newPerson, outStream, Context.NESTED);
-      } else if (value.newAuction != null) {
-        INT_CODER.encode(1, outStream, Context.NESTED);
-        Auction.CODER.encode(value.newAuction, outStream, Context.NESTED);
-      } else if (value.bid != null) {
-        INT_CODER.encode(2, outStream, Context.NESTED);
-        Bid.CODER.encode(value.bid, outStream, Context.NESTED);
-      } else {
-        throw new RuntimeException("invalid event");
-      }
-    }
-
-    @Override
-    public Event decode(
-        InputStream inStream, Coder.Context context)
-        throws CoderException, IOException {
-      int tag = INT_CODER.decode(inStream, context);
-      if (tag == 0) {
-        Person person = Person.CODER.decode(inStream, Context.NESTED);
-        return new Event(person);
-      } else if (tag == 1) {
-        Auction auction = Auction.CODER.decode(inStream, Context.NESTED);
-        return new Event(auction);
-      } else if (tag == 2) {
-        Bid bid = Bid.CODER.decode(inStream, Context.NESTED);
-        return new Event(bid);
-      } else {
-        throw new RuntimeException("invalid event encoding");
-      }
-    }
-  };
-
-  @Nullable
-  @org.apache.avro.reflect.Nullable
-  public final Person newPerson;
-
-  @Nullable
-  @org.apache.avro.reflect.Nullable
-  public final Auction newAuction;
-
-  @Nullable
-  @org.apache.avro.reflect.Nullable
-  public final Bid bid;
-
-  // For Avro only.
-  @SuppressWarnings("unused")
-  private Event() {
-    newPerson = null;
-    newAuction = null;
-    bid = null;
-  }
-
-  public Event(Person newPerson) {
-    this.newPerson = newPerson;
-    newAuction = null;
-    bid = null;
-  }
-
-  public Event(Auction newAuction) {
-    newPerson = null;
-    this.newAuction = newAuction;
-    bid = null;
-  }
-
-  public Event(Bid bid) {
-    newPerson = null;
-    newAuction = null;
-    this.bid = bid;
-  }
-
-  /**
-   * Return a copy of event which captures {@code annotation}.
-   * (Used for debugging).
-   */
-  public Event withAnnotation(String annotation) {
-    if (newPerson != null) {
-      return new Event(newPerson.withAnnotation(annotation));
-    } else if (newAuction != null) {
-      return new Event(newAuction.withAnnotation(annotation));
-    } else {
-      return new Event(bid.withAnnotation(annotation));
-    }
-  }
-
-  /**
-   * Does event have {@code annotation}? (Used for debugging.)
-   */
-  public boolean hasAnnotation(String annotation) {
-    if (newPerson != null) {
-      return newPerson.hasAnnotation(annotation);
-    } else if (newAuction != null) {
-      return newAuction.hasAnnotation(annotation);
-    } else {
-      return bid.hasAnnotation(annotation);
-    }
-  }
-
-  /**
-   * Remove {@code annotation} from event. (Used for debugging.)
-   */
-  public Event withoutAnnotation(String annotation) {
-    if (newPerson != null) {
-      return new Event(newPerson.withoutAnnotation(annotation));
-    } else if (newAuction != null) {
-      return new Event(newAuction.withoutAnnotation(annotation));
-    } else {
-      return new Event(bid.withoutAnnotation(annotation));
-    }
-  }
-
-  @Override
-  public long sizeInBytes() {
-    if (newPerson != null) {
-      return 1 + newPerson.sizeInBytes();
-    } else if (newAuction != null) {
-      return 1 + newAuction.sizeInBytes();
-    } else if (bid != null) {
-      return 1 + bid.sizeInBytes();
-    } else {
-      throw new RuntimeException("invalid event");
-    }
-  }
-
-  @Override
-  public String toString() {
-    if (newPerson != null) {
-      return newPerson.toString();
-    } else if (newAuction != null) {
-      return newAuction.toString();
-    } else if (bid != null) {
-      return bid.toString();
-    } else {
-      throw new RuntimeException("invalid event");
-    }
-  }
-}


[17/55] [abbrv] beam git commit: Fix Apex driver and update execution matrix

Posted by ie...@apache.org.
Fix Apex driver and update execution matrix


Project: http://git-wip-us.apache.org/repos/asf/beam/repo
Commit: http://git-wip-us.apache.org/repos/asf/beam/commit/9ce9bf07
Tree: http://git-wip-us.apache.org/repos/asf/beam/tree/9ce9bf07
Diff: http://git-wip-us.apache.org/repos/asf/beam/diff/9ce9bf07

Branch: refs/heads/master
Commit: 9ce9bf076032e1c9aeb3a6dce806ad4b96127157
Parents: 1bd5735
Author: Ismaël Mejía <ie...@apache.org>
Authored: Tue Mar 21 18:29:20 2017 +0100
Committer: Ismaël Mejía <ie...@gmail.com>
Committed: Wed Aug 23 19:07:27 2017 +0200

----------------------------------------------------------------------
 integration/java/nexmark/README.md              | 109 +++++++++++--------
 integration/java/nexmark/pom.xml                |  27 ++++-
 .../integration/nexmark/NexmarkApexRunner.java  |   5 -
 .../nexmark/NexmarkDirectRunner.java            |   5 -
 .../integration/nexmark/NexmarkFlinkRunner.java |   5 -
 .../nexmark/NexmarkGoogleRunner.java            |   5 -
 .../beam/integration/nexmark/NexmarkRunner.java |   9 --
 .../integration/nexmark/NexmarkSparkRunner.java |   5 -
 .../apache/beam/integration/nexmark/Query5.java |   1 -
 .../nexmark/src/main/resources/log4j.properties |   9 ++
 .../nexmark/UnboundedEventSourceTest.java       |   4 +-
 11 files changed, 100 insertions(+), 84 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/beam/blob/9ce9bf07/integration/java/nexmark/README.md
----------------------------------------------------------------------
diff --git a/integration/java/nexmark/README.md b/integration/java/nexmark/README.md
index 4c08c28..7a91ab2 100644
--- a/integration/java/nexmark/README.md
+++ b/integration/java/nexmark/README.md
@@ -122,63 +122,80 @@ Number of events generators
 
     --numEventGenerators=4
 
-## Flink specific configuration
+## Apex specific configuration
 
---suite=SMOKE --manageResources=false --monitorJobs=false \
---flinkMaster=local
+--suite=SMOKE --manageResources=false --monitorJobs=true
 
 ## Direct specific configuration
 
---suite=SMOKE --manageResources=false --monitorJobs=false \
+--suite=SMOKE --manageResources=false --monitorJobs=true \
 --enforceEncodability=false --enforceImmutability=false
 
+## Flink specific configuration
+
+--suite=SMOKE --manageResources=false --monitorJobs=true \
+--flinkMaster=local
+
 ## Spark specific configuration
 
---suite=SMOKE
---manageResources=false --monitorJobs=false --sparkMaster=local
--Dspark.ui.enabled=false
--DSPARK_LOCAL_IP=localhost
--Dsun.io.serialization.extendedDebugInfo=true
+--suite=SMOKE --manageResources=false --monitorJobs=true --sparkMaster=local \
+-Dspark.ui.enabled=false -DSPARK_LOCAL_IP=localhost -Dsun.io.serialization.extendedDebugInfo=true
 
 # Current Status
 
-Open issues are currently opened on [github](https://github.com/iemejia/beam/issues):
-
-## Batch Mode / Synthetic / Local
-
-| Query | Direct | Spark  | Flink  | Apex   |
-| ----: | ------ | ------ | ------ | ------ |
-|     0 | Ok     |   #1   | Ok     |        |
-|     1 | Ok     |   #1   | Ok     |        |
-|     2 | Ok     | NEX-01 | Ok     |        |
-|     3 | NEX-07 | NEX-07 | NEX-07 |        |
-|     4 | Ok     | Ok     | NEX-02 |        |
-|     5 | Ok     | NEX-03 | Ok     |        |
-|     6 | Ok     | OK     | NEX-02 |        |
-|     7 | Ok     | NEX-01 | Ok     |        |
-|     8 | Ok     | NEX-01 | Ok     |        |
-|     9 | Ok     | OK     | NEX-02 |        |
-|    10 | NEX-05 | NEX-04 | Ok     |        |
-|    11 | Ok     | NEX-01 | Ok     |        |
-|    12 | Ok     | NEX-01 | Ok     |        |
-
-## Streaming Mode / Synthetic / Local
-
-| Query | Direct | Spark  | Flink  | Apex   |
-| ----: | ------ | ------ | ------ | ------ |
-|     0 | Ok     |        |        |        |
-|     1 | Ok     |        |        |        |
-|     2 | Ok     |        |        |        |
-|     3 | NEX-07 |        |        |        |
-|     4 | Ok     |        |        |        |
-|     5 | Ok     |        |        |        |
-|     6 | Ok     |        |        |        |
-|     7 | Ok     |        |        |        |
-|     8 | Ok     |        |        |        |
-|     9 | Ok     |        |        |        |
-|    10 | NEX-05 |        |        |        |
-|    11 | Ok     |        |        |        |
-|    12 | Ok     |        |        |        |
+Open issues are tracked [here](https://github.com../../../../../issues):
+
+## Batch / Synthetic / Local
+
+| Query | Direct                         | Spark                          | Flink                          | Apex                            |
+| ----: | ------------------------------ | ------------------------------ | ------------------------------ | ------------------------------- |
+|     0 | ok                             | [#1](../../../../../issues/1)  | ok                             | ok                              |
+|     1 | ok                             | [#1](../../../../../issues/1)  | ok                             | ok                              |
+|     2 | ok                             | [#1](../../../../../issues/1)  | ok                             | ok                              |
+|     3 | [#7](../../../../../issues/7)  | [#7](../../../../../issues/7)  | [#7](../../../../../issues/7)  | [#7](../../../../../issues/7)   |
+|     4 | ok                             | ok                             | [#2](../../../../../issues/2)  | ok                              |
+|     5 | ok                             | [#3](../../../../../issues/3)  | ok                             | ok                              |
+|     6 | ok                             | ok                             | [#2](../../../../../issues/2)  | ok                              |
+|     7 | ok                             | [#1](../../../../../issues/1)  | ok                             | [#24](../../../../../issues/24) |
+|     8 | ok                             | [#1](../../../../../issues/1)  | ok                             | ok                              |
+|     9 | ok                             | ok                             | [#2](../../../../../issues/2)  | ok                              |
+|    10 | [#5](../../../../../issues/5)  | [#4](../../../../../issues/4)  | ok                             | ok                              |
+|    11 | ok                             | [#1](../../../../../issues/1)  | ok                             | ok                              |
+|    12 | ok                             | [#1](../../../../../issues/1)  | ok                             | ok                              |
+
+## Streaming / Synthetic / Local
+
+| Query | Direct                         | Spark                          | Flink                          | Apex                           |
+| ----: | ------------------------------ | ------------------------------ | ------------------------------ | ------------------------------ |
+|     0 | ok                             |                                |                                | ok                             |
+|     1 | ok                             |                                |                                | ok                             |
+|     2 | ok                             |                                |                                | ok                             |
+|     3 | [#7](../../../../../issues/7)  |                                |                                | [#7](../../../../../issues/7)  |
+|     4 | ok                             |                                |                                | ok                             |
+|     5 | ok                             |                                |                                | ok                             |
+|     6 | ok                             |                                |                                | ok                             |
+|     7 | ok                             |                                |                                | ?                              |
+|     8 | ok                             |                                |                                | ok                             |
+|     9 | ok                             |                                |                                | ok                             |
+|    10 | [#5](../../../../../issues/5)  |                                |                                | ?                              |
+|    11 | ok                             |                                |                                | Ok                             |
+|    12 | ok                             |                                |                                | Ok                             |
+
+## Batch / Synthetic / Cluster
+
+TODO
+
+| Query | Dataflow                       | Spark                          | Flink                          | Apex                           |
+| ----: | ------------------------------ | ------------------------------ | ------------------------------ | ------------------------------ |
+|     0 |                                |                                |                                |                                |
+
+## Streaming / Synthetic / Cluster
+
+TODO
+
+| Query | Dataflow                       | Spark                          | Flink                          | Apex                           |
+| ----: | ------------------------------ | ------------------------------ | ------------------------------ | ------------------------------ |
+|     0 |                                |                                |                                |                                |
 
 # Running Nexmark
 

http://git-wip-us.apache.org/repos/asf/beam/blob/9ce9bf07/integration/java/nexmark/pom.xml
----------------------------------------------------------------------
diff --git a/integration/java/nexmark/pom.xml b/integration/java/nexmark/pom.xml
index 27abb0e..0ecc298 100644
--- a/integration/java/nexmark/pom.xml
+++ b/integration/java/nexmark/pom.xml
@@ -28,7 +28,7 @@
   </parent>
 
   <artifactId>beam-integration-java</artifactId>
-  <name>Apache Beam :: Integration Tests :: Java All</name>
+  <name>Apache Beam :: Integration Tests :: Java :: Nexmark</name>
 
   <packaging>jar</packaging>
 
@@ -37,6 +37,7 @@
     <project.reporting.outputEncoding>UTF-8</project.reporting.outputEncoding>
     <flink.version>1.2.0</flink.version>
     <spark.version>1.6.3</spark.version>
+    <apex.codehaus.jackson.version>1.9.3</apex.codehaus.jackson.version>
     <skipITs>true</skipITs>
   </properties>
 
@@ -207,6 +208,30 @@
       <groupId>org.apache.beam</groupId>
       <artifactId>beam-runners-apex</artifactId>
     </dependency>
+    <dependency>
+      <groupId>com.esotericsoftware.kryo</groupId>
+      <artifactId>kryo</artifactId>
+      <version>${apex.kryo.version}</version>
+      <scope>runtime</scope>
+    </dependency>
+    <dependency>
+      <groupId>com.fasterxml.jackson.core</groupId>
+      <artifactId>jackson-databind</artifactId>
+      <version>${jackson.version}</version>
+      <scope>runtime</scope>
+    </dependency>
+    <dependency>
+      <groupId>org.codehaus.jackson</groupId>
+      <artifactId>jackson-mapper-asl</artifactId>
+      <version>${apex.codehaus.jackson.version}</version>
+      <scope>runtime</scope>
+    </dependency>
+    <dependency>
+      <groupId>org.codehaus.jackson</groupId>
+      <artifactId>jackson-core-asl</artifactId>
+      <version>${apex.codehaus.jackson.version}</version>
+      <scope>runtime</scope>
+    </dependency>
 
     <!-- IOs -->
     <dependency>

http://git-wip-us.apache.org/repos/asf/beam/blob/9ce9bf07/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkApexRunner.java
----------------------------------------------------------------------
diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkApexRunner.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkApexRunner.java
index ea46082..3b8993a 100644
--- a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkApexRunner.java
+++ b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkApexRunner.java
@@ -39,11 +39,6 @@ public class NexmarkApexRunner extends NexmarkRunner<NexmarkApexDriver.NexmarkAp
   }
 
   @Override
-  protected boolean canMonitor() {
-    return false;
-  }
-
-  @Override
   protected void invokeBuilderForPublishOnlyPipeline(
       PipelineBuilder builder) {
     builder.build(options);

http://git-wip-us.apache.org/repos/asf/beam/blob/9ce9bf07/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkDirectRunner.java
----------------------------------------------------------------------
diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkDirectRunner.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkDirectRunner.java
index c70e41e..0119bbc 100644
--- a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkDirectRunner.java
+++ b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkDirectRunner.java
@@ -41,11 +41,6 @@ class NexmarkDirectRunner extends NexmarkRunner<NexmarkDirectDriver.NexmarkDirec
   }
 
   @Override
-  protected boolean canMonitor() {
-    return true;
-  }
-
-  @Override
   protected void invokeBuilderForPublishOnlyPipeline(PipelineBuilder builder) {
     throw new UnsupportedOperationException(
         "Cannot use --pubSubMode=COMBINED with DirectRunner");

http://git-wip-us.apache.org/repos/asf/beam/blob/9ce9bf07/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkFlinkRunner.java
----------------------------------------------------------------------
diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkFlinkRunner.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkFlinkRunner.java
index 8e22917..95ab1ad 100644
--- a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkFlinkRunner.java
+++ b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkFlinkRunner.java
@@ -37,11 +37,6 @@ public class NexmarkFlinkRunner extends NexmarkRunner<NexmarkFlinkDriver.Nexmark
   }
 
   @Override
-  protected boolean canMonitor() {
-    return true;
-  }
-
-  @Override
   protected void invokeBuilderForPublishOnlyPipeline(
       PipelineBuilder builder) {
     builder.build(options);

http://git-wip-us.apache.org/repos/asf/beam/blob/9ce9bf07/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkGoogleRunner.java
----------------------------------------------------------------------
diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkGoogleRunner.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkGoogleRunner.java
index 135d428..f4bfb1e 100644
--- a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkGoogleRunner.java
+++ b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkGoogleRunner.java
@@ -60,11 +60,6 @@ class NexmarkGoogleRunner extends NexmarkRunner<NexmarkGoogleDriver.NexmarkGoogl
   }
 
   @Override
-  protected boolean canMonitor() {
-    return true;
-  }
-
-  @Override
   protected String getJobId(PipelineResult job) {
     return ((DataflowPipelineJob) job).getJobId();
   }

http://git-wip-us.apache.org/repos/asf/beam/blob/9ce9bf07/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkRunner.java
----------------------------------------------------------------------
diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkRunner.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkRunner.java
index 8d4c1f1..d311dc4 100644
--- a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkRunner.java
+++ b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkRunner.java
@@ -158,11 +158,6 @@ public abstract class NexmarkRunner<OptionT extends Options> {
   protected abstract int maxNumWorkers();
 
   /**
-   * Return true if runner can monitor running jobs.
-   */
-  protected abstract boolean canMonitor();
-
-  /**
    * Return the current value for a long counter, or -1 if can't be retrieved.
    */
   protected long getLong(PipelineResult job, Aggregator<Long, Long> aggregator) {
@@ -1089,10 +1084,6 @@ public abstract class NexmarkRunner<OptionT extends Options> {
    */
   @Nullable
   public NexmarkPerf run(NexmarkConfiguration runConfiguration) {
-    if (options.getMonitorJobs() && !canMonitor()) {
-      throw new RuntimeException("Cannot use --monitorJobs with this runner since it does not "
-                                 + "support monitoring.");
-    }
     if (options.getManageResources() && !options.getMonitorJobs()) {
       throw new RuntimeException("If using --manageResources then must also use --monitorJobs.");
     }

http://git-wip-us.apache.org/repos/asf/beam/blob/9ce9bf07/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkSparkRunner.java
----------------------------------------------------------------------
diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkSparkRunner.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkSparkRunner.java
index 32fee30..30ae9ca 100644
--- a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkSparkRunner.java
+++ b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkSparkRunner.java
@@ -37,11 +37,6 @@ public class NexmarkSparkRunner extends NexmarkRunner<NexmarkSparkDriver.Nexmark
     }
 
     @Override
-    protected boolean canMonitor() {
-        return true;
-    }
-
-    @Override
     protected void invokeBuilderForPublishOnlyPipeline(
             PipelineBuilder builder) {
         builder.build(options);

http://git-wip-us.apache.org/repos/asf/beam/blob/9ce9bf07/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/Query5.java
----------------------------------------------------------------------
diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/Query5.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/Query5.java
index 9020494..2c9fb9b 100644
--- a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/Query5.java
+++ b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/Query5.java
@@ -67,7 +67,6 @@ class Query5 extends NexmarkQuery {
         // Count the number of bids per auction id.
         .apply(Count.<Long>perElement())
 
-      //TODO replace by simple key
       // We'll want to keep all auctions with the maximal number of bids.
         // Start by lifting each into a singleton list.
         .apply(name + ".ToSingletons",

http://git-wip-us.apache.org/repos/asf/beam/blob/9ce9bf07/integration/java/nexmark/src/main/resources/log4j.properties
----------------------------------------------------------------------
diff --git a/integration/java/nexmark/src/main/resources/log4j.properties b/integration/java/nexmark/src/main/resources/log4j.properties
index 9d20aea..bc09794 100644
--- a/integration/java/nexmark/src/main/resources/log4j.properties
+++ b/integration/java/nexmark/src/main/resources/log4j.properties
@@ -35,8 +35,17 @@ log4j.logger.org.apache.spark=WARN
 log4j.logger.org.spark-project=WARN
 log4j.logger.io.netty=INFO
 
+# Settings to quiet flink logs
 log4j.logger.org.apache.flink=WARN
 
+# Settings to quiet apex logs
+log4j.logger.org.apache.beam.runners.apex=INFO
+log4j.logger.com.datatorrent=ERROR
+log4j.logger.org.apache.hadoop.metrics2=WARN
+log4j.logger.org.apache.commons=WARN
+log4j.logger.org.apache.hadoop.security=WARN
+log4j.logger.org.apache.hadoop.util=WARN
+
 # SPARK-9183: Settings to avoid annoying messages when looking up nonexistent UDFs in SparkSQL with Hive support
 log4j.logger.org.apache.hadoop.hive.metastore.RetryingHMSHandler=FATAL
 log4j.logger.org.apache.hadoop.hive.ql.exec.FunctionRegistry=ERROR

http://git-wip-us.apache.org/repos/asf/beam/blob/9ce9bf07/integration/java/nexmark/src/test/java/org/apache/beam/integration/nexmark/UnboundedEventSourceTest.java
----------------------------------------------------------------------
diff --git a/integration/java/nexmark/src/test/java/org/apache/beam/integration/nexmark/UnboundedEventSourceTest.java b/integration/java/nexmark/src/test/java/org/apache/beam/integration/nexmark/UnboundedEventSourceTest.java
index 02761d6..35b3aed 100644
--- a/integration/java/nexmark/src/test/java/org/apache/beam/integration/nexmark/UnboundedEventSourceTest.java
+++ b/integration/java/nexmark/src/test/java/org/apache/beam/integration/nexmark/UnboundedEventSourceTest.java
@@ -87,8 +87,8 @@ public class UnboundedEventSourceTest {
     Generator modelGenerator = new Generator(config);
 
     EventIdChecker checker = new EventIdChecker();
-    Pipeline p = TestPipeline.create();
-    PipelineOptions options = p.getOptions();
+    PipelineOptions options = TestPipeline.testingPipelineOptions();
+    Pipeline p = TestPipeline.create(options);
     UnboundedEventSource source = new UnboundedEventSource(config, 1, 0, false);
     UnboundedReader<Event> reader = source.createReader(options, null);
 


[05/55] [abbrv] beam git commit: NexMark

Posted by ie...@apache.org.
http://git-wip-us.apache.org/repos/asf/beam/blob/1f08970a/integration/java/src/main/java/org/apache/beam/integration/nexmark/Generator.java
----------------------------------------------------------------------
diff --git a/integration/java/src/main/java/org/apache/beam/integration/nexmark/Generator.java b/integration/java/src/main/java/org/apache/beam/integration/nexmark/Generator.java
new file mode 100644
index 0000000..98f4f00
--- /dev/null
+++ b/integration/java/src/main/java/org/apache/beam/integration/nexmark/Generator.java
@@ -0,0 +1,590 @@
+/*
+ * 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.beam.integration.nexmark;
+
+import org.apache.beam.sdk.coders.AtomicCoder;
+import org.apache.beam.sdk.coders.Coder;
+import org.apache.beam.sdk.coders.CoderException;
+import org.apache.beam.sdk.coders.VarLongCoder;
+import org.apache.beam.sdk.io.UnboundedSource;
+import org.apache.beam.sdk.values.TimestampedValue;
+import com.google.common.base.Preconditions;
+
+import org.joda.time.Instant;
+
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
+import java.io.Serializable;
+import java.util.Arrays;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Random;
+
+/**
+ * A generator for synthetic events. We try to make the data vaguely reasonable. We also ensure
+ * most primary key/foreign key relations are correct. Eg: a {@link Bid} event will usually have
+ * valid auction and bidder ids which can be joined to already-generated Auction and Person events.
+ *
+ * <p>To help with testing, we generate timestamps relative to a given {@code baseTime}. Each new
+ * event is given a timestamp advanced from the previous timestamp by {@code interEventDelayUs}
+ * (in microseconds). The event stream is thus fully deterministic and does not depend on
+ * wallclock time.
+ *
+ * <p>This class implements {@link org.apache.beam.sdk.io.UnboundedSource.CheckpointMark}
+ * so that we can resume generating events from a saved snapshot.
+ */
+public class Generator implements Iterator<TimestampedValue<Event>>, Serializable {
+  /**
+   * Keep the number of categories small so the example queries will find results even with
+   * a small batch of events.
+   */
+  private static final int NUM_CATEGORIES = 5;
+
+  /** Smallest random string size. */
+  private static final int MIN_STRING_LENGTH = 3;
+
+  /**
+   * Keep the number of states small so that the example queries will find results even with
+   * a small batch of events.
+   */
+  private static final List<String> US_STATES = Arrays.asList(("AZ,CA,ID,OR,WA,WY").split(","));
+
+  private static final List<String> US_CITIES =
+      Arrays.asList(
+          ("Phoenix,Los Angeles,San Francisco,Boise,Portland,Bend,Redmond,Seattle,Kent,Cheyenne")
+              .split(","));
+
+  private static final List<String> FIRST_NAMES =
+      Arrays.asList(("Peter,Paul,Luke,John,Saul,Vicky,Kate,Julie,Sarah,Deiter,Walter").split(","));
+
+  private static final List<String> LAST_NAMES =
+      Arrays.asList(("Shultz,Abrams,Spencer,White,Bartels,Walton,Smith,Jones,Noris").split(","));
+
+  /**
+   * Number of yet-to-be-created people and auction ids allowed.
+   */
+  private static final int PERSON_ID_LEAD = 10;
+  private static final int AUCTION_ID_LEAD = 10;
+
+  /**
+   * Fraction of people/auctions which may be 'hot' sellers/bidders/auctions are 1
+   * over these values.
+   */
+  private static final int HOT_AUCTION_RATIO = 100;
+  private static final int HOT_SELLER_RATIO = 100;
+  private static final int HOT_BIDDER_RATIO = 100;
+
+  /**
+   * Just enough state to be able to restore a generator back to where it was checkpointed.
+   */
+  public static class Checkpoint implements UnboundedSource.CheckpointMark {
+    private static final Coder<Long> LONG_CODER = VarLongCoder.of();
+
+    /** Coder for this class. */
+    public static final Coder<Checkpoint> CODER_INSTANCE =
+        new AtomicCoder<Checkpoint>() {
+          @Override
+          public void encode(
+              Checkpoint value,
+              OutputStream outStream,
+              Coder.Context context)
+              throws CoderException, IOException {
+            LONG_CODER.encode(value.numEvents, outStream, Context.NESTED);
+            LONG_CODER.encode(value.wallclockBaseTime, outStream, Context.NESTED);
+          }
+
+          @Override
+          public Checkpoint decode(
+              InputStream inStream, Coder.Context context)
+              throws CoderException, IOException {
+            long numEvents = LONG_CODER.decode(inStream, Context.NESTED);
+            long wallclockBaseTime = LONG_CODER.decode(inStream, Context.NESTED);
+            return new Checkpoint(numEvents, wallclockBaseTime);
+          }
+        };
+
+    private long numEvents;
+    private long wallclockBaseTime;
+
+    private Checkpoint(long numEvents, long wallclockBaseTime) {
+      this.numEvents = numEvents;
+      this.wallclockBaseTime = wallclockBaseTime;
+    }
+
+    public Generator toGenerator(GeneratorConfig config) {
+      return new Generator(config, numEvents, wallclockBaseTime);
+    }
+
+    @Override
+    public void finalizeCheckpoint() throws IOException {
+      // Nothing to finalize.
+    }
+
+    @Override
+    public String toString() {
+      return String.format("Generator.Checkpoint{numEvents:%d;wallclockBaseTime:%d}",
+          numEvents, wallclockBaseTime);
+    }
+  }
+
+  /**
+   * The next event and its various timestamps. Ordered by increasing wallclock timestamp, then
+   * (arbitrary but stable) event hash order.
+   */
+  public static class NextEvent implements Comparable<NextEvent> {
+    /** When, in wallclock time, should this event be emitted? */
+    public final long wallclockTimestamp;
+
+    /** When, in event time, should this event be considered to have occured? */
+    public final long eventTimestamp;
+
+    /** The event itself. */
+    public final Event event;
+
+    /** The minimum of this and all future event timestamps. */
+    public final long watermark;
+
+    public NextEvent(long wallclockTimestamp, long eventTimestamp, Event event, long watermark) {
+      this.wallclockTimestamp = wallclockTimestamp;
+      this.eventTimestamp = eventTimestamp;
+      this.event = event;
+      this.watermark = watermark;
+    }
+
+    /**
+     * Return a deep clone of next event with delay added to wallclock timestamp and
+     * event annotate as 'LATE'.
+     */
+    public NextEvent withDelay(long delayMs) {
+      return new NextEvent(
+          wallclockTimestamp + delayMs, eventTimestamp, event.withAnnotation("LATE"), watermark);
+    }
+
+    @Override
+    public int compareTo(NextEvent other) {
+      int i = Long.compare(wallclockTimestamp, other.wallclockTimestamp);
+      if (i != 0) {
+        return i;
+      }
+      return Integer.compare(event.hashCode(), other.event.hashCode());
+    }
+  }
+
+  /**
+   * Configuration to generate events against. Note that it may be replaced by a call to
+   * {@link #splitAtEventId}.
+   */
+  private GeneratorConfig config;
+
+  /** Number of events generated by this generator. */
+  private long numEvents;
+
+  /**
+   * Wallclock time at which we emitted the first event (ms since epoch). Initially -1.
+   */
+  private long wallclockBaseTime;
+
+  private Generator(GeneratorConfig config, long numEvents, long wallclockBaseTime) {
+    Preconditions.checkNotNull(config);
+    this.config = config;
+    this.numEvents = numEvents;
+    this.wallclockBaseTime = wallclockBaseTime;
+  }
+
+  /**
+   * Create a fresh generator according to {@code config}.
+   */
+  public Generator(GeneratorConfig config) {
+    this(config, 0, -1);
+  }
+
+  /**
+   * Return a checkpoint for the current generator.
+   */
+  public Checkpoint toCheckpoint() {
+    return new Checkpoint(numEvents, wallclockBaseTime);
+  }
+
+  /**
+   * Return a deep clone of this generator.
+   */
+  @Override
+  public Generator clone() {
+    return new Generator(config.clone(), numEvents, wallclockBaseTime);
+  }
+
+  /**
+   * Return the current config for this generator. Note that configs may be replaced by {@link
+   * #splitAtEventId}.
+   */
+  public GeneratorConfig getCurrentConfig() {
+    return config;
+  }
+
+  /**
+   * Mutate this generator so that it will only generate events up to but not including
+   * {@code eventId}. Return a config to represent the events this generator will no longer yield.
+   * The generators will run in on a serial timeline.
+   */
+  public GeneratorConfig splitAtEventId(long eventId) {
+    long newMaxEvents = eventId - (config.firstEventId + config.firstEventNumber);
+    GeneratorConfig remainConfig = config.cloneWith(config.firstEventId,
+        config.maxEvents - newMaxEvents, config.firstEventNumber + newMaxEvents);
+    config = config.cloneWith(config.firstEventId, newMaxEvents, config.firstEventNumber);
+    return remainConfig;
+  }
+
+  /**
+   * Return the next 'event id'. Though events don't have ids we can simulate them to
+   * help with bookkeeping.
+   */
+  public long getNextEventId() {
+    return config.firstEventId + config.nextAdjustedEventNumber(numEvents);
+  }
+
+  /**
+   * Return the last valid person id (ignoring FIRST_PERSON_ID). Will be the current person id if
+   * due to generate a person.
+   */
+  private long lastBase0PersonId() {
+    long eventId = getNextEventId();
+    long epoch = eventId / GeneratorConfig.PROPORTION_DENOMINATOR;
+    long offset = eventId % GeneratorConfig.PROPORTION_DENOMINATOR;
+    if (offset >= GeneratorConfig.PERSON_PROPORTION) {
+      // About to generate an auction or bid.
+      // Go back to the last person generated in this epoch.
+      offset = GeneratorConfig.PERSON_PROPORTION - 1;
+    }
+    // About to generate a person.
+    return epoch * GeneratorConfig.PERSON_PROPORTION + offset;
+  }
+
+  /**
+   * Return the last valid auction id (ignoring FIRST_AUCTION_ID). Will be the current auction id if
+   * due to generate an auction.
+   */
+  private long lastBase0AuctionId() {
+    long eventId = getNextEventId();
+    long epoch = eventId / GeneratorConfig.PROPORTION_DENOMINATOR;
+    long offset = eventId % GeneratorConfig.PROPORTION_DENOMINATOR;
+    if (offset < GeneratorConfig.PERSON_PROPORTION) {
+      // About to generate a person.
+      // Go back to the last auction in the last epoch.
+      epoch--;
+      offset = GeneratorConfig.AUCTION_PROPORTION - 1;
+    } else if (offset >= GeneratorConfig.PERSON_PROPORTION + GeneratorConfig.AUCTION_PROPORTION) {
+      // About to generate a bid.
+      // Go back to the last auction generated in this epoch.
+      offset = GeneratorConfig.AUCTION_PROPORTION - 1;
+    } else {
+      // About to generate an auction.
+      offset -= GeneratorConfig.PERSON_PROPORTION;
+    }
+    return epoch * GeneratorConfig.AUCTION_PROPORTION + offset;
+  }
+
+  /** return a random US state. */
+  private static String nextUSState(Random random) {
+    return US_STATES.get(random.nextInt(US_STATES.size()));
+  }
+
+  /** Return a random US city. */
+  private static String nextUSCity(Random random) {
+    return US_CITIES.get(random.nextInt(US_CITIES.size()));
+  }
+
+  /** Return a random person name. */
+  private static String nextPersonName(Random random) {
+    return FIRST_NAMES.get(random.nextInt(FIRST_NAMES.size())) + " "
+        + LAST_NAMES.get(random.nextInt(LAST_NAMES.size()));
+  }
+
+  /** Return a random string of up to {@code maxLength}. */
+  private static String nextString(Random random, int maxLength) {
+    int len = MIN_STRING_LENGTH + random.nextInt(maxLength - MIN_STRING_LENGTH);
+    StringBuilder sb = new StringBuilder();
+    while (len-- > 0) {
+      if (random.nextInt(13) == 0) {
+        sb.append(' ');
+      } else {
+        sb.append((char) ('a' + random.nextInt(26)));
+      }
+    }
+    return sb.toString().trim();
+  }
+
+  /** Return a random string of exactly {@code length}. */
+  private static String nextExactString(Random random, int length) {
+    StringBuilder sb = new StringBuilder();
+    while (length-- > 0) {
+      sb.append((char) ('a' + random.nextInt(26)));
+    }
+    return sb.toString();
+  }
+
+  /** Return a random email address. */
+  private static String nextEmail(Random random) {
+    return nextString(random, 7) + "@" + nextString(random, 5) + ".com";
+  }
+
+  /** Return a random credit card number. */
+  private static String nextCreditCard(Random random) {
+    StringBuilder sb = new StringBuilder();
+    for (int i = 0; i < 4; i++) {
+      if (i > 0) {
+        sb.append(' ');
+      }
+      sb.append(String.format("%04d", random.nextInt(10000)));
+    }
+    return sb.toString();
+  }
+
+  /** Return a random price. */
+  private static long nextPrice(Random random) {
+    return Math.round(Math.pow(10.0, random.nextDouble() * 6.0) * 100.0);
+  }
+
+  /** Return a random time delay, in milliseconds, for length of auctions. */
+  private long nextAuctionLengthMs(Random random, long timestamp) {
+    // What's our current event number?
+    long currentEventNumber = config.nextAdjustedEventNumber(numEvents);
+    // How many events till we've generated numInFlightAuctions?
+    long numEventsForAuctions =
+        (config.configuration.numInFlightAuctions * GeneratorConfig.PROPORTION_DENOMINATOR)
+        / GeneratorConfig.AUCTION_PROPORTION;
+    // When will the auction numInFlightAuctions beyond now be generated?
+    long futureAuction =
+        config.timestampAndInterEventDelayUsForEvent(currentEventNumber + numEventsForAuctions)
+            .getKey();
+    // System.out.printf("*** auction will be for %dms (%d events ahead) ***\n",
+    //     futureAuction - timestamp, numEventsForAuctions);
+    // Choose a length with average horizonMs.
+    long horizonMs = futureAuction - timestamp;
+    return 1L + nextLong(random, Math.max(horizonMs * 2, 1L));
+  }
+
+  /**
+   * Return a random {@code string} such that {@code currentSize + string.length()} is on average
+   * {@code averageSize}.
+   */
+  private static String nextExtra(Random random, int currentSize, int desiredAverageSize) {
+    if (currentSize > desiredAverageSize) {
+      return "";
+    }
+    desiredAverageSize -= currentSize;
+    int delta = (int) Math.round(desiredAverageSize * 0.2);
+    int minSize = desiredAverageSize - delta;
+    int desiredSize = minSize + (delta == 0 ? 0 : random.nextInt(2 * delta));
+    return nextExactString(random, desiredSize);
+  }
+
+  /** Return a random long from {@code [0, n)}. */
+  private static long nextLong(Random random, long n) {
+    if (n < Integer.MAX_VALUE) {
+      return random.nextInt((int) n);
+    } else {
+      // TODO: Very skewed distribution! Bad!
+      return Math.abs(random.nextLong()) % n;
+    }
+  }
+
+  /**
+   * Generate and return a random person with next available id.
+   */
+  private Person nextPerson(Random random, long timestamp) {
+    long id = lastBase0PersonId() + GeneratorConfig.FIRST_PERSON_ID;
+    String name = nextPersonName(random);
+    String email = nextEmail(random);
+    String creditCard = nextCreditCard(random);
+    String city = nextUSCity(random);
+    String state = nextUSState(random);
+    int currentSize =
+        8 + name.length() + email.length() + creditCard.length() + city.length() + state.length();
+    String extra = nextExtra(random, currentSize, config.configuration.avgPersonByteSize);
+    return new Person(id, name, email, creditCard, city, state, timestamp, extra);
+  }
+
+  /**
+   * Return a random person id (base 0).
+   */
+  private long nextBase0PersonId(Random random) {
+    // Choose a random person from any of the 'active' people, plus a few 'leads'.
+    // By limiting to 'active' we ensure the density of bids or auctions per person
+    // does not decrease over time for long running jobs.
+    // By choosing a person id ahead of the last valid person id we will make
+    // newPerson and newAuction events appear to have been swapped in time.
+    long numPeople = lastBase0PersonId() + 1;
+    long activePeople = Math.min(numPeople, config.configuration.numActivePeople);
+    long n = nextLong(random, activePeople + PERSON_ID_LEAD);
+    return numPeople - activePeople + n;
+  }
+
+  /**
+   * Return a random auction id (base 0).
+   */
+  private long nextBase0AuctionId(Random random) {
+    // Choose a random auction for any of those which are likely to still be in flight,
+    // plus a few 'leads'.
+    // Note that ideally we'd track non-expired auctions exactly, but that state
+    // is difficult to split.
+    long minAuction = Math.max(lastBase0AuctionId() - config.configuration.numInFlightAuctions, 0);
+    long maxAuction = lastBase0AuctionId();
+    return minAuction + nextLong(random, maxAuction - minAuction + 1 + AUCTION_ID_LEAD);
+  }
+
+  /**
+   * Generate and return a random auction with next available id.
+   */
+  private Auction nextAuction(Random random, long timestamp) {
+    long id = lastBase0AuctionId() + GeneratorConfig.FIRST_AUCTION_ID;
+
+    long seller;
+    // Here P(auction will be for a hot seller) = 1 - 1/hotSellersRatio.
+    if (random.nextInt(config.configuration.hotSellersRatio) > 0) {
+      // Choose the first person in the batch of last HOT_SELLER_RATIO people.
+      seller = (lastBase0PersonId() / HOT_SELLER_RATIO) * HOT_SELLER_RATIO;
+    } else {
+      seller = nextBase0PersonId(random);
+    }
+    seller += GeneratorConfig.FIRST_PERSON_ID;
+
+    long category = GeneratorConfig.FIRST_CATEGORY_ID + random.nextInt(NUM_CATEGORIES);
+    long initialBid = nextPrice(random);
+    long dateTime = timestamp;
+    long expires = timestamp + nextAuctionLengthMs(random, timestamp);
+    String name = nextString(random, 20);
+    String desc = nextString(random, 100);
+    long reserve = initialBid + nextPrice(random);
+    int currentSize = 8 + name.length() + desc.length() + 8 + 8 + 8 + 8 + 8;
+    String extra = nextExtra(random, currentSize, config.configuration.avgAuctionByteSize);
+    return new Auction(id, name, desc, initialBid, reserve, dateTime, expires, seller, category,
+        extra);
+  }
+
+  /**
+   * Generate and return a random bid with next available id.
+   */
+  private Bid nextBid(Random random, long timestamp) {
+    long auction;
+    // Here P(bid will be for a hot auction) = 1 - 1/hotAuctionRatio.
+    if (random.nextInt(config.configuration.hotAuctionRatio) > 0) {
+      // Choose the first auction in the batch of last HOT_AUCTION_RATIO auctions.
+      auction = (lastBase0AuctionId() / HOT_AUCTION_RATIO) * HOT_AUCTION_RATIO;
+    } else {
+      auction = nextBase0AuctionId(random);
+    }
+    auction += GeneratorConfig.FIRST_AUCTION_ID;
+
+    long bidder;
+    // Here P(bid will be by a hot bidder) = 1 - 1/hotBiddersRatio
+    if (random.nextInt(config.configuration.hotBiddersRatio) > 0) {
+      // Choose the second person (so hot bidders and hot sellers don't collide) in the batch of
+      // last HOT_BIDDER_RATIO people.
+      bidder = (lastBase0PersonId() / HOT_BIDDER_RATIO) * HOT_BIDDER_RATIO + 1;
+    } else {
+      bidder = nextBase0PersonId(random);
+    }
+    bidder += GeneratorConfig.FIRST_PERSON_ID;
+
+    long price = nextPrice(random);
+    int currentSize = 8 + 8 + 8 + 8;
+    String extra = nextExtra(random, currentSize, config.configuration.avgBidByteSize);
+    return new Bid(auction, bidder, price, timestamp, extra);
+  }
+
+  @Override
+  public boolean hasNext() {
+    return numEvents < config.maxEvents;
+  }
+
+  /**
+   * Return the next event. The outer timestamp is in wallclock time and corresponds to
+   * when the event should fire. The inner timestamp is in event-time and represents the
+   * time the event is purported to have taken place in the simulation.
+   */
+  public NextEvent nextEvent() {
+    if (wallclockBaseTime < 0) {
+      wallclockBaseTime = System.currentTimeMillis();
+    }
+    // When, in event time, we should generate the event. Monotonic.
+    long eventTimestamp =
+        config.timestampAndInterEventDelayUsForEvent(config.nextEventNumber(numEvents)).getKey();
+    // When, in event time, the event should say it was generated. Depending on outOfOrderGroupSize
+    // may have local jitter.
+    long adjustedEventTimestamp =
+        config.timestampAndInterEventDelayUsForEvent(config.nextAdjustedEventNumber(numEvents))
+            .getKey();
+    // The minimum of this and all future adjusted event timestamps. Accounts for jitter in
+    // the event timestamp.
+    long watermark =
+        config.timestampAndInterEventDelayUsForEvent(config.nextEventNumberForWatermark(numEvents))
+            .getKey();
+    // When, in wallclock time, we should emit the event.
+    long wallclockTimestamp = wallclockBaseTime + (eventTimestamp - getCurrentConfig().baseTime);
+
+    // Seed the random number generator with the next 'event id'.
+    Random random = new Random(getNextEventId());
+    long rem = getNextEventId() % GeneratorConfig.PROPORTION_DENOMINATOR;
+
+    Event event;
+    if (rem < GeneratorConfig.PERSON_PROPORTION) {
+      event = new Event(nextPerson(random, adjustedEventTimestamp));
+    } else if (rem < GeneratorConfig.PERSON_PROPORTION + GeneratorConfig.AUCTION_PROPORTION) {
+      event = new Event(nextAuction(random, adjustedEventTimestamp));
+    } else {
+      event = new Event(nextBid(random, adjustedEventTimestamp));
+    }
+
+    numEvents++;
+    return new NextEvent(wallclockTimestamp, adjustedEventTimestamp, event, watermark);
+  }
+
+  @Override
+  public TimestampedValue<Event> next() {
+    NextEvent next = nextEvent();
+    return TimestampedValue.of(next.event, new Instant(next.eventTimestamp));
+  }
+
+  @Override
+  public void remove() {
+    throw new UnsupportedOperationException();
+  }
+
+  /**
+   * Return how many microseconds till we emit the next event.
+   */
+  public long currentInterEventDelayUs() {
+    return config.timestampAndInterEventDelayUsForEvent(config.nextEventNumber(numEvents))
+        .getValue();
+  }
+
+  /**
+   * Return an estimate of fraction of output consumed.
+   */
+  public double getFractionConsumed() {
+    return (double) numEvents / config.maxEvents;
+  }
+
+  @Override
+  public String toString() {
+    return String.format("Generator{config:%s; numEvents:%d; wallclockBaseTime:%d}", config,
+        numEvents, wallclockBaseTime);
+  }
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/1f08970a/integration/java/src/main/java/org/apache/beam/integration/nexmark/GeneratorConfig.java
----------------------------------------------------------------------
diff --git a/integration/java/src/main/java/org/apache/beam/integration/nexmark/GeneratorConfig.java b/integration/java/src/main/java/org/apache/beam/integration/nexmark/GeneratorConfig.java
new file mode 100644
index 0000000..59aaf49
--- /dev/null
+++ b/integration/java/src/main/java/org/apache/beam/integration/nexmark/GeneratorConfig.java
@@ -0,0 +1,295 @@
+/*
+ * 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.beam.integration.nexmark;
+
+import org.apache.beam.sdk.values.KV;
+
+import java.io.Serializable;
+import java.util.ArrayList;
+import java.util.List;
+
+/**
+ * Parameters controlling how {@link Generator} synthesizes {@link Event} elements.
+ */
+class GeneratorConfig implements Serializable {
+  /**
+   * We start the ids at specific values to help ensure the queries find a match even on
+   * small synthesized dataset sizes.
+   */
+  public static final long FIRST_AUCTION_ID = 1000L;
+  public static final long FIRST_PERSON_ID = 1000L;
+  public static final long FIRST_CATEGORY_ID = 10L;
+
+  /**
+   * Proportions of people/auctions/bids to synthesize.
+   */
+  public static final int PERSON_PROPORTION = 1;
+  public static final int AUCTION_PROPORTION = 3;
+  public static final int BID_PROPORTION = 46;
+  public static final int PROPORTION_DENOMINATOR =
+      PERSON_PROPORTION + AUCTION_PROPORTION + BID_PROPORTION;
+
+  /**
+   * Environment options.
+   */
+  public final NexmarkConfiguration configuration;
+
+  /**
+   * Delay between events, in microseconds. If the array has more than one entry then
+   * the rate is changed every {@link #stepLengthSec}, and wraps around.
+   */
+  public final long[] interEventDelayUs;
+
+  /**
+   * Delay before changing the current inter-event delay.
+   */
+  public final long stepLengthSec;
+
+  /**
+   * Time for first event (ms since epoch).
+   */
+  public final long baseTime;
+
+  /**
+   * Event id of first event to be generated. Event ids are unique over all generators, and
+   * are used as a seed to generate each event's data.
+   */
+  public final long firstEventId;
+
+  /**
+   * Maximum number of events to generate.
+   */
+  public final long maxEvents;
+
+  /**
+   * First event number. Generators running in parallel time may share the same event number,
+   * and the event number is used to determine the event timestamp.
+   */
+  public final long firstEventNumber;
+
+  /**
+   * True period of epoch in milliseconds. Derived from above.
+   * (Ie time to run through cycle for all interEventDelayUs entries).
+   */
+  public final long epochPeriodMs;
+
+  /**
+   * Number of events per epoch. Derived from above.
+   * (Ie number of events to run through cycle for all interEventDelayUs entries).
+   */
+  public final long eventsPerEpoch;
+
+  public GeneratorConfig(
+      NexmarkConfiguration configuration, long baseTime, long firstEventId,
+      long maxEventsOrZero, long firstEventNumber) {
+    this.configuration = configuration;
+    this.interEventDelayUs = configuration.rateShape.interEventDelayUs(
+        configuration.firstEventRate, configuration.nextEventRate,
+        configuration.rateUnit, configuration.numEventGenerators);
+    this.stepLengthSec = configuration.rateShape.stepLengthSec(configuration.ratePeriodSec);
+    this.baseTime = baseTime;
+    this.firstEventId = firstEventId;
+    if (maxEventsOrZero == 0) {
+      // Scale maximum down to avoid overflow in getEstimatedSizeBytes.
+      this.maxEvents =
+          Long.MAX_VALUE / (PROPORTION_DENOMINATOR
+                            * Math.max(
+              Math.max(configuration.avgPersonByteSize, configuration.avgAuctionByteSize),
+              configuration.avgBidByteSize));
+    } else {
+      this.maxEvents = maxEventsOrZero;
+    }
+    this.firstEventNumber = firstEventNumber;
+
+    long eventsPerEpoch = 0;
+    long epochPeriodMs = 0;
+    if (interEventDelayUs.length > 1) {
+      for (int i = 0; i < interEventDelayUs.length; i++) {
+        long numEventsForThisCycle = (stepLengthSec * 1_000_000L) / interEventDelayUs[i];
+        eventsPerEpoch += numEventsForThisCycle;
+        epochPeriodMs += (numEventsForThisCycle * interEventDelayUs[i]) / 1000L;
+      }
+    }
+    this.eventsPerEpoch = eventsPerEpoch;
+    this.epochPeriodMs = epochPeriodMs;
+  }
+
+  /**
+   * Return a clone of this config.
+   */
+  @Override
+  public GeneratorConfig clone() {
+    return new GeneratorConfig(configuration, baseTime, firstEventId, maxEvents, firstEventNumber);
+  }
+
+  /**
+   * Return clone of this config except with given parameters.
+   */
+  public GeneratorConfig cloneWith(long firstEventId, long maxEvents, long firstEventNumber) {
+    return new GeneratorConfig(configuration, baseTime, firstEventId, maxEvents, firstEventNumber);
+  }
+
+  /**
+   * Split this config into {@code n} sub-configs with roughly equal number of
+   * possible events, but distinct value spaces. The generators will run on parallel timelines.
+   * This config should no longer be used.
+   */
+  public List<GeneratorConfig> split(int n) {
+    List<GeneratorConfig> results = new ArrayList<>();
+    if (n == 1) {
+      // No split required.
+      results.add(this);
+    } else {
+      long subMaxEvents = maxEvents / n;
+      long subFirstEventId = firstEventId;
+      for (int i = 0; i < n; i++) {
+        if (i == n - 1) {
+          // Don't loose any events to round-down.
+          subMaxEvents = maxEvents - subMaxEvents * (n - 1);
+        }
+        results.add(cloneWith(subFirstEventId, subMaxEvents, firstEventNumber));
+        subFirstEventId += subMaxEvents;
+      }
+    }
+    return results;
+  }
+
+  /**
+   * Return an estimate of the bytes needed by {@code numEvents}.
+   */
+  public long estimatedBytesForEvents(long numEvents) {
+    long numPersons =
+        (numEvents * GeneratorConfig.PERSON_PROPORTION) / GeneratorConfig.PROPORTION_DENOMINATOR;
+    long numAuctions = (numEvents * AUCTION_PROPORTION) / PROPORTION_DENOMINATOR;
+    long numBids = (numEvents * BID_PROPORTION) / PROPORTION_DENOMINATOR;
+    return numPersons * configuration.avgPersonByteSize
+           + numAuctions * configuration.avgAuctionByteSize
+           + numBids * configuration.avgBidByteSize;
+  }
+
+  /**
+   * Return an estimate of the byte-size of all events a generator for this config would yield.
+   */
+  public long getEstimatedSizeBytes() {
+    return estimatedBytesForEvents(maxEvents);
+  }
+
+  /**
+   * Return the first 'event id' which could be generated from this config. Though events don't
+   * have ids we can simulate them to help bookkeeping.
+   */
+  public long getStartEventId() {
+    return firstEventId + firstEventNumber;
+  }
+
+  /**
+   * Return one past the last 'event id' which could be generated from this config.
+   */
+  public long getStopEventId() {
+    return firstEventId + firstEventNumber + maxEvents;
+  }
+
+  /**
+   * Return the next event number for a generator which has so far emitted {@code numEvents}.
+   */
+  public long nextEventNumber(long numEvents) {
+    return firstEventNumber + numEvents;
+  }
+
+  /**
+   * Return the next event number for a generator which has so far emitted {@code numEvents},
+   * but adjusted to account for {@code outOfOrderGroupSize}.
+   */
+  public long nextAdjustedEventNumber(long numEvents) {
+    long n = configuration.outOfOrderGroupSize;
+    long eventNumber = nextEventNumber(numEvents);
+    long base = (eventNumber / n) * n;
+    long offset = (eventNumber * 953) % n;
+    return base + offset;
+  }
+
+  /**
+   * Return the event number who's event time will be a suitable watermark for
+   * a generator which has so far emitted {@code numEvents}.
+   */
+  public long nextEventNumberForWatermark(long numEvents) {
+    long n = configuration.outOfOrderGroupSize;
+    long eventNumber = nextEventNumber(numEvents);
+    return (eventNumber / n) * n;
+  }
+
+  /**
+   * What timestamp should the event with {@code eventNumber} have for this generator? And
+   * what inter-event delay (in microseconds) is current?
+   */
+  public KV<Long, Long> timestampAndInterEventDelayUsForEvent(long eventNumber) {
+    if (interEventDelayUs.length == 1) {
+      long timestamp = baseTime + (eventNumber * interEventDelayUs[0]) / 1000L;
+      return KV.of(timestamp, interEventDelayUs[0]);
+    }
+
+    long epoch = eventNumber / eventsPerEpoch;
+    long n = eventNumber % eventsPerEpoch;
+    long offsetInEpochMs = 0;
+    for (int i = 0; i < interEventDelayUs.length; i++) {
+      long numEventsForThisCycle = (stepLengthSec * 1_000_000L) / interEventDelayUs[i];
+      if (n < numEventsForThisCycle) {
+        long offsetInCycleUs = n * interEventDelayUs[i];
+        long timestamp =
+            baseTime + epoch * epochPeriodMs + offsetInEpochMs + (offsetInCycleUs / 1000L);
+        return KV.of(timestamp, interEventDelayUs[i]);
+      }
+      n -= numEventsForThisCycle;
+      offsetInEpochMs += (numEventsForThisCycle * interEventDelayUs[i]) / 1000L;
+    }
+    throw new RuntimeException("internal eventsPerEpoch incorrect"); // can't reach
+  }
+
+  @Override
+  public String toString() {
+    StringBuilder sb = new StringBuilder();
+    sb.append("GeneratorConfig");
+    sb.append("{configuration:");
+    sb.append(configuration.toString());
+    sb.append(";interEventDelayUs=[");
+    for (int i = 0; i < interEventDelayUs.length; i++) {
+      if (i > 0) {
+        sb.append(",");
+      }
+      sb.append(interEventDelayUs[i]);
+    }
+    sb.append("]");
+    sb.append(";stepLengthSec:");
+    sb.append(stepLengthSec);
+    sb.append(";baseTime:");
+    sb.append(baseTime);
+    sb.append(";firstEventId:");
+    sb.append(firstEventId);
+    sb.append(";maxEvents:");
+    sb.append(maxEvents);
+    sb.append(";firstEventNumber:");
+    sb.append(firstEventNumber);
+    sb.append(";epochPeriodMs:");
+    sb.append(epochPeriodMs);
+    sb.append(";eventsPerEpoch:");
+    sb.append(eventsPerEpoch);
+    sb.append("}");
+    return sb.toString();
+  }
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/1f08970a/integration/java/src/main/java/org/apache/beam/integration/nexmark/IdNameReserve.java
----------------------------------------------------------------------
diff --git a/integration/java/src/main/java/org/apache/beam/integration/nexmark/IdNameReserve.java b/integration/java/src/main/java/org/apache/beam/integration/nexmark/IdNameReserve.java
new file mode 100644
index 0000000..c72b76a
--- /dev/null
+++ b/integration/java/src/main/java/org/apache/beam/integration/nexmark/IdNameReserve.java
@@ -0,0 +1,100 @@
+/*
+ * 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.beam.integration.nexmark;
+
+import org.apache.beam.sdk.coders.AtomicCoder;
+import org.apache.beam.sdk.coders.Coder;
+import org.apache.beam.sdk.coders.CoderException;
+import org.apache.beam.sdk.coders.StringUtf8Coder;
+import org.apache.beam.sdk.coders.VarLongCoder;
+
+import com.fasterxml.jackson.annotation.JsonProperty;
+import com.fasterxml.jackson.core.JsonProcessingException;
+
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
+import java.io.Serializable;
+
+/**
+ * Result type of {@link Query8}.
+ */
+public class IdNameReserve implements KnownSize, Serializable {
+  private static final Coder<Long> LONG_CODER = VarLongCoder.of();
+  private static final Coder<String> STRING_CODER = StringUtf8Coder.of();
+
+  public static final Coder<IdNameReserve> CODER = new AtomicCoder<IdNameReserve>() {
+    @Override
+    public void encode(IdNameReserve value, OutputStream outStream,
+        Coder.Context context)
+        throws CoderException, IOException {
+      LONG_CODER.encode(value.id, outStream, Context.NESTED);
+      STRING_CODER.encode(value.name, outStream, Context.NESTED);
+      LONG_CODER.encode(value.reserve, outStream, Context.NESTED);
+    }
+
+    @Override
+    public IdNameReserve decode(
+        InputStream inStream, Coder.Context context)
+        throws CoderException, IOException {
+      long id = LONG_CODER.decode(inStream, Context.NESTED);
+      String name = STRING_CODER.decode(inStream, Context.NESTED);
+      long reserve = LONG_CODER.decode(inStream, Context.NESTED);
+      return new IdNameReserve(id, name, reserve);
+    }
+  };
+
+  @JsonProperty
+  public final long id;
+
+  @JsonProperty
+  public final String name;
+
+  /** Reserve price in cents. */
+  @JsonProperty
+  public final long reserve;
+
+  // For Avro only.
+  @SuppressWarnings("unused")
+  private IdNameReserve() {
+    id = 0;
+    name = null;
+    reserve = 0;
+  }
+
+  public IdNameReserve(long id, String name, long reserve) {
+    this.id = id;
+    this.name = name;
+    this.reserve = reserve;
+  }
+
+  @Override
+  public long sizeInBytes() {
+    return 8 + name.length() + 1 + 8;
+  }
+
+  @Override
+  public String toString() {
+    try {
+      return NexmarkUtils.MAPPER.writeValueAsString(this);
+    } catch (JsonProcessingException e) {
+      throw new RuntimeException(e);
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/1f08970a/integration/java/src/main/java/org/apache/beam/integration/nexmark/KnownSize.java
----------------------------------------------------------------------
diff --git a/integration/java/src/main/java/org/apache/beam/integration/nexmark/KnownSize.java b/integration/java/src/main/java/org/apache/beam/integration/nexmark/KnownSize.java
new file mode 100644
index 0000000..394b6db
--- /dev/null
+++ b/integration/java/src/main/java/org/apache/beam/integration/nexmark/KnownSize.java
@@ -0,0 +1,27 @@
+/*
+ * 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.beam.integration.nexmark;
+
+/**
+ * Interface for elements which can quickly estimate their encoded byte size.
+ */
+public interface KnownSize {
+  long sizeInBytes();
+}
+

http://git-wip-us.apache.org/repos/asf/beam/blob/1f08970a/integration/java/src/main/java/org/apache/beam/integration/nexmark/Monitor.java
----------------------------------------------------------------------
diff --git a/integration/java/src/main/java/org/apache/beam/integration/nexmark/Monitor.java b/integration/java/src/main/java/org/apache/beam/integration/nexmark/Monitor.java
new file mode 100644
index 0000000..6874578
--- /dev/null
+++ b/integration/java/src/main/java/org/apache/beam/integration/nexmark/Monitor.java
@@ -0,0 +1,102 @@
+/*
+ * 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.beam.integration.nexmark;
+
+import org.apache.beam.sdk.transforms.Aggregator;
+import org.apache.beam.sdk.transforms.DoFn;
+import org.apache.beam.sdk.transforms.Max.MaxLongFn;
+import org.apache.beam.sdk.transforms.Min.MinLongFn;
+import org.apache.beam.sdk.transforms.PTransform;
+import org.apache.beam.sdk.transforms.ParDo;
+import org.apache.beam.sdk.transforms.Sum.SumLongFn;
+import org.apache.beam.sdk.values.PCollection;
+
+import java.io.Serializable;
+
+/**
+ * A monitor of elements with support for later retrieving their aggregators.
+ *
+ * @param <T> Type of element we are monitoring.
+ */
+public class Monitor<T extends KnownSize> implements Serializable {
+  private class MonitorDoFn extends DoFn<T, T> {
+    public final Aggregator<Long, Long> elementCounter =
+        createAggregator(counterNamePrefix + "_elements", new SumLongFn());
+    public final Aggregator<Long, Long> bytesCounter =
+        createAggregator(counterNamePrefix + "_bytes", new SumLongFn());
+    public final Aggregator<Long, Long> startTime =
+        createAggregator(counterNamePrefix + "_startTime", new MinLongFn());
+    public final Aggregator<Long, Long> endTime =
+        createAggregator(counterNamePrefix + "_endTime", new MaxLongFn());
+    public final Aggregator<Long, Long> startTimestamp =
+        createAggregator("startTimestamp", new MinLongFn());
+    public final Aggregator<Long, Long> endTimestamp =
+        createAggregator("endTimestamp", new MaxLongFn());
+
+    @Override
+    public void processElement(ProcessContext c) {
+      elementCounter.addValue(1L);
+      bytesCounter.addValue(c.element().sizeInBytes());
+      long now = System.currentTimeMillis();
+      startTime.addValue(now);
+      endTime.addValue(now);
+      startTimestamp.addValue(c.timestamp().getMillis());
+      endTimestamp.addValue(c.timestamp().getMillis());
+      c.output(c.element());
+    }
+  }
+
+  final MonitorDoFn doFn;
+  final PTransform<PCollection<? extends T>, PCollection<T>> transform;
+  private String counterNamePrefix;
+
+  public Monitor(String name, String counterNamePrefix) {
+    this.counterNamePrefix = counterNamePrefix;
+    doFn = new MonitorDoFn();
+    transform = ParDo.named(name + ".Monitor").of(doFn);
+  }
+
+  public PTransform<PCollection<? extends T>, PCollection<T>> getTransform() {
+    return transform;
+  }
+
+  public Aggregator<Long, Long> getElementCounter() {
+    return doFn.elementCounter;
+  }
+
+  public Aggregator<Long, Long> getBytesCounter() {
+    return doFn.bytesCounter;
+  }
+
+  public Aggregator<Long, Long> getStartTime() {
+    return doFn.startTime;
+  }
+
+  public Aggregator<Long, Long> getEndTime() {
+    return doFn.endTime;
+  }
+
+  public Aggregator<Long, Long> getStartTimestamp() {
+    return doFn.startTimestamp;
+  }
+
+  public Aggregator<Long, Long> getEndTimestamp() {
+    return doFn.endTimestamp;
+  }
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/1f08970a/integration/java/src/main/java/org/apache/beam/integration/nexmark/NameCityStateId.java
----------------------------------------------------------------------
diff --git a/integration/java/src/main/java/org/apache/beam/integration/nexmark/NameCityStateId.java b/integration/java/src/main/java/org/apache/beam/integration/nexmark/NameCityStateId.java
new file mode 100644
index 0000000..2753d2e
--- /dev/null
+++ b/integration/java/src/main/java/org/apache/beam/integration/nexmark/NameCityStateId.java
@@ -0,0 +1,106 @@
+/*
+ * 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.beam.integration.nexmark;
+
+import org.apache.beam.sdk.coders.AtomicCoder;
+import org.apache.beam.sdk.coders.Coder;
+import org.apache.beam.sdk.coders.CoderException;
+import org.apache.beam.sdk.coders.StringUtf8Coder;
+import org.apache.beam.sdk.coders.VarLongCoder;
+
+import com.fasterxml.jackson.annotation.JsonProperty;
+import com.fasterxml.jackson.core.JsonProcessingException;
+
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
+import java.io.Serializable;
+
+/**
+ * Result of {@link Query3}.
+ */
+public class NameCityStateId implements KnownSize, Serializable {
+  private static final Coder<Long> LONG_CODER = VarLongCoder.of();
+  private static final Coder<String> STRING_CODER = StringUtf8Coder.of();
+
+  public static final Coder<NameCityStateId> CODER = new AtomicCoder<NameCityStateId>() {
+    @Override
+    public void encode(NameCityStateId value, OutputStream outStream,
+        Coder.Context context)
+        throws CoderException, IOException {
+      STRING_CODER.encode(value.name, outStream, Context.NESTED);
+      STRING_CODER.encode(value.city, outStream, Context.NESTED);
+      STRING_CODER.encode(value.state, outStream, Context.NESTED);
+      LONG_CODER.encode(value.id, outStream, Context.NESTED);
+    }
+
+    @Override
+    public NameCityStateId decode(
+        InputStream inStream, Coder.Context context)
+        throws CoderException, IOException {
+      String name = STRING_CODER.decode(inStream, Context.NESTED);
+      String city = STRING_CODER.decode(inStream, Context.NESTED);
+      String state = STRING_CODER.decode(inStream, Context.NESTED);
+      long id = LONG_CODER.decode(inStream, Context.NESTED);
+      return new NameCityStateId(name, city, state, id);
+    }
+  };
+
+  @JsonProperty
+  public final String name;
+
+  @JsonProperty
+  public final String city;
+
+  @JsonProperty
+  public final String state;
+
+  @JsonProperty
+  public final long id;
+
+  // For Avro only.
+  @SuppressWarnings("unused")
+  private NameCityStateId() {
+    name = null;
+    city = null;
+    state = null;
+    id = 0;
+  }
+
+  public NameCityStateId(String name, String city, String state, long id) {
+    this.name = name;
+    this.city = city;
+    this.state = state;
+    this.id = id;
+  }
+
+  @Override
+  public long sizeInBytes() {
+    return name.length() + 1 + city.length() + 1 + state.length() + 1 + 8;
+  }
+
+  @Override
+  public String toString() {
+    try {
+      return NexmarkUtils.MAPPER.writeValueAsString(this);
+    } catch (JsonProcessingException e) {
+      throw new RuntimeException(e);
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/1f08970a/integration/java/src/main/java/org/apache/beam/integration/nexmark/NexmarkConfiguration.java
----------------------------------------------------------------------
diff --git a/integration/java/src/main/java/org/apache/beam/integration/nexmark/NexmarkConfiguration.java b/integration/java/src/main/java/org/apache/beam/integration/nexmark/NexmarkConfiguration.java
new file mode 100644
index 0000000..2292ba5
--- /dev/null
+++ b/integration/java/src/main/java/org/apache/beam/integration/nexmark/NexmarkConfiguration.java
@@ -0,0 +1,662 @@
+/*
+ * 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.beam.integration.nexmark;
+
+import com.fasterxml.jackson.annotation.JsonProperty;
+import com.fasterxml.jackson.core.JsonProcessingException;
+
+import java.io.IOException;
+import java.io.Serializable;
+import java.util.Objects;
+
+
+/**
+ * Configuration controlling how a query is run. May be supplied by command line or
+ * programmatically. We only capture properties which may influence the resulting
+ * pipeline performance, as captured by {@link NexmarkPerf}.
+ */
+class NexmarkConfiguration implements Serializable {
+  public static final NexmarkConfiguration DEFAULT = new NexmarkConfiguration();
+
+  /** If {@literal true}, include additional debugging and monitoring stats. */
+  @JsonProperty
+  public boolean debug = true;
+
+  /** Which query to run, in [0,9]. */
+  @JsonProperty
+  public int query = 0;
+
+  /** Where events come from. */
+  @JsonProperty
+  public NexmarkUtils.SourceType sourceType = NexmarkUtils.SourceType.DIRECT;
+
+  /** Where results go to. */
+  @JsonProperty
+  public NexmarkUtils.SinkType sinkType = NexmarkUtils.SinkType.DEVNULL;
+
+  /**
+   * Control whether pub/sub publishing is done in a stand-alone pipeline or is integrated
+   * into the overall query pipeline.
+   */
+  @JsonProperty
+  public NexmarkUtils.PubSubMode pubSubMode = NexmarkUtils.PubSubMode.COMBINED;
+
+  /**
+   * Number of events to generate. If zero, generate as many as possible without overflowing
+   * internal counters etc.
+   */
+  @JsonProperty
+  public long numEvents = 100000;
+
+  /**
+   * Number of event generators to use. Each generates events in its own timeline.
+   */
+  @JsonProperty
+  public int numEventGenerators = 100;
+
+  /**
+   * Shape of event rate curve.
+   */
+  @JsonProperty
+  public NexmarkUtils.RateShape rateShape = NexmarkUtils.RateShape.SINE;
+
+  /**
+   * Initial overall event rate (in {@link #rateUnit}).
+   */
+  @JsonProperty
+  public int firstEventRate = 10000;
+
+  /**
+   * Next overall event rate (in {@link #rateUnit}).
+   */
+  @JsonProperty
+  public int nextEventRate = 10000;
+
+  /**
+   * Unit for rates.
+   */
+  @JsonProperty
+  public NexmarkUtils.RateUnit rateUnit = NexmarkUtils.RateUnit.PER_SECOND;
+
+  /**
+   * Overall period of rate shape, in seconds.
+   */
+  @JsonProperty
+  public int ratePeriodSec = 600;
+
+  /**
+   * Time in seconds to preload the subscription with data, at the initial input rate of the
+   * pipeline.
+   */
+  @JsonProperty
+  public int preloadSeconds = 0;
+
+  /**
+   * If true, and in streaming mode, generate events only when they are due according to their
+   * timestamp.
+   */
+  @JsonProperty
+  public boolean isRateLimited = false;
+
+  /**
+   * If true, use wallclock time as event time. Otherwise, use a deterministic
+   * time in the past so that multiple runs will see exactly the same event streams
+   * and should thus have exactly the same results.
+   */
+  @JsonProperty
+  public boolean useWallclockEventTime = false;
+
+  /** Average idealized size of a 'new person' event, in bytes. */
+  @JsonProperty
+  public int avgPersonByteSize = 200;
+
+  /** Average idealized size of a 'new auction' event, in bytes. */
+  @JsonProperty
+  public int avgAuctionByteSize = 500;
+
+  /** Average idealized size of a 'bid' event, in bytes. */
+  @JsonProperty
+  public int avgBidByteSize = 100;
+
+  /** Ratio of bids to 'hot' auctions compared to all other auctions. */
+  @JsonProperty
+  public int hotAuctionRatio = 1;
+
+  /** Ratio of auctions for 'hot' sellers compared to all other people. */
+  @JsonProperty
+  public int hotSellersRatio = 1;
+
+  /** Ratio of bids for 'hot' bidders compared to all other people. */
+  @JsonProperty
+  public int hotBiddersRatio = 1;
+
+  /** Window size, in seconds, for queries 3, 5, 7 and 8. */
+  @JsonProperty
+  public long windowSizeSec = 10;
+
+  /** Sliding window period, in seconds, for query 5. */
+  @JsonProperty
+  public long windowPeriodSec = 5;
+
+  /** Number of seconds to hold back events according to their reported timestamp. */
+  @JsonProperty
+  public long watermarkHoldbackSec = 0;
+
+  /** Average number of auction which should be inflight at any time, per generator. */
+  @JsonProperty
+  public int numInFlightAuctions = 100;
+
+  /** Maximum number of people to consider as active for placing auctions or bids. */
+  @JsonProperty
+  public int numActivePeople = 1000;
+
+  /** Coder strategy to follow. */
+  @JsonProperty
+  public NexmarkUtils.CoderStrategy coderStrategy = NexmarkUtils.CoderStrategy.HAND;
+
+  /**
+   * Delay, in milliseconds, for each event. This will peg one core for this number
+   * of milliseconds to simulate CPU-bound computation.
+   */
+  @JsonProperty
+  public long cpuDelayMs = 0;
+
+  /**
+   * Extra data, in bytes, to save to persistent state for each event. This will force
+   * i/o all the way to durable storage to simulate an I/O-bound computation.
+   */
+  @JsonProperty
+  public long diskBusyBytes = 0;
+
+  /**
+   * Skip factor for query 2. We select bids for every {@code auctionSkip}'th auction.
+   */
+  @JsonProperty
+  public int auctionSkip = 123;
+
+  /**
+   * Fanout for queries 4 (groups by category id), 5 and 7 (find a global maximum).
+   */
+  @JsonProperty
+  public int fanout = 5;
+
+  /**
+   * Length of occasional delay to impose on events (in seconds).
+   */
+  @JsonProperty
+  public long occasionalDelaySec = 0;
+
+  /**
+   * Probability that an event will be delayed by delayS.
+   */
+  @JsonProperty
+  public double probDelayedEvent = 0.0;
+
+  /**
+   * Maximum size of each log file (in events). For Query10 only.
+   */
+  @JsonProperty
+  public int maxLogEvents = 100_000;
+
+  /**
+   * If true, use pub/sub publish time instead of event time.
+   */
+  @JsonProperty
+  public boolean usePubsubPublishTime = false;
+
+  /**
+   * Number of events in out-of-order groups. 1 implies no out-of-order events. 1000 implies
+   * every 1000 events per generator are emitted in pseudo-random order.
+   */
+  @JsonProperty
+  public long outOfOrderGroupSize = 1;
+
+  /**
+   * Replace any properties of this configuration which have been supplied by the command line.
+   */
+  public void overrideFromOptions(Options options) {
+    if (options.getDebug() != null) {
+      debug = options.getDebug();
+    }
+    if (options.getQuery() != null) {
+      query = options.getQuery();
+    }
+    if (options.getSourceType() != null) {
+      sourceType = options.getSourceType();
+    }
+    if (options.getSinkType() != null) {
+      sinkType = options.getSinkType();
+    }
+    if (options.getPubSubMode() != null) {
+      pubSubMode = options.getPubSubMode();
+    }
+    if (options.getNumEvents() != null) {
+      numEvents = options.getNumEvents();
+    }
+    if (options.getNumEventGenerators() != null) {
+      numEventGenerators = options.getNumEventGenerators();
+    }
+    if (options.getRateShape() != null) {
+      rateShape = options.getRateShape();
+    }
+    if (options.getFirstEventRate() != null) {
+      firstEventRate = options.getFirstEventRate();
+    }
+    if (options.getNextEventRate() != null) {
+      nextEventRate = options.getNextEventRate();
+    }
+    if (options.getRateUnit() != null) {
+      rateUnit = options.getRateUnit();
+    }
+    if (options.getRatePeriodSec() != null) {
+      ratePeriodSec = options.getRatePeriodSec();
+    }
+    if (options.getPreloadSeconds() != null) {
+      preloadSeconds = options.getPreloadSeconds();
+    }
+    if (options.getIsRateLimited() != null) {
+      isRateLimited = options.getIsRateLimited();
+    }
+    if (options.getUseWallclockEventTime() != null) {
+      useWallclockEventTime = options.getUseWallclockEventTime();
+    }
+    if (options.getAvgPersonByteSize() != null) {
+      avgPersonByteSize = options.getAvgPersonByteSize();
+    }
+    if (options.getAvgAuctionByteSize() != null) {
+      avgAuctionByteSize = options.getAvgAuctionByteSize();
+    }
+    if (options.getAvgBidByteSize() != null) {
+      avgBidByteSize = options.getAvgBidByteSize();
+    }
+    if (options.getHotAuctionRatio() != null) {
+      hotAuctionRatio = options.getHotAuctionRatio();
+    }
+    if (options.getHotSellersRatio() != null) {
+      hotSellersRatio = options.getHotSellersRatio();
+    }
+    if (options.getHotBiddersRatio() != null) {
+      hotBiddersRatio = options.getHotBiddersRatio();
+    }
+    if (options.getWindowSizeSec() != null) {
+      windowSizeSec = options.getWindowSizeSec();
+    }
+    if (options.getWindowPeriodSec() != null) {
+      windowPeriodSec = options.getWindowPeriodSec();
+    }
+    if (options.getWatermarkHoldbackSec() != null) {
+      watermarkHoldbackSec = options.getWatermarkHoldbackSec();
+    }
+    if (options.getNumInFlightAuctions() != null) {
+      numInFlightAuctions = options.getNumInFlightAuctions();
+    }
+    if (options.getNumActivePeople() != null) {
+      numActivePeople = options.getNumActivePeople();
+    }
+    if (options.getCoderStrategy() != null) {
+      coderStrategy = options.getCoderStrategy();
+    }
+    if (options.getCpuDelayMs() != null) {
+      cpuDelayMs = options.getCpuDelayMs();
+    }
+    if (options.getDiskBusyBytes() != null) {
+      diskBusyBytes = options.getDiskBusyBytes();
+    }
+    if (options.getAuctionSkip() != null) {
+      auctionSkip = options.getAuctionSkip();
+    }
+    if (options.getFanout() != null) {
+      fanout = options.getFanout();
+    }
+    if (options.getOccasionalDelaySec() != null) {
+      occasionalDelaySec = options.getOccasionalDelaySec();
+    }
+    if (options.getProbDelayedEvent() != null) {
+      probDelayedEvent = options.getProbDelayedEvent();
+    }
+    if (options.getMaxLogEvents() != null) {
+      maxLogEvents = options.getMaxLogEvents();
+    }
+    if (options.getUsePubsubPublishTime() != null) {
+      usePubsubPublishTime = options.getUsePubsubPublishTime();
+    }
+    if (options.getOutOfOrderGroupSize() != null) {
+      outOfOrderGroupSize = options.getOutOfOrderGroupSize();
+    }
+  }
+
+  /**
+   * Return clone of configuration with given label.
+   */
+  @Override
+  public NexmarkConfiguration clone() {
+    NexmarkConfiguration result = new NexmarkConfiguration();
+    result.debug = debug;
+    result.query = query;
+    result.sourceType = sourceType;
+    result.sinkType = sinkType;
+    result.pubSubMode = pubSubMode;
+    result.numEvents = numEvents;
+    result.numEventGenerators = numEventGenerators;
+    result.rateShape = rateShape;
+    result.firstEventRate = firstEventRate;
+    result.nextEventRate = nextEventRate;
+    result.rateUnit = rateUnit;
+    result.ratePeriodSec = ratePeriodSec;
+    result.preloadSeconds = preloadSeconds;
+    result.isRateLimited = isRateLimited;
+    result.useWallclockEventTime = useWallclockEventTime;
+    result.avgPersonByteSize = avgPersonByteSize;
+    result.avgAuctionByteSize = avgAuctionByteSize;
+    result.avgBidByteSize = avgBidByteSize;
+    result.hotAuctionRatio = hotAuctionRatio;
+    result.hotSellersRatio = hotSellersRatio;
+    result.hotBiddersRatio = hotBiddersRatio;
+    result.windowSizeSec = windowSizeSec;
+    result.windowPeriodSec = windowPeriodSec;
+    result.watermarkHoldbackSec = watermarkHoldbackSec;
+    result.numInFlightAuctions = numInFlightAuctions;
+    result.numActivePeople = numActivePeople;
+    result.coderStrategy = coderStrategy;
+    result.cpuDelayMs = cpuDelayMs;
+    result.diskBusyBytes = diskBusyBytes;
+    result.auctionSkip = auctionSkip;
+    result.fanout = fanout;
+    result.occasionalDelaySec = occasionalDelaySec;
+    result.probDelayedEvent = probDelayedEvent;
+    result.maxLogEvents = maxLogEvents;
+    result.usePubsubPublishTime = usePubsubPublishTime;
+    result.outOfOrderGroupSize = outOfOrderGroupSize;
+    return result;
+  }
+
+  /**
+   * Return short description of configuration (suitable for use in logging). We only render
+   * the core fields plus those which do not have default values.
+   */
+  public String toShortString() {
+    StringBuilder sb = new StringBuilder();
+    sb.append(String.format("query:%d", query));
+    if (debug != DEFAULT.debug) {
+      sb.append(String.format("; debug:%s", debug));
+    }
+    if (sourceType != DEFAULT.sourceType) {
+      sb.append(String.format("; sourceType:%s", sourceType));
+    }
+    if (sinkType != DEFAULT.sinkType) {
+      sb.append(String.format("; sinkType:%s", sinkType));
+    }
+    if (pubSubMode != DEFAULT.pubSubMode) {
+      sb.append(String.format("; pubSubMode:%s", pubSubMode));
+    }
+    if (numEvents != DEFAULT.numEvents) {
+      sb.append(String.format("; numEvents:%d", numEvents));
+    }
+    if (numEventGenerators != DEFAULT.numEventGenerators) {
+      sb.append(String.format("; numEventGenerators:%d", numEventGenerators));
+    }
+    if (rateShape != DEFAULT.rateShape) {
+      sb.append(String.format("; rateShape:%s", rateShape));
+    }
+    if (firstEventRate != DEFAULT.firstEventRate || nextEventRate != DEFAULT.nextEventRate) {
+      sb.append(String.format("; firstEventRate:%d", firstEventRate));
+      sb.append(String.format("; nextEventRate:%d", nextEventRate));
+    }
+    if (rateUnit != DEFAULT.rateUnit) {
+      sb.append(String.format("; rateUnit:%s", rateUnit));
+    }
+    if (ratePeriodSec != DEFAULT.ratePeriodSec) {
+      sb.append(String.format("; ratePeriodSec:%d", ratePeriodSec));
+    }
+    if (preloadSeconds != DEFAULT.preloadSeconds) {
+      sb.append(String.format("; preloadSeconds:%d", preloadSeconds));
+    }
+    if (isRateLimited != DEFAULT.isRateLimited) {
+      sb.append(String.format("; isRateLimited:%s", isRateLimited));
+    }
+    if (useWallclockEventTime != DEFAULT.useWallclockEventTime) {
+      sb.append(String.format("; useWallclockEventTime:%s", useWallclockEventTime));
+    }
+    if (avgPersonByteSize != DEFAULT.avgPersonByteSize) {
+      sb.append(String.format("; avgPersonByteSize:%d", avgPersonByteSize));
+    }
+    if (avgAuctionByteSize != DEFAULT.avgAuctionByteSize) {
+      sb.append(String.format("; avgAuctionByteSize:%d", avgAuctionByteSize));
+    }
+    if (avgBidByteSize != DEFAULT.avgBidByteSize) {
+      sb.append(String.format("; avgBidByteSize:%d", avgBidByteSize));
+    }
+    if (hotAuctionRatio != DEFAULT.hotAuctionRatio) {
+      sb.append(String.format("; hotAuctionRatio:%d", hotAuctionRatio));
+    }
+    if (hotSellersRatio != DEFAULT.hotSellersRatio) {
+      sb.append(String.format("; hotSellersRatio:%d", hotSellersRatio));
+    }
+    if (hotBiddersRatio != DEFAULT.hotBiddersRatio) {
+      sb.append(String.format("; hotBiddersRatio:%d", hotBiddersRatio));
+    }
+    if (windowSizeSec != DEFAULT.windowSizeSec) {
+      sb.append(String.format("; windowSizeSec:%d", windowSizeSec));
+    }
+    if (windowPeriodSec != DEFAULT.windowPeriodSec) {
+      sb.append(String.format("; windowPeriodSec:%d", windowPeriodSec));
+    }
+    if (watermarkHoldbackSec != DEFAULT.watermarkHoldbackSec) {
+      sb.append(String.format("; watermarkHoldbackSec:%d", watermarkHoldbackSec));
+    }
+    if (numInFlightAuctions != DEFAULT.numInFlightAuctions) {
+      sb.append(String.format("; numInFlightAuctions:%d", numInFlightAuctions));
+    }
+    if (numActivePeople != DEFAULT.numActivePeople) {
+      sb.append(String.format("; numActivePeople:%d", numActivePeople));
+    }
+    if (coderStrategy != DEFAULT.coderStrategy) {
+      sb.append(String.format("; coderStrategy:%s", coderStrategy));
+    }
+    if (cpuDelayMs != DEFAULT.cpuDelayMs) {
+      sb.append(String.format("; cpuSlowdownMs:%d", cpuDelayMs));
+    }
+    if (diskBusyBytes != DEFAULT.diskBusyBytes) {
+      sb.append(String.format("; diskBuysBytes:%d", diskBusyBytes));
+    }
+    if (auctionSkip != DEFAULT.auctionSkip) {
+      sb.append(String.format("; auctionSkip:%d", auctionSkip));
+    }
+    if (fanout != DEFAULT.fanout) {
+      sb.append(String.format("; fanout:%d", fanout));
+    }
+    if (occasionalDelaySec != DEFAULT.occasionalDelaySec) {
+      sb.append(String.format("; occasionalDelaySec:%d", occasionalDelaySec));
+    }
+    if (probDelayedEvent != DEFAULT.probDelayedEvent) {
+      sb.append(String.format("; probDelayedEvent:%f", probDelayedEvent));
+    }
+    if (maxLogEvents != DEFAULT.maxLogEvents) {
+      sb.append(String.format("; maxLogEvents:%d", maxLogEvents));
+    }
+    if (usePubsubPublishTime != DEFAULT.usePubsubPublishTime) {
+      sb.append(String.format("; usePubsubPublishTime:%s", usePubsubPublishTime));
+    }
+    if (outOfOrderGroupSize != DEFAULT.outOfOrderGroupSize) {
+      sb.append(String.format("; outOfOrderGroupSize:%d", outOfOrderGroupSize));
+    }
+    return sb.toString();
+  }
+
+  /**
+   * Return full description as a string.
+   */
+  @Override
+  public String toString() {
+    try {
+      return NexmarkUtils.MAPPER.writeValueAsString(this);
+    } catch (JsonProcessingException e) {
+      throw new RuntimeException(e);
+    }
+  }
+
+  /**
+   * Parse an object from {@code string}.
+   *
+   * @throws IOException
+   */
+  public static NexmarkConfiguration fromString(String string) {
+    try {
+      return NexmarkUtils.MAPPER.readValue(string, NexmarkConfiguration.class);
+    } catch (IOException e) {
+      throw new RuntimeException("Unable to parse nexmark configuration: ", e);
+    }
+  }
+
+  @Override
+  public int hashCode() {
+    return Objects.hash(debug, query, sourceType, sinkType, pubSubMode,
+        numEvents, numEventGenerators, rateShape, firstEventRate, nextEventRate, rateUnit,
+        ratePeriodSec, preloadSeconds, isRateLimited, useWallclockEventTime, avgPersonByteSize,
+        avgAuctionByteSize, avgBidByteSize, hotAuctionRatio, hotSellersRatio, hotBiddersRatio,
+        windowSizeSec, windowPeriodSec, watermarkHoldbackSec, numInFlightAuctions, numActivePeople,
+        coderStrategy, cpuDelayMs, diskBusyBytes, auctionSkip, fanout,
+        occasionalDelaySec, probDelayedEvent, maxLogEvents, usePubsubPublishTime,
+        outOfOrderGroupSize);
+  }
+
+  @Override
+  public boolean equals(Object obj) {
+    if (this == obj) {
+      return true;
+    }
+    if (obj == null) {
+      return false;
+    }
+    if (getClass() != obj.getClass()) {
+      return false;
+    }
+    NexmarkConfiguration other = (NexmarkConfiguration) obj;
+    if (debug != other.debug) {
+      return false;
+    }
+    if (auctionSkip != other.auctionSkip) {
+      return false;
+    }
+    if (avgAuctionByteSize != other.avgAuctionByteSize) {
+      return false;
+    }
+    if (avgBidByteSize != other.avgBidByteSize) {
+      return false;
+    }
+    if (avgPersonByteSize != other.avgPersonByteSize) {
+      return false;
+    }
+    if (coderStrategy != other.coderStrategy) {
+      return false;
+    }
+    if (cpuDelayMs != other.cpuDelayMs) {
+      return false;
+    }
+    if (diskBusyBytes != other.diskBusyBytes) {
+      return false;
+    }
+    if (fanout != other.fanout) {
+      return false;
+    }
+    if (firstEventRate != other.firstEventRate) {
+      return false;
+    }
+    if (hotAuctionRatio != other.hotAuctionRatio) {
+      return false;
+    }
+    if (hotBiddersRatio != other.hotBiddersRatio) {
+      return false;
+    }
+    if (hotSellersRatio != other.hotSellersRatio) {
+      return false;
+    }
+    if (isRateLimited != other.isRateLimited) {
+      return false;
+    }
+    if (maxLogEvents != other.maxLogEvents) {
+      return false;
+    }
+    if (nextEventRate != other.nextEventRate) {
+      return false;
+    }
+    if (rateUnit != other.rateUnit) {
+      return false;
+    }
+    if (numEventGenerators != other.numEventGenerators) {
+      return false;
+    }
+    if (numEvents != other.numEvents) {
+      return false;
+    }
+    if (numInFlightAuctions != other.numInFlightAuctions) {
+      return false;
+    }
+    if (numActivePeople != other.numActivePeople) {
+      return false;
+    }
+    if (occasionalDelaySec != other.occasionalDelaySec) {
+      return false;
+    }
+    if (preloadSeconds != other.preloadSeconds) {
+      return false;
+    }
+    if (Double.doubleToLongBits(probDelayedEvent)
+        != Double.doubleToLongBits(other.probDelayedEvent)) {
+      return false;
+    }
+    if (pubSubMode != other.pubSubMode) {
+      return false;
+    }
+    if (ratePeriodSec != other.ratePeriodSec) {
+      return false;
+    }
+    if (rateShape != other.rateShape) {
+      return false;
+    }
+    if (query != other.query) {
+      return false;
+    }
+    if (sinkType != other.sinkType) {
+      return false;
+    }
+    if (sourceType != other.sourceType) {
+      return false;
+    }
+    if (useWallclockEventTime != other.useWallclockEventTime) {
+      return false;
+    }
+    if (watermarkHoldbackSec != other.watermarkHoldbackSec) {
+      return false;
+    }
+    if (windowPeriodSec != other.windowPeriodSec) {
+      return false;
+    }
+    if (windowSizeSec != other.windowSizeSec) {
+      return false;
+    }
+    if (usePubsubPublishTime != other.usePubsubPublishTime) {
+      return false;
+    }
+    if (outOfOrderGroupSize != other.outOfOrderGroupSize) {
+      return false;
+    }
+    return true;
+  }
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/1f08970a/integration/java/src/main/java/org/apache/beam/integration/nexmark/NexmarkDriver.java
----------------------------------------------------------------------
diff --git a/integration/java/src/main/java/org/apache/beam/integration/nexmark/NexmarkDriver.java b/integration/java/src/main/java/org/apache/beam/integration/nexmark/NexmarkDriver.java
new file mode 100644
index 0000000..dbc1ce2
--- /dev/null
+++ b/integration/java/src/main/java/org/apache/beam/integration/nexmark/NexmarkDriver.java
@@ -0,0 +1,297 @@
+/*
+ * 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.beam.integration.nexmark;
+
+import org.joda.time.Duration;
+import org.joda.time.Instant;
+
+import java.io.IOException;
+import java.nio.charset.StandardCharsets;
+import java.nio.file.Files;
+import java.nio.file.Paths;
+import java.nio.file.StandardOpenOption;
+import java.util.ArrayList;
+import java.util.LinkedHashMap;
+import java.util.List;
+import java.util.Map;
+
+import javax.annotation.Nullable;
+
+/**
+ * An implementation of the 'NEXMark queries' for Google Dataflow.
+ * These are 11 queries over a three table schema representing on online auction system:
+ * <ul>
+ * <li>{@link Person} represents a person submitting an item for auction and/or making a bid
+ * on an auction.
+ * <li>{@link Auction} represents an item under auction.
+ * <li>{@link Bid} represents a bid for an item under auction.
+ * </ul>
+ * The queries exercise many aspects of streaming dataflow.
+ * <p>
+ * <p>We synthesize the creation of people, auctions and bids in real-time. The data is not
+ * particularly sensible.
+ * <p>
+ * <p>See
+ * <a href="http://datalab.cs.pdx.edu/niagaraST/NEXMark/">
+ * http://datalab.cs.pdx.edu/niagaraST/NEXMark/</a>
+ */
+public class NexmarkDriver<OptionT extends Options> {
+
+  /**
+   * Entry point.
+   */
+  public void runAll(OptionT options, NexmarkRunner runner) {
+    Instant start = Instant.now();
+    Map<NexmarkConfiguration, NexmarkPerf> baseline = loadBaseline(options.getBaselineFilename());
+    Map<NexmarkConfiguration, NexmarkPerf> actual = new LinkedHashMap<>();
+    Iterable<NexmarkConfiguration> configurations = options.getSuite().getConfigurations(options);
+
+    boolean successful = true;
+    try {
+      // Run all the configurations.
+      for (NexmarkConfiguration configuration : configurations) {
+        NexmarkPerf perf = runner.run(configuration);
+        if (perf != null) {
+          if (perf.errors == null || perf.errors.size() > 0) {
+            successful = false;
+          }
+          appendPerf(options.getPerfFilename(), configuration, perf);
+          actual.put(configuration, perf);
+          // Summarize what we've run so far.
+          saveSummary(null, configurations, actual, baseline, start);
+        }
+      }
+    } finally {
+      if (options.getMonitorJobs()) {
+        // Report overall performance.
+        saveSummary(options.getSummaryFilename(), configurations, actual, baseline, start);
+        saveJavascript(options.getJavascriptFilename(), configurations, actual, baseline, start);
+      }
+    }
+
+    if (!successful) {
+      System.exit(1);
+    }
+  }
+
+  /**
+   * Append the pair of {@code configuration} and {@code perf} to perf file.
+   */
+  private void appendPerf(
+      @Nullable String perfFilename, NexmarkConfiguration configuration,
+      NexmarkPerf perf) {
+    if (perfFilename == null) {
+      return;
+    }
+    List<String> lines = new ArrayList<>();
+    lines.add("");
+    lines.add(String.format("# %s", Instant.now()));
+    lines.add(String.format("# %s", configuration.toShortString()));
+    lines.add(configuration.toString());
+    lines.add(perf.toString());
+    try {
+      Files.write(Paths.get(perfFilename), lines, StandardCharsets.UTF_8, StandardOpenOption.CREATE,
+          StandardOpenOption.APPEND);
+    } catch (IOException e) {
+      throw new RuntimeException("Unable to write perf file: ", e);
+    }
+    NexmarkUtils.console("appended results to perf file %s.", perfFilename);
+  }
+
+  /**
+   * Load the baseline perf.
+   */
+  @Nullable
+  private static Map<NexmarkConfiguration, NexmarkPerf> loadBaseline(
+      @Nullable String baselineFilename) {
+    if (baselineFilename == null) {
+      return null;
+    }
+    Map<NexmarkConfiguration, NexmarkPerf> baseline = new LinkedHashMap<>();
+    List<String> lines;
+    try {
+      lines = Files.readAllLines(Paths.get(baselineFilename), StandardCharsets.UTF_8);
+    } catch (IOException e) {
+      throw new RuntimeException("Unable to read baseline perf file: ", e);
+    }
+    for (int i = 0; i < lines.size(); i++) {
+      if (lines.get(i).startsWith("#") || lines.get(i).trim().isEmpty()) {
+        continue;
+      }
+      NexmarkConfiguration configuration = NexmarkConfiguration.fromString(lines.get(i++));
+      NexmarkPerf perf = NexmarkPerf.fromString(lines.get(i));
+      baseline.put(configuration, perf);
+    }
+    NexmarkUtils.console("loaded %d entries from baseline file %s.", baseline.size(),
+        baselineFilename);
+    return baseline;
+  }
+
+  private static final String LINE =
+      "==========================================================================================";
+
+  /**
+   * Print summary  of {@code actual} vs (if non-null) {@code baseline}.
+   *
+   * @throws IOException
+   */
+  private static void saveSummary(
+      @Nullable String summaryFilename,
+      Iterable<NexmarkConfiguration> configurations, Map<NexmarkConfiguration, NexmarkPerf> actual,
+      @Nullable Map<NexmarkConfiguration, NexmarkPerf> baseline, Instant start) {
+    List<String> lines = new ArrayList<>();
+
+    lines.add("");
+    lines.add(LINE);
+
+    lines.add(
+        String.format("Run started %s and ran for %s", start, new Duration(start, Instant.now())));
+    lines.add("");
+
+    lines.add("Default configuration:");
+    lines.add(NexmarkConfiguration.DEFAULT.toString());
+    lines.add("");
+
+    lines.add("Configurations:");
+    lines.add("  Conf  Description");
+    int conf = 0;
+    for (NexmarkConfiguration configuration : configurations) {
+      lines.add(String.format("  %04d  %s", conf++, configuration.toShortString()));
+      NexmarkPerf actualPerf = actual.get(configuration);
+      if (actualPerf != null && actualPerf.jobId != null) {
+        lines.add(String.format("  %4s  [Ran as job %s]", "", actualPerf.jobId));
+      }
+    }
+
+    lines.add("");
+    lines.add("Performance:");
+    lines.add(String.format("  %4s  %12s  %12s  %12s  %12s  %12s  %12s", "Conf", "Runtime(sec)",
+        "(Baseline)", "Events(/sec)", "(Baseline)", "Results", "(Baseline)"));
+    conf = 0;
+    for (NexmarkConfiguration configuration : configurations) {
+      String line = String.format("  %04d  ", conf++);
+      NexmarkPerf actualPerf = actual.get(configuration);
+      if (actualPerf == null) {
+        line += "*** not run ***";
+      } else {
+        NexmarkPerf baselinePerf = baseline == null ? null : baseline.get(configuration);
+        double runtimeSec = actualPerf.runtimeSec;
+        line += String.format("%12.1f  ", runtimeSec);
+        if (baselinePerf == null) {
+          line += String.format("%12s  ", "");
+        } else {
+          double baselineRuntimeSec = baselinePerf.runtimeSec;
+          double diff = ((runtimeSec - baselineRuntimeSec) / baselineRuntimeSec) * 100.0;
+          line += String.format("%+11.2f%%  ", diff);
+        }
+
+        double eventsPerSec = actualPerf.eventsPerSec;
+        line += String.format("%12.1f  ", eventsPerSec);
+        if (baselinePerf == null) {
+          line += String.format("%12s  ", "");
+        } else {
+          double baselineEventsPerSec = baselinePerf.eventsPerSec;
+          double diff = ((eventsPerSec - baselineEventsPerSec) / baselineEventsPerSec) * 100.0;
+          line += String.format("%+11.2f%%  ", diff);
+        }
+
+        long numResults = actualPerf.numResults;
+        line += String.format("%12d  ", numResults);
+        if (baselinePerf == null) {
+          line += String.format("%12s", "");
+        } else {
+          long baselineNumResults = baselinePerf.numResults;
+          long diff = numResults - baselineNumResults;
+          line += String.format("%+12d", diff);
+        }
+      }
+      lines.add(line);
+
+      if (actualPerf != null) {
+        List<String> errors = actualPerf.errors;
+        if (errors == null) {
+          errors = new ArrayList<String>();
+          errors.add("NexmarkGoogleRunner returned null errors list");
+        }
+        for (String error : errors) {
+          lines.add(String.format("  %4s  *** %s ***", "", error));
+        }
+      }
+    }
+
+    lines.add(LINE);
+    lines.add("");
+
+    for (String line : lines) {
+      System.out.println(line);
+    }
+
+    if (summaryFilename != null) {
+      try {
+        Files.write(Paths.get(summaryFilename), lines, StandardCharsets.UTF_8,
+            StandardOpenOption.CREATE, StandardOpenOption.APPEND);
+      } catch (IOException e) {
+        throw new RuntimeException("Unable to save summary file: ", e);
+      }
+      NexmarkUtils.console("appended summary to summary file %s.", summaryFilename);
+    }
+  }
+
+  /**
+   * Write all perf data and any baselines to a javascript file which can be used by
+   * graphing page etc.
+   */
+  private static void saveJavascript(
+      @Nullable String javascriptFilename,
+      Iterable<NexmarkConfiguration> configurations, Map<NexmarkConfiguration, NexmarkPerf> actual,
+      @Nullable Map<NexmarkConfiguration, NexmarkPerf> baseline, Instant start) {
+    if (javascriptFilename == null) {
+      return;
+    }
+
+    List<String> lines = new ArrayList<>();
+    lines.add(String.format(
+        "// Run started %s and ran for %s", start, new Duration(start, Instant.now())));
+    lines.add("var all = [");
+
+    for (NexmarkConfiguration configuration : configurations) {
+      lines.add("  {");
+      lines.add(String.format("    config: %s", configuration));
+      NexmarkPerf actualPerf = actual.get(configuration);
+      if (actualPerf != null) {
+        lines.add(String.format("    ,perf: %s", actualPerf));
+      }
+      NexmarkPerf baselinePerf = baseline == null ? null : baseline.get(configuration);
+      if (baselinePerf != null) {
+        lines.add(String.format("    ,baseline: %s", baselinePerf));
+      }
+      lines.add("  },");
+    }
+
+    lines.add("];");
+
+    try {
+      Files.write(Paths.get(javascriptFilename), lines, StandardCharsets.UTF_8,
+          StandardOpenOption.CREATE, StandardOpenOption.TRUNCATE_EXISTING);
+    } catch (IOException e) {
+      throw new RuntimeException("Unable to save javascript file: ", e);
+    }
+    NexmarkUtils.console("saved javascript to file %s.", javascriptFilename);
+  }
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/1f08970a/integration/java/src/main/java/org/apache/beam/integration/nexmark/NexmarkFlinkDriver.java
----------------------------------------------------------------------
diff --git a/integration/java/src/main/java/org/apache/beam/integration/nexmark/NexmarkFlinkDriver.java b/integration/java/src/main/java/org/apache/beam/integration/nexmark/NexmarkFlinkDriver.java
new file mode 100644
index 0000000..0029a36
--- /dev/null
+++ b/integration/java/src/main/java/org/apache/beam/integration/nexmark/NexmarkFlinkDriver.java
@@ -0,0 +1,49 @@
+/*
+ * 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.beam.integration.nexmark;
+
+import org.apache.beam.runners.flink.FlinkPipelineOptions;
+import org.apache.beam.runners.flink.FlinkPipelineRunner;
+import org.apache.beam.sdk.options.PipelineOptionsFactory;
+
+/**
+ * Run NexMark queries using Beam-on-Flink runner.
+ */
+public class NexmarkFlinkDriver extends NexmarkDriver<NexmarkFlinkDriver.NexmarkFlinkOptions> {
+  /**
+   * Command line flags.
+   */
+  public interface NexmarkFlinkOptions extends Options, FlinkPipelineOptions {
+  }
+
+  /**
+   * Entry point.
+   */
+  public static void main(String[] args) {
+    // Gather command line args, baseline, configurations, etc.
+    NexmarkFlinkOptions options = PipelineOptionsFactory.fromArgs(args)
+                                                        .withValidation()
+                                                        .as(NexmarkFlinkOptions.class);
+    options.setRunner(FlinkPipelineRunner.class);
+    NexmarkFlinkRunner runner = new NexmarkFlinkRunner(options);
+    new NexmarkFlinkDriver().runAll(options, runner);
+  }
+}
+
+

http://git-wip-us.apache.org/repos/asf/beam/blob/1f08970a/integration/java/src/main/java/org/apache/beam/integration/nexmark/NexmarkFlinkRunner.java
----------------------------------------------------------------------
diff --git a/integration/java/src/main/java/org/apache/beam/integration/nexmark/NexmarkFlinkRunner.java b/integration/java/src/main/java/org/apache/beam/integration/nexmark/NexmarkFlinkRunner.java
new file mode 100644
index 0000000..569aef6
--- /dev/null
+++ b/integration/java/src/main/java/org/apache/beam/integration/nexmark/NexmarkFlinkRunner.java
@@ -0,0 +1,67 @@
+/*
+ * 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.beam.integration.nexmark;
+
+import javax.annotation.Nullable;
+
+/**
+ * Run a specific Nexmark query using the Bean-on-Flink runner.
+ */
+public class NexmarkFlinkRunner extends NexmarkRunner<NexmarkFlinkDriver.NexmarkFlinkOptions> {
+  @Override
+  protected boolean isStreaming() {
+    return options.isStreaming();
+  }
+
+  @Override
+  protected int coresPerWorker() {
+    return 4;
+  }
+
+  @Override
+  protected int maxNumWorkers() {
+    return 5;
+  }
+
+  @Override
+  protected boolean canMonitor() {
+    return false;
+  }
+
+  @Override
+  protected void invokeBuilderForPublishOnlyPipeline(
+      PipelineBuilder builder) {
+    builder.build(options);
+  }
+
+  @Override
+  protected void waitForPublisherPreload() {
+    throw new UnsupportedOperationException();
+  }
+
+  @Override
+  @Nullable
+  protected NexmarkPerf monitor(NexmarkQuery query) {
+    return null;
+  }
+
+  public NexmarkFlinkRunner(NexmarkFlinkDriver.NexmarkFlinkOptions options) {
+    super(options);
+  }
+}


[46/55] [abbrv] beam git commit: Move module beam-integration-java-nexmark to beam-sdks-java-nexmark

Posted by ie...@apache.org.
http://git-wip-us.apache.org/repos/asf/beam/blob/f4333df7/integration/java/pom.xml
----------------------------------------------------------------------
diff --git a/integration/java/pom.xml b/integration/java/pom.xml
deleted file mode 100644
index b0c3853..0000000
--- a/integration/java/pom.xml
+++ /dev/null
@@ -1,37 +0,0 @@
-<?xml version="1.0" encoding="UTF-8"?>
-<!--
-    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.
--->
-<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
-
-  <modelVersion>4.0.0</modelVersion>
-
-  <parent>
-    <groupId>org.apache.beam</groupId>
-    <artifactId>beam-integration-parent</artifactId>
-    <version>2.1.0-SNAPSHOT</version>
-    <relativePath>../pom.xml</relativePath>
-  </parent>
-
-  <artifactId>beam-integration-java-parent</artifactId>
-  <packaging>pom</packaging>
-  <name>Apache Beam :: Integration Tests :: Java</name>
-
-  <modules>
-    <module>nexmark</module>
-  </modules>
-
-</project>

http://git-wip-us.apache.org/repos/asf/beam/blob/f4333df7/integration/pom.xml
----------------------------------------------------------------------
diff --git a/integration/pom.xml b/integration/pom.xml
deleted file mode 100644
index 4254819..0000000
--- a/integration/pom.xml
+++ /dev/null
@@ -1,51 +0,0 @@
-<?xml version="1.0" encoding="UTF-8"?>
-<!--
-    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.
--->
-<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
-
-  <modelVersion>4.0.0</modelVersion>
-
-  <parent>
-    <groupId>org.apache.beam</groupId>
-    <artifactId>beam-parent</artifactId>
-    <version>2.1.0-SNAPSHOT</version>
-    <relativePath>../pom.xml</relativePath>
-  </parent>
-
-  <artifactId>beam-integration-parent</artifactId>
-  <packaging>pom</packaging>
-  <name>Apache Beam :: Integration Tests</name>
-
-  <profiles>
-    <profile>
-      <id>release</id>
-      <build>
-        <plugins>
-          <plugin>
-            <groupId>org.codehaus.mojo</groupId>
-            <artifactId>findbugs-maven-plugin</artifactId>
-          </plugin>
-        </plugins>
-      </build>
-    </profile>
-  </profiles>
-
-  <modules>
-    <module>java</module>
-  </modules>
-
-</project>

http://git-wip-us.apache.org/repos/asf/beam/blob/f4333df7/pom.xml
----------------------------------------------------------------------
diff --git a/pom.xml b/pom.xml
index bddbf1f..5fd1297 100644
--- a/pom.xml
+++ b/pom.xml
@@ -187,7 +187,6 @@
     <module>sdks</module>
     <module>runners</module>
     <module>examples</module>
-    <module>integration</module>
     <!-- sdks/java/javadoc builds project-wide Javadoc. It has to run last. -->
     <module>sdks/java/javadoc</module>
   </modules>

http://git-wip-us.apache.org/repos/asf/beam/blob/f4333df7/sdks/java/nexmark/README.md
----------------------------------------------------------------------
diff --git a/sdks/java/nexmark/README.md b/sdks/java/nexmark/README.md
new file mode 100644
index 0000000..a9acd63
--- /dev/null
+++ b/sdks/java/nexmark/README.md
@@ -0,0 +1,340 @@
+<!--
+    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.
+-->
+
+# NEXMark integration suite
+
+This is a suite of pipelines inspired by the 'continuous data stream'
+queries in [http://datalab.cs.pdx.edu/niagaraST/NEXMark/]
+(http://datalab.cs.pdx.edu/niagaraST/NEXMark/).
+
+These are multiple queries over a three entities model representing on online auction system:
+
+ - **Person** represents a person submitting an item for auction and/or making a bid
+    on an auction.
+ - **Auction** represents an item under auction.
+ - **Bid** represents a bid for an item under auction.
+
+The queries exercise many aspects of Beam model:
+
+* **Query1**: What are the bid values in Euro's?
+  Illustrates a simple map.
+* **Query2**: What are the auctions with particular auction numbers?
+  Illustrates a simple filter.
+* **Query3**: Who is selling in particular US states?
+  Illustrates an incremental join (using per-key state and timer) and filter.
+* **Query4**: What is the average selling price for each auction
+  category?
+  Illustrates complex join (using custom window functions) and
+  aggregation.
+* **Query5**: Which auctions have seen the most bids in the last period?
+  Illustrates sliding windows and combiners.
+* **Query6**: What is the average selling price per seller for their
+  last 10 closed auctions.
+  Shares the same 'winning bids' core as for **Query4**, and
+  illustrates a specialized combiner.
+* **Query7**: What are the highest bids per period?
+  Deliberately implemented using a side input to illustrate fanout.
+* **Query8**: Who has entered the system and created an auction in
+  the last period?
+  Illustrates a simple join.
+
+We have augmented the original queries with five more:
+
+* **Query0**: Pass-through.
+  Allows us to measure the monitoring overhead.
+* **Query9**: Winning-bids.
+  A common sub-query shared by **Query4** and **Query6**.
+* **Query10**: Log all events to GCS files.
+  Illustrates windows with large side effects on firing.
+* **Query11**: How many bids did a user make in each session they
+  were active?
+  Illustrates session windows.
+* **Query12**: How many bids does a user make within a fixed
+  processing time limit?
+  Illustrates working in processing time in the Global window, as
+  compared with event time in non-Global windows for all the other
+  queries.
+
+We can specify the Beam runner to use with maven profiles, available profiles are:
+
+* direct-runner
+* spark-runner
+* flink-runner
+* apex-runner
+
+The runner must also be specified like in any other Beam pipeline using
+
+    --runner
+
+
+Test data is deterministically synthesized on demand. The test
+data may be synthesized in the same pipeline as the query itself,
+or may be published to Pubsub.
+
+The query results may be:
+
+* Published to Pubsub.
+* Written to text files as plain text.
+* Written to text files using an Avro encoding.
+* Send to BigQuery.
+* Discarded.
+
+# Configuration
+
+## Common configuration parameters
+
+Decide if batch or streaming:
+
+    --streaming=true
+
+Number of events generators
+
+    --numEventGenerators=4
+
+Run query N
+
+    --query=N
+
+## Available Suites
+The suite to run can be chosen using this configuration parameter:
+
+    --suite=SUITE
+
+Available suites are:
+* DEFAULT: Test default configuration with query 0.
+* SMOKE: Run the 12 default configurations.
+* STRESS: Like smoke but for 1m events.
+* FULL_THROTTLE: Like SMOKE but 100m events.
+
+   
+
+## Apex specific configuration
+
+    --manageResources=false --monitorJobs=false
+
+## Dataflow specific configuration
+
+    --manageResources=false --monitorJobs=true \
+    --enforceEncodability=false --enforceImmutability=false
+    --project=<your project> \
+    --zone=<your zone> \
+    --workerMachineType=n1-highmem-8 \
+    --stagingLocation=<a gs path for staging> \
+    --runner=DataflowRunner \
+    --tempLocation=gs://talend-imejia/nexmark/temp/ \
+    --stagingLocation=gs://talend-imejia/nexmark/temp/staging/ \
+    --filesToStage=target/beam-integration-java-0.7.0-SNAPSHOT.jar
+
+## Direct specific configuration
+
+    --manageResources=false --monitorJobs=true \
+    --enforceEncodability=false --enforceImmutability=false
+
+## Flink specific configuration
+
+    --manageResources=false --monitorJobs=true \
+    --flinkMaster=local --parallelism=#numcores
+
+## Spark specific configuration
+
+    --manageResources=false --monitorJobs=true \
+    --sparkMaster=local \
+    -Dspark.ui.enabled=false -DSPARK_LOCAL_IP=localhost -Dsun.io.serialization.extendedDebugInfo=true
+
+# Current Status
+
+Open issues are tracked [here](https://github.com../../../../../issues):
+
+## Batch / Synthetic / Local
+
+| Query | Direct | Spark                                                        | Flink                                                      | Apex                                                         |
+| ----: | ------ | ------------------------------------------------------------ | ---------------------------------------------------------- | ------------------------------------------------------------ |
+|     0 | ok     | ok                                                           | ok                                                         | ok                                                           |
+|     1 | ok     | ok                                                           | ok                                                         | ok                                                           |
+|     2 | ok     | ok                                                           | ok                                                         | ok                                                           |
+|     3 | ok     | [BEAM-1035](https://issues.apache.org/jira/browse/BEAM-1035) | ok                                                         | [BEAM-1037](https://issues.apache.org/jira/browse/BEAM-1037) |
+|     4 | ok     | ok                                                           | ok                                                         | ok                                                           |
+|     5 | ok     | ok                                                           | ok                                                         | ok                                                           |
+|     6 | ok     | ok                                                           | ok                                                         | ok                                                           |
+|     7 | ok     | ok                                                           | ok                                                         | ok                                                           |
+|     8 | ok     | ok                                                           | ok                                                         | ok                                                           |
+|     9 | ok     | ok                                                           | ok                                                         | ok                                                           |
+|    10 | ok     | ok                                                           | ok                                                         | ok                                                           |
+|    11 | ok     | ok                                                           | ok                                                         | ok                                                           |
+|    12 | ok     | ok                                                           | ok                                                         | ok                                                           |
+
+## Streaming / Synthetic / Local
+
+| Query | Direct | Spark                                                        | Flink                                                      | Apex                                                         |
+| ----: | ------ | ------------------------------------------------------------ | ---------------------------------------------------------- | ------------------------------------------------------------ |
+|     0 | ok     | ok                                                           | [BEAM-593](https://issues.apache.org/jira/browse/BEAM-593) | ok                                                           |
+|     1 | ok     | ok                                                           | [BEAM-593](https://issues.apache.org/jira/browse/BEAM-593) | ok                                                           |
+|     2 | ok     | ok                                                           | [BEAM-593](https://issues.apache.org/jira/browse/BEAM-593) | ok                                                           |
+|     3 | ok     | [BEAM-1035](https://issues.apache.org/jira/browse/BEAM-1035) | [BEAM-593](https://issues.apache.org/jira/browse/BEAM-593) | [BEAM-1037](https://issues.apache.org/jira/browse/BEAM-1037) |
+|     4 | ok     | ok                                                           | [BEAM-593](https://issues.apache.org/jira/browse/BEAM-593) | ok                                                           |
+|     5 | ok     | ok                                                           | [BEAM-593](https://issues.apache.org/jira/browse/BEAM-593) | ok                                                           |
+|     6 | ok     | ok                                                           | [BEAM-593](https://issues.apache.org/jira/browse/BEAM-593) | ok                                                           |
+|     7 | ok     | [BEAM-2112](https://issues.apache.org/jira/browse/BEAM-2112) | [BEAM-593](https://issues.apache.org/jira/browse/BEAM-593) | ok                                                           |
+|     8 | ok     | ok                                                           | [BEAM-593](https://issues.apache.org/jira/browse/BEAM-593) | ok                                                           |
+|     9 | ok     | ok                                                           | [BEAM-593](https://issues.apache.org/jira/browse/BEAM-593) | ok                                                           |
+|    10 | ok     | ok                                                           | [BEAM-593](https://issues.apache.org/jira/browse/BEAM-593) | ok                                                           |
+|    11 | ok     | ok                                                           | [BEAM-593](https://issues.apache.org/jira/browse/BEAM-593) | ok                                                           |
+|    12 | ok     | ok                                                           | [BEAM-593](https://issues.apache.org/jira/browse/BEAM-593) | ok                                                           |
+
+## Batch / Synthetic / Cluster
+
+TODO
+
+| Query | Dataflow                       | Spark                          | Flink                          | Apex                           |
+| ----: | ------------------------------ | ------------------------------ | ------------------------------ | ------------------------------ |
+|     0 |                                |                                |                                |                                |
+
+## Streaming / Synthetic / Cluster
+
+TODO
+
+| Query | Dataflow                       | Spark                          | Flink                          | Apex                           |
+| ----: | ------------------------------ | ------------------------------ | ------------------------------ | ------------------------------ |
+|     0 |                                |                                |                                |                                |
+
+# Running Nexmark
+
+## Running SMOKE suite on the DirectRunner (local)
+
+Batch Mode
+
+    mvn exec:java -Dexec.mainClass=org.apache.beam.integration.nexmark.Main -Pdirect-runner -Dexec.args="--runner=DirectRunner --suite=SMOKE --streaming=false --manageResources=false --monitorJobs=true --enforceEncodability=true --enforceImmutability=true"
+
+Streaming Mode
+
+    mvn exec:java -Dexec.mainClass=org.apache.beam.integration.nexmark.Main -Pdirect-runner -Dexec.args="--runner=DirectRunner --suite=SMOKE --streaming=true --manageResources=false --monitorJobs=true --enforceEncodability=true --enforceImmutability=true"
+
+
+## Running SMOKE suite on the SparkRunner (local)
+
+Batch Mode
+
+    mvn exec:java -Dexec.mainClass=org.apache.beam.integration.nexmark.Main -Pspark-runner "-Dexec.args=--runner=SparkRunner --suite=SMOKE --streamTimeout=60 --streaming=false --manageResources=false --monitorJobs=true"
+
+Streaming Mode
+
+    mvn exec:java -Dexec.mainClass=org.apache.beam.integration.nexmark.Main -Pspark-runner "-Dexec.args=--runner=SparkRunner --suite=SMOKE --streamTimeout=60 --streaming=true --manageResources=false --monitorJobs=true"
+
+
+## Running SMOKE suite on the FlinkRunner (local)
+
+Batch Mode
+
+    mvn exec:java -Dexec.mainClass=org.apache.beam.integration.nexmark.Main -Pflink-runner "-Dexec.args=--runner=FlinkRunner --suite=SMOKE --streamTimeout=60 --streaming=false --manageResources=false --monitorJobs=true  --flinkMaster=local"
+
+Streaming Mode
+
+    mvn exec:java -Dexec.mainClass=org.apache.beam.integration.nexmark.Main -Pflink-runner "-Dexec.args=--runner=FlinkRunner --suite=SMOKE --streamTimeout=60 --streaming=true --manageResources=false --monitorJobs=true  --flinkMaster=local"
+
+
+## Running SMOKE suite on the ApexRunner (local)
+
+Batch Mode
+
+    mvn exec:java -Dexec.mainClass=org.apache.beam.integration.nexmark.Main -Papex-runner "-Dexec.args=--runner=ApexRunner --suite=SMOKE --streamTimeout=60 --streaming=false --manageResources=false --monitorJobs=false"
+
+Streaming Mode
+
+    mvn exec:java -Dexec.mainClass=org.apache.beam.integration.nexmark.Main -Papex-runner "-Dexec.args=--runner=ApexRunner --suite=SMOKE --streamTimeout=60 --streaming=true --manageResources=false --monitorJobs=false"
+
+
+## Running SMOKE suite on Google Cloud Dataflow
+
+Building package
+
+    mvn clean package -Pdataflow-runner
+
+Submit to Google Dataflow service
+
+
+```
+java -cp integration/java/target/beam-integration-java-nexmark-bundled-2.1.0-SNAPSHOT.jar \
+  org.apache.beam.integration.nexmark.Main \
+  --runner=DataflowRunner
+  --project=<your project> \
+  --zone=<your zone> \
+  --workerMachineType=n1-highmem-8 \
+  --stagingLocation=<a gs path for staging> \
+  --streaming=true \
+  --sourceType=PUBSUB \
+  --pubSubMode=PUBLISH_ONLY \
+  --pubsubTopic=<an existing Pubsub topic> \
+  --resourceNameMode=VERBATIM \
+  --manageResources=false \
+  --monitorJobs=false \
+  --numEventGenerators=64 \
+  --numWorkers=16 \
+  --maxNumWorkers=16 \
+  --suite=SMOKE \
+  --firstEventRate=100000 \
+  --nextEventRate=100000 \
+  --ratePeriodSec=3600 \
+  --isRateLimited=true \
+  --avgPersonByteSize=500 \
+  --avgAuctionByteSize=500 \
+  --avgBidByteSize=500 \
+  --probDelayedEvent=0.000001 \
+  --occasionalDelaySec=3600 \
+  --numEvents=0 \
+  --useWallclockEventTime=true \
+  --usePubsubPublishTime=true \
+  --experiments=enable_custom_pubsub_sink
+```
+
+```
+java -cp integration/java/target/beam-integration-java-nexmark-bundled-2.1.0-SNAPSHOT.jar \
+  org.apache.beam.integration.nexmark.Main \
+  --runner=DataflowRunner
+  --project=<your project> \
+  --zone=<your zone> \
+  --workerMachineType=n1-highmem-8 \
+  --stagingLocation=<a gs path for staging> \
+  --streaming=true \
+  --sourceType=PUBSUB \
+  --pubSubMode=SUBSCRIBE_ONLY \
+  --pubsubSubscription=<an existing Pubsub subscription to above topic> \
+  --resourceNameMode=VERBATIM \
+  --manageResources=false \
+  --monitorJobs=false \
+  --numWorkers=64 \
+  --maxNumWorkers=64 \
+  --suite=SMOKE \
+  --usePubsubPublishTime=true \
+  --outputPath=<a gs path under which log files will be written> \
+  --windowSizeSec=600 \
+  --occasionalDelaySec=3600 \
+  --maxLogEvents=10000 \
+  --experiments=enable_custom_pubsub_source
+```
+
+## Running query 0 on a Spark cluster with yarn
+
+Building package
+
+    mvn clean package -Pspark-runner
+
+Submit to the cluster
+
+    spark-submit --master yarn-client --class org.apache.beam.integration.nexmark.Main --driver-memory 512m --executor-memory 512m --executor-cores 1 beam-integration-java-nexmark-bundled-2.1.0-SNAPSHOT.jar --runner=SparkRunner --query=0 --streamTimeout=60 --streaming=false --manageResources=false --monitorJobs=true
+

http://git-wip-us.apache.org/repos/asf/beam/blob/f4333df7/sdks/java/nexmark/pom.xml
----------------------------------------------------------------------
diff --git a/sdks/java/nexmark/pom.xml b/sdks/java/nexmark/pom.xml
new file mode 100644
index 0000000..c1b6025
--- /dev/null
+++ b/sdks/java/nexmark/pom.xml
@@ -0,0 +1,292 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<!--
+    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.
+-->
+<project xmlns="http://maven.apache.org/POM/4.0.0"
+         xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
+         xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
+  <modelVersion>4.0.0</modelVersion>
+
+  <parent>
+    <groupId>org.apache.beam</groupId>
+    <artifactId>beam-sdks-java-parent</artifactId>
+    <version>2.1.0-SNAPSHOT</version>
+    <relativePath>../pom.xml</relativePath>
+  </parent>
+
+  <artifactId>beam-sdks-java-nexmark</artifactId>
+  <name>Apache Beam :: SDKs :: Java :: Nexmark</name>
+  <packaging>jar</packaging>
+
+  <profiles>
+
+    <!--
+      The direct runner is available by default.
+      You can also include it on the classpath explicitly with -P direct-runner
+    -->
+    <profile>
+      <id>direct-runner</id>
+      <activation>
+        <activeByDefault>true</activeByDefault>
+      </activation>
+      <dependencies>
+        <dependency>
+          <groupId>org.apache.beam</groupId>
+          <artifactId>beam-runners-direct-java</artifactId>
+          <scope>runtime</scope>
+        </dependency>
+      </dependencies>
+    </profile>
+
+    <!-- Include the Apache Apex runner with -P apex-runner -->
+    <profile>
+      <id>apex-runner</id>
+      <dependencies>
+        <dependency>
+          <groupId>org.apache.beam</groupId>
+          <artifactId>beam-runners-apex</artifactId>
+          <scope>runtime</scope>
+        </dependency>
+        <!--
+          Apex depends on httpclient version 4.3.5, project has a transitive dependency to httpclient 4.0.1 from
+          google-http-client. Apex dependency version being specified explicitly so that it gets picked up. This
+          can be removed when the project no longer has a dependency on a different httpclient version.
+        -->
+        <dependency>
+          <groupId>org.apache.httpcomponents</groupId>
+          <artifactId>httpclient</artifactId>
+          <version>4.3.5</version>
+          <scope>runtime</scope>
+          <exclusions>
+            <exclusion>
+              <groupId>commons-codec</groupId>
+              <artifactId>commons-codec</artifactId>
+            </exclusion>
+          </exclusions>
+        </dependency>
+      </dependencies>
+    </profile>
+
+    <!-- Include the Apache Flink runner with -P flink-runner -->
+    <profile>
+      <id>flink-runner</id>
+      <dependencies>
+        <dependency>
+          <groupId>org.apache.beam</groupId>
+          <artifactId>beam-runners-flink_2.10</artifactId>
+          <scope>runtime</scope>
+        </dependency>
+      </dependencies>
+    </profile>
+
+    <!-- Include the Apache Spark runner -P spark-runner -->
+    <profile>
+      <id>spark-runner</id>
+      <dependencies>
+        <dependency>
+          <groupId>org.apache.beam</groupId>
+          <artifactId>beam-runners-spark</artifactId>
+          <scope>runtime</scope>
+        </dependency>
+        <dependency>
+          <groupId>org.apache.spark</groupId>
+          <artifactId>spark-streaming_2.10</artifactId>
+          <version>${spark.version}</version>
+          <scope>runtime</scope>
+        </dependency>
+        <dependency>
+          <groupId>org.apache.spark</groupId>
+          <artifactId>spark-core_2.10</artifactId>
+          <version>${spark.version}</version>
+          <scope>runtime</scope>
+          <exclusions>
+            <exclusion>
+              <groupId>org.slf4j</groupId>
+              <artifactId>jul-to-slf4j</artifactId>
+            </exclusion>
+          </exclusions>
+        </dependency>
+      </dependencies>
+    </profile>
+
+    <!-- Include the Google Cloud Dataflow runner -P dataflow-runner -->
+    <profile>
+      <id>dataflow-runner</id>
+      <dependencies>
+        <dependency>
+          <groupId>org.apache.beam</groupId>
+          <artifactId>beam-runners-google-cloud-dataflow-java</artifactId>
+          <scope>runtime</scope>
+        </dependency>
+      </dependencies>
+    </profile>
+  </profiles>
+
+  <build>
+    <plugins>
+      <plugin>
+        <groupId>org.apache.maven.plugins</groupId>
+        <artifactId>maven-shade-plugin</artifactId>
+        <executions>
+          <execution>
+            <phase>package</phase>
+            <goals>
+              <goal>shade</goal>
+            </goals>
+            <configuration>
+              <finalName>${project.artifactId}-bundled-${project.version}</finalName>
+              <artifactSet>
+                <includes>
+                  <include>*:*</include>
+                </includes>
+              </artifactSet>
+              <filters>
+                <filter>
+                  <artifact>*:*</artifact>
+                  <excludes>
+                    <exclude>META-INF/*.SF</exclude>
+                    <exclude>META-INF/*.DSA</exclude>
+                    <exclude>META-INF/*.RSA</exclude>
+                  </excludes>
+                </filter>
+              </filters>
+            </configuration>
+          </execution>
+        </executions>
+      </plugin>
+
+      <!-- Avro plugin for automatic code generation -->
+      <plugin>
+        <groupId>org.apache.avro</groupId>
+        <artifactId>avro-maven-plugin</artifactId>
+        <executions>
+          <execution>
+            <id>schemas</id>
+            <phase>generate-sources</phase>
+            <goals>
+              <goal>schema</goal>
+            </goals>
+            <configuration>
+              <sourceDirectory>${project.basedir}/src/main/</sourceDirectory>
+              <outputDirectory>${project.build.directory}/generated-sources/java</outputDirectory>
+            </configuration>
+          </execution>
+        </executions>
+      </plugin>
+
+      <!-- Coverage analysis for unit tests. -->
+      <plugin>
+        <groupId>org.jacoco</groupId>
+        <artifactId>jacoco-maven-plugin</artifactId>
+      </plugin>
+    </plugins>
+  </build>
+
+  <dependencies>
+    <!-- Java SDK -->
+    <dependency>
+      <groupId>org.apache.beam</groupId>
+      <artifactId>beam-sdks-java-core</artifactId>
+    </dependency>
+
+    <!-- IOs -->
+    <dependency>
+      <groupId>org.apache.beam</groupId>
+      <artifactId>beam-sdks-java-io-google-cloud-platform</artifactId>
+    </dependency>
+
+    <dependency>
+      <groupId>org.apache.beam</groupId>
+      <artifactId>beam-sdks-java-extensions-google-cloud-platform-core</artifactId>
+    </dependency>
+
+    <dependency>
+      <groupId>com.google.apis</groupId>
+      <artifactId>google-api-services-bigquery</artifactId>
+    </dependency>
+
+    <!-- Extra libraries -->
+    <dependency>
+      <groupId>com.fasterxml.jackson.core</groupId>
+      <artifactId>jackson-core</artifactId>
+    </dependency>
+
+    <dependency>
+      <groupId>com.fasterxml.jackson.core</groupId>
+      <artifactId>jackson-annotations</artifactId>
+    </dependency>
+
+    <dependency>
+      <groupId>com.fasterxml.jackson.core</groupId>
+      <artifactId>jackson-databind</artifactId>
+    </dependency>
+
+    <dependency>
+      <groupId>org.apache.avro</groupId>
+      <artifactId>avro</artifactId>
+    </dependency>
+
+    <dependency>
+      <groupId>joda-time</groupId>
+      <artifactId>joda-time</artifactId>
+    </dependency>
+
+    <dependency>
+      <groupId>com.google.guava</groupId>
+      <artifactId>guava</artifactId>
+    </dependency>
+
+    <dependency>
+      <groupId>org.slf4j</groupId>
+      <artifactId>slf4j-api</artifactId>
+    </dependency>
+
+    <dependency>
+      <groupId>org.slf4j</groupId>
+      <artifactId>slf4j-jdk14</artifactId>
+      <scope>runtime</scope>
+    </dependency>
+
+    <dependency>
+      <groupId>com.google.code.findbugs</groupId>
+      <artifactId>jsr305</artifactId>
+    </dependency>
+
+    <dependency>
+      <groupId>junit</groupId>
+      <artifactId>junit</artifactId>
+      <scope>compile</scope>
+    </dependency>
+
+    <dependency>
+      <groupId>org.hamcrest</groupId>
+      <artifactId>hamcrest-core</artifactId>
+      <scope>compile</scope>
+    </dependency>
+
+    <dependency>
+      <groupId>org.hamcrest</groupId>
+      <artifactId>hamcrest-all</artifactId>
+    </dependency>
+
+    <!-- Test -->
+    <dependency>
+      <groupId>org.apache.beam</groupId>
+      <artifactId>beam-runners-direct-java</artifactId>
+      <scope>test</scope>
+    </dependency>
+  </dependencies>
+</project>

http://git-wip-us.apache.org/repos/asf/beam/blob/f4333df7/sdks/java/nexmark/src/main/java/org/apache/beam/sdk/nexmark/Main.java
----------------------------------------------------------------------
diff --git a/sdks/java/nexmark/src/main/java/org/apache/beam/sdk/nexmark/Main.java b/sdks/java/nexmark/src/main/java/org/apache/beam/sdk/nexmark/Main.java
new file mode 100644
index 0000000..ab2284c
--- /dev/null
+++ b/sdks/java/nexmark/src/main/java/org/apache/beam/sdk/nexmark/Main.java
@@ -0,0 +1,303 @@
+/*
+ * 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.beam.sdk.nexmark;
+
+import java.io.IOException;
+import java.nio.charset.StandardCharsets;
+import java.nio.file.Files;
+import java.nio.file.Paths;
+import java.nio.file.StandardOpenOption;
+import java.util.ArrayList;
+import java.util.LinkedHashMap;
+import java.util.List;
+import java.util.Map;
+import javax.annotation.Nullable;
+import org.apache.beam.sdk.nexmark.model.Auction;
+import org.apache.beam.sdk.nexmark.model.Bid;
+import org.apache.beam.sdk.nexmark.model.Person;
+import org.apache.beam.sdk.options.PipelineOptionsFactory;
+import org.joda.time.Duration;
+import org.joda.time.Instant;
+
+/**
+ * An implementation of the 'NEXMark queries' for Beam.
+ * These are multiple queries over a three table schema representing an online auction system:
+ * <ul>
+ * <li>{@link Person} represents a person submitting an item for auction and/or making a bid
+ * on an auction.
+ * <li>{@link Auction} represents an item under auction.
+ * <li>{@link Bid} represents a bid for an item under auction.
+ * </ul>
+ * The queries exercise many aspects of the Beam model.
+ *
+ * <p>We synthesize the creation of people, auctions and bids in real-time. The data is not
+ * particularly sensible.
+ *
+ * <p>See <a href="http://datalab.cs.pdx.edu/niagaraST/NEXMark/">
+ * http://datalab.cs.pdx.edu/niagaraST/NEXMark/</a>
+ */
+public class Main<OptionT extends NexmarkOptions> {
+
+  /**
+   * Entry point.
+   */
+  void runAll(OptionT options, NexmarkLauncher nexmarkLauncher) {
+    Instant start = Instant.now();
+    Map<NexmarkConfiguration, NexmarkPerf> baseline = loadBaseline(options.getBaselineFilename());
+    Map<NexmarkConfiguration, NexmarkPerf> actual = new LinkedHashMap<>();
+    Iterable<NexmarkConfiguration> configurations = options.getSuite().getConfigurations(options);
+
+    boolean successful = true;
+    try {
+      // Run all the configurations.
+      for (NexmarkConfiguration configuration : configurations) {
+        NexmarkPerf perf = nexmarkLauncher.run(configuration);
+        if (perf != null) {
+          if (perf.errors == null || perf.errors.size() > 0) {
+            successful = false;
+          }
+          appendPerf(options.getPerfFilename(), configuration, perf);
+          actual.put(configuration, perf);
+          // Summarize what we've run so far.
+          saveSummary(null, configurations, actual, baseline, start);
+        }
+      }
+    } finally {
+      if (options.getMonitorJobs()) {
+        // Report overall performance.
+        saveSummary(options.getSummaryFilename(), configurations, actual, baseline, start);
+        saveJavascript(options.getJavascriptFilename(), configurations, actual, baseline, start);
+      }
+    }
+
+    if (!successful) {
+      throw new RuntimeException("Execution was not successful");
+    }
+  }
+
+  /**
+   * Append the pair of {@code configuration} and {@code perf} to perf file.
+   */
+  private void appendPerf(
+      @Nullable String perfFilename, NexmarkConfiguration configuration,
+      NexmarkPerf perf) {
+    if (perfFilename == null) {
+      return;
+    }
+    List<String> lines = new ArrayList<>();
+    lines.add("");
+    lines.add(String.format("# %s", Instant.now()));
+    lines.add(String.format("# %s", configuration.toShortString()));
+    lines.add(configuration.toString());
+    lines.add(perf.toString());
+    try {
+      Files.write(Paths.get(perfFilename), lines, StandardCharsets.UTF_8, StandardOpenOption.CREATE,
+          StandardOpenOption.APPEND);
+    } catch (IOException e) {
+      throw new RuntimeException("Unable to write perf file: ", e);
+    }
+    NexmarkUtils.console("appended results to perf file %s.", perfFilename);
+  }
+
+  /**
+   * Load the baseline perf.
+   */
+  @Nullable
+  private static Map<NexmarkConfiguration, NexmarkPerf> loadBaseline(
+      @Nullable String baselineFilename) {
+    if (baselineFilename == null) {
+      return null;
+    }
+    Map<NexmarkConfiguration, NexmarkPerf> baseline = new LinkedHashMap<>();
+    List<String> lines;
+    try {
+      lines = Files.readAllLines(Paths.get(baselineFilename), StandardCharsets.UTF_8);
+    } catch (IOException e) {
+      throw new RuntimeException("Unable to read baseline perf file: ", e);
+    }
+    for (int i = 0; i < lines.size(); i++) {
+      if (lines.get(i).startsWith("#") || lines.get(i).trim().isEmpty()) {
+        continue;
+      }
+      NexmarkConfiguration configuration = NexmarkConfiguration.fromString(lines.get(i++));
+      NexmarkPerf perf = NexmarkPerf.fromString(lines.get(i));
+      baseline.put(configuration, perf);
+    }
+    NexmarkUtils.console("loaded %d entries from baseline file %s.", baseline.size(),
+        baselineFilename);
+    return baseline;
+  }
+
+  private static final String LINE =
+      "==========================================================================================";
+
+  /**
+   * Print summary  of {@code actual} vs (if non-null) {@code baseline}.
+   */
+  private static void saveSummary(
+      @Nullable String summaryFilename,
+      Iterable<NexmarkConfiguration> configurations, Map<NexmarkConfiguration, NexmarkPerf> actual,
+      @Nullable Map<NexmarkConfiguration, NexmarkPerf> baseline, Instant start) {
+    List<String> lines = new ArrayList<>();
+
+    lines.add("");
+    lines.add(LINE);
+
+    lines.add(
+        String.format("Run started %s and ran for %s", start, new Duration(start, Instant.now())));
+    lines.add("");
+
+    lines.add("Default configuration:");
+    lines.add(NexmarkConfiguration.DEFAULT.toString());
+    lines.add("");
+
+    lines.add("Configurations:");
+    lines.add("  Conf  Description");
+    int conf = 0;
+    for (NexmarkConfiguration configuration : configurations) {
+      lines.add(String.format("  %04d  %s", conf++, configuration.toShortString()));
+      NexmarkPerf actualPerf = actual.get(configuration);
+      if (actualPerf != null && actualPerf.jobId != null) {
+        lines.add(String.format("  %4s  [Ran as job %s]", "", actualPerf.jobId));
+      }
+    }
+
+    lines.add("");
+    lines.add("Performance:");
+    lines.add(String.format("  %4s  %12s  %12s  %12s  %12s  %12s  %12s", "Conf", "Runtime(sec)",
+        "(Baseline)", "Events(/sec)", "(Baseline)", "Results", "(Baseline)"));
+    conf = 0;
+    for (NexmarkConfiguration configuration : configurations) {
+      String line = String.format("  %04d  ", conf++);
+      NexmarkPerf actualPerf = actual.get(configuration);
+      if (actualPerf == null) {
+        line += "*** not run ***";
+      } else {
+        NexmarkPerf baselinePerf = baseline == null ? null : baseline.get(configuration);
+        double runtimeSec = actualPerf.runtimeSec;
+        line += String.format("%12.1f  ", runtimeSec);
+        if (baselinePerf == null) {
+          line += String.format("%12s  ", "");
+        } else {
+          double baselineRuntimeSec = baselinePerf.runtimeSec;
+          double diff = ((runtimeSec - baselineRuntimeSec) / baselineRuntimeSec) * 100.0;
+          line += String.format("%+11.2f%%  ", diff);
+        }
+
+        double eventsPerSec = actualPerf.eventsPerSec;
+        line += String.format("%12.1f  ", eventsPerSec);
+        if (baselinePerf == null) {
+          line += String.format("%12s  ", "");
+        } else {
+          double baselineEventsPerSec = baselinePerf.eventsPerSec;
+          double diff = ((eventsPerSec - baselineEventsPerSec) / baselineEventsPerSec) * 100.0;
+          line += String.format("%+11.2f%%  ", diff);
+        }
+
+        long numResults = actualPerf.numResults;
+        line += String.format("%12d  ", numResults);
+        if (baselinePerf == null) {
+          line += String.format("%12s", "");
+        } else {
+          long baselineNumResults = baselinePerf.numResults;
+          long diff = numResults - baselineNumResults;
+          line += String.format("%+12d", diff);
+        }
+      }
+      lines.add(line);
+
+      if (actualPerf != null) {
+        List<String> errors = actualPerf.errors;
+        if (errors == null) {
+          errors = new ArrayList<>();
+          errors.add("NexmarkGoogleRunner returned null errors list");
+        }
+        for (String error : errors) {
+          lines.add(String.format("  %4s  *** %s ***", "", error));
+        }
+      }
+    }
+
+    lines.add(LINE);
+    lines.add("");
+
+    for (String line : lines) {
+      System.out.println(line);
+    }
+
+    if (summaryFilename != null) {
+      try {
+        Files.write(Paths.get(summaryFilename), lines, StandardCharsets.UTF_8,
+            StandardOpenOption.CREATE, StandardOpenOption.APPEND);
+      } catch (IOException e) {
+        throw new RuntimeException("Unable to save summary file: ", e);
+      }
+      NexmarkUtils.console("appended summary to summary file %s.", summaryFilename);
+    }
+  }
+
+  /**
+   * Write all perf data and any baselines to a javascript file which can be used by
+   * graphing page etc.
+   */
+  private static void saveJavascript(
+      @Nullable String javascriptFilename,
+      Iterable<NexmarkConfiguration> configurations, Map<NexmarkConfiguration, NexmarkPerf> actual,
+      @Nullable Map<NexmarkConfiguration, NexmarkPerf> baseline, Instant start) {
+    if (javascriptFilename == null) {
+      return;
+    }
+
+    List<String> lines = new ArrayList<>();
+    lines.add(String.format(
+        "// Run started %s and ran for %s", start, new Duration(start, Instant.now())));
+    lines.add("var all = [");
+
+    for (NexmarkConfiguration configuration : configurations) {
+      lines.add("  {");
+      lines.add(String.format("    config: %s", configuration));
+      NexmarkPerf actualPerf = actual.get(configuration);
+      if (actualPerf != null) {
+        lines.add(String.format("    ,perf: %s", actualPerf));
+      }
+      NexmarkPerf baselinePerf = baseline == null ? null : baseline.get(configuration);
+      if (baselinePerf != null) {
+        lines.add(String.format("    ,baseline: %s", baselinePerf));
+      }
+      lines.add("  },");
+    }
+
+    lines.add("];");
+
+    try {
+      Files.write(Paths.get(javascriptFilename), lines, StandardCharsets.UTF_8,
+          StandardOpenOption.CREATE, StandardOpenOption.TRUNCATE_EXISTING);
+    } catch (IOException e) {
+      throw new RuntimeException("Unable to save javascript file: ", e);
+    }
+    NexmarkUtils.console("saved javascript to file %s.", javascriptFilename);
+  }
+
+  public static void main(String[] args) {
+    NexmarkOptions options = PipelineOptionsFactory.fromArgs(args)
+      .withValidation()
+      .as(NexmarkOptions.class);
+    NexmarkLauncher<NexmarkOptions> nexmarkLauncher = new NexmarkLauncher<>(options);
+    new Main<>().runAll(options, nexmarkLauncher);
+  }
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/f4333df7/sdks/java/nexmark/src/main/java/org/apache/beam/sdk/nexmark/Monitor.java
----------------------------------------------------------------------
diff --git a/sdks/java/nexmark/src/main/java/org/apache/beam/sdk/nexmark/Monitor.java b/sdks/java/nexmark/src/main/java/org/apache/beam/sdk/nexmark/Monitor.java
new file mode 100644
index 0000000..f45c387
--- /dev/null
+++ b/sdks/java/nexmark/src/main/java/org/apache/beam/sdk/nexmark/Monitor.java
@@ -0,0 +1,78 @@
+/*
+ * 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.beam.sdk.nexmark;
+
+import java.io.Serializable;
+import org.apache.beam.sdk.metrics.Counter;
+import org.apache.beam.sdk.metrics.Distribution;
+import org.apache.beam.sdk.metrics.Metrics;
+import org.apache.beam.sdk.nexmark.model.KnownSize;
+import org.apache.beam.sdk.transforms.DoFn;
+import org.apache.beam.sdk.transforms.PTransform;
+import org.apache.beam.sdk.transforms.ParDo;
+import org.apache.beam.sdk.values.PCollection;
+
+/**
+ * A monitor of elements with support for later retrieving their metrics.
+ *
+ * @param <T> Type of element we are monitoring.
+ */
+public class Monitor<T extends KnownSize> implements Serializable {
+  private class MonitorDoFn extends DoFn<T, T> {
+    final Counter elementCounter =
+      Metrics.counter(name , prefix + ".elements");
+    final Counter bytesCounter =
+      Metrics.counter(name , prefix + ".bytes");
+    final Distribution startTime =
+      Metrics.distribution(name , prefix + ".startTime");
+    final Distribution endTime =
+      Metrics.distribution(name , prefix + ".endTime");
+    final Distribution startTimestamp =
+      Metrics.distribution(name , prefix + ".startTimestamp");
+    final Distribution endTimestamp =
+      Metrics.distribution(name , prefix + ".endTimestamp");
+
+    @ProcessElement
+    public void processElement(ProcessContext c) {
+      elementCounter.inc();
+      bytesCounter.inc(c.element().sizeInBytes());
+      long now = System.currentTimeMillis();
+      startTime.update(now);
+      endTime.update(now);
+      startTimestamp.update(c.timestamp().getMillis());
+      endTimestamp.update(c.timestamp().getMillis());
+      c.output(c.element());
+    }
+  }
+
+  public final String name;
+  public final String prefix;
+  private final MonitorDoFn doFn;
+  private final PTransform<PCollection<? extends T>, PCollection<T>> transform;
+
+  public Monitor(String name, String prefix) {
+    this.name = name;
+    this.prefix = prefix;
+    doFn = new MonitorDoFn();
+    transform = ParDo.of(doFn);
+  }
+
+  public PTransform<PCollection<? extends T>, PCollection<T>> getTransform() {
+    return transform;
+  }
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/f4333df7/sdks/java/nexmark/src/main/java/org/apache/beam/sdk/nexmark/NexmarkConfiguration.java
----------------------------------------------------------------------
diff --git a/sdks/java/nexmark/src/main/java/org/apache/beam/sdk/nexmark/NexmarkConfiguration.java b/sdks/java/nexmark/src/main/java/org/apache/beam/sdk/nexmark/NexmarkConfiguration.java
new file mode 100644
index 0000000..904fcd5
--- /dev/null
+++ b/sdks/java/nexmark/src/main/java/org/apache/beam/sdk/nexmark/NexmarkConfiguration.java
@@ -0,0 +1,721 @@
+/*
+ * 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.beam.sdk.nexmark;
+
+import com.fasterxml.jackson.annotation.JsonProperty;
+import com.fasterxml.jackson.core.JsonProcessingException;
+import java.io.IOException;
+import java.io.Serializable;
+import java.util.Objects;
+
+/**
+ * Configuration controlling how a query is run. May be supplied by command line or
+ * programmatically. We only capture properties which may influence the resulting
+ * pipeline performance, as captured by {@link NexmarkPerf}.
+ */
+public class NexmarkConfiguration implements Serializable {
+  public static final NexmarkConfiguration DEFAULT = new NexmarkConfiguration();
+
+  /** If {@literal true}, include additional debugging and monitoring stats. */
+  @JsonProperty
+  public boolean debug = true;
+
+  /** Which query to run, in [0,9]. */
+  @JsonProperty
+  public int query = 0;
+
+  /** Where events come from. */
+  @JsonProperty
+  public NexmarkUtils.SourceType sourceType = NexmarkUtils.SourceType.DIRECT;
+
+  /** Where results go to. */
+  @JsonProperty
+  public NexmarkUtils.SinkType sinkType = NexmarkUtils.SinkType.DEVNULL;
+
+  /**
+   * Control whether pub/sub publishing is done in a stand-alone pipeline or is integrated
+   * into the overall query pipeline.
+   */
+  @JsonProperty
+  public NexmarkUtils.PubSubMode pubSubMode = NexmarkUtils.PubSubMode.COMBINED;
+
+  /**
+   * Number of events to generate. If zero, generate as many as possible without overflowing
+   * internal counters etc.
+   */
+  @JsonProperty
+  public long numEvents = 100000;
+
+  /**
+   * Number of event generators to use. Each generates events in its own timeline.
+   */
+  @JsonProperty
+  public int numEventGenerators = 100;
+
+  /**
+   * Shape of event rate curve.
+   */
+  @JsonProperty
+  public NexmarkUtils.RateShape rateShape = NexmarkUtils.RateShape.SINE;
+
+  /**
+   * Initial overall event rate (in {@link #rateUnit}).
+   */
+  @JsonProperty
+  public int firstEventRate = 10000;
+
+  /**
+   * Next overall event rate (in {@link #rateUnit}).
+   */
+  @JsonProperty
+  public int nextEventRate = 10000;
+
+  /**
+   * Unit for rates.
+   */
+  @JsonProperty
+  public NexmarkUtils.RateUnit rateUnit = NexmarkUtils.RateUnit.PER_SECOND;
+
+  /**
+   * Overall period of rate shape, in seconds.
+   */
+  @JsonProperty
+  public int ratePeriodSec = 600;
+
+  /**
+   * Time in seconds to preload the subscription with data, at the initial input rate of the
+   * pipeline.
+   */
+  @JsonProperty
+  public int preloadSeconds = 0;
+
+  /**
+   * Timeout for stream pipelines to stop in seconds.
+   */
+  @JsonProperty
+  public int streamTimeout = 240;
+
+  /**
+   * If true, and in streaming mode, generate events only when they are due according to their
+   * timestamp.
+   */
+  @JsonProperty
+  public boolean isRateLimited = false;
+
+  /**
+   * If true, use wallclock time as event time. Otherwise, use a deterministic
+   * time in the past so that multiple runs will see exactly the same event streams
+   * and should thus have exactly the same results.
+   */
+  @JsonProperty
+  public boolean useWallclockEventTime = false;
+
+  /** Average idealized size of a 'new person' event, in bytes. */
+  @JsonProperty
+  public int avgPersonByteSize = 200;
+
+  /** Average idealized size of a 'new auction' event, in bytes. */
+  @JsonProperty
+  public int avgAuctionByteSize = 500;
+
+  /** Average idealized size of a 'bid' event, in bytes. */
+  @JsonProperty
+  public int avgBidByteSize = 100;
+
+  /** Ratio of bids to 'hot' auctions compared to all other auctions. */
+  @JsonProperty
+  public int hotAuctionRatio = 2;
+
+  /** Ratio of auctions for 'hot' sellers compared to all other people. */
+  @JsonProperty
+  public int hotSellersRatio = 4;
+
+  /** Ratio of bids for 'hot' bidders compared to all other people. */
+  @JsonProperty
+  public int hotBiddersRatio = 4;
+
+  /** Window size, in seconds, for queries 3, 5, 7 and 8. */
+  @JsonProperty
+  public long windowSizeSec = 10;
+
+  /** Sliding window period, in seconds, for query 5. */
+  @JsonProperty
+  public long windowPeriodSec = 5;
+
+  /** Number of seconds to hold back events according to their reported timestamp. */
+  @JsonProperty
+  public long watermarkHoldbackSec = 0;
+
+  /** Average number of auction which should be inflight at any time, per generator. */
+  @JsonProperty
+  public int numInFlightAuctions = 100;
+
+  /** Maximum number of people to consider as active for placing auctions or bids. */
+  @JsonProperty
+  public int numActivePeople = 1000;
+
+  /** Coder strategy to follow. */
+  @JsonProperty
+  public NexmarkUtils.CoderStrategy coderStrategy = NexmarkUtils.CoderStrategy.HAND;
+
+  /**
+   * Delay, in milliseconds, for each event. This will peg one core for this number
+   * of milliseconds to simulate CPU-bound computation.
+   */
+  @JsonProperty
+  public long cpuDelayMs = 0;
+
+  /**
+   * Extra data, in bytes, to save to persistent state for each event. This will force
+   * i/o all the way to durable storage to simulate an I/O-bound computation.
+   */
+  @JsonProperty
+  public long diskBusyBytes = 0;
+
+  /**
+   * Skip factor for query 2. We select bids for every {@code auctionSkip}'th auction.
+   */
+  @JsonProperty
+  public int auctionSkip = 123;
+
+  /**
+   * Fanout for queries 4 (groups by category id), 5 and 7 (find a global maximum).
+   */
+  @JsonProperty
+  public int fanout = 5;
+
+  /**
+   * Maximum waiting time to clean personState in query3
+   * (ie maximum waiting of the auctions related to person in state in seconds in event time).
+   */
+  @JsonProperty
+  public int maxAuctionsWaitingTime = 600;
+
+  /**
+   * Length of occasional delay to impose on events (in seconds).
+   */
+  @JsonProperty
+  public long occasionalDelaySec = 3;
+
+  /**
+   * Probability that an event will be delayed by delayS.
+   */
+  @JsonProperty
+  public double probDelayedEvent = 0.1;
+
+  /**
+   * Maximum size of each log file (in events). For Query10 only.
+   */
+  @JsonProperty
+  public int maxLogEvents = 100_000;
+
+  /**
+   * If true, use pub/sub publish time instead of event time.
+   */
+  @JsonProperty
+  public boolean usePubsubPublishTime = false;
+
+  /**
+   * Number of events in out-of-order groups. 1 implies no out-of-order events. 1000 implies
+   * every 1000 events per generator are emitted in pseudo-random order.
+   */
+  @JsonProperty
+  public long outOfOrderGroupSize = 1;
+
+  /**
+   * Replace any properties of this configuration which have been supplied by the command line.
+   */
+  public void overrideFromOptions(NexmarkOptions options) {
+    if (options.getDebug() != null) {
+      debug = options.getDebug();
+    }
+    if (options.getQuery() != null) {
+      query = options.getQuery();
+    }
+    if (options.getSourceType() != null) {
+      sourceType = options.getSourceType();
+    }
+    if (options.getSinkType() != null) {
+      sinkType = options.getSinkType();
+    }
+    if (options.getPubSubMode() != null) {
+      pubSubMode = options.getPubSubMode();
+    }
+    if (options.getNumEvents() != null) {
+      numEvents = options.getNumEvents();
+    }
+    if (options.getNumEventGenerators() != null) {
+      numEventGenerators = options.getNumEventGenerators();
+    }
+    if (options.getRateShape() != null) {
+      rateShape = options.getRateShape();
+    }
+    if (options.getFirstEventRate() != null) {
+      firstEventRate = options.getFirstEventRate();
+    }
+    if (options.getNextEventRate() != null) {
+      nextEventRate = options.getNextEventRate();
+    }
+    if (options.getRateUnit() != null) {
+      rateUnit = options.getRateUnit();
+    }
+    if (options.getRatePeriodSec() != null) {
+      ratePeriodSec = options.getRatePeriodSec();
+    }
+    if (options.getPreloadSeconds() != null) {
+      preloadSeconds = options.getPreloadSeconds();
+    }
+    if (options.getStreamTimeout() != null) {
+      streamTimeout = options.getStreamTimeout();
+    }
+    if (options.getIsRateLimited() != null) {
+      isRateLimited = options.getIsRateLimited();
+    }
+    if (options.getUseWallclockEventTime() != null) {
+      useWallclockEventTime = options.getUseWallclockEventTime();
+    }
+    if (options.getAvgPersonByteSize() != null) {
+      avgPersonByteSize = options.getAvgPersonByteSize();
+    }
+    if (options.getAvgAuctionByteSize() != null) {
+      avgAuctionByteSize = options.getAvgAuctionByteSize();
+    }
+    if (options.getAvgBidByteSize() != null) {
+      avgBidByteSize = options.getAvgBidByteSize();
+    }
+    if (options.getHotAuctionRatio() != null) {
+      hotAuctionRatio = options.getHotAuctionRatio();
+    }
+    if (options.getHotSellersRatio() != null) {
+      hotSellersRatio = options.getHotSellersRatio();
+    }
+    if (options.getHotBiddersRatio() != null) {
+      hotBiddersRatio = options.getHotBiddersRatio();
+    }
+    if (options.getWindowSizeSec() != null) {
+      windowSizeSec = options.getWindowSizeSec();
+    }
+    if (options.getWindowPeriodSec() != null) {
+      windowPeriodSec = options.getWindowPeriodSec();
+    }
+    if (options.getWatermarkHoldbackSec() != null) {
+      watermarkHoldbackSec = options.getWatermarkHoldbackSec();
+    }
+    if (options.getNumInFlightAuctions() != null) {
+      numInFlightAuctions = options.getNumInFlightAuctions();
+    }
+    if (options.getNumActivePeople() != null) {
+      numActivePeople = options.getNumActivePeople();
+    }
+    if (options.getCoderStrategy() != null) {
+      coderStrategy = options.getCoderStrategy();
+    }
+    if (options.getCpuDelayMs() != null) {
+      cpuDelayMs = options.getCpuDelayMs();
+    }
+    if (options.getDiskBusyBytes() != null) {
+      diskBusyBytes = options.getDiskBusyBytes();
+    }
+    if (options.getAuctionSkip() != null) {
+      auctionSkip = options.getAuctionSkip();
+    }
+    if (options.getFanout() != null) {
+      fanout = options.getFanout();
+    }
+    if (options.getMaxAuctionsWaitingTime() != null) {
+      fanout = options.getMaxAuctionsWaitingTime();
+    }
+    if (options.getOccasionalDelaySec() != null) {
+      occasionalDelaySec = options.getOccasionalDelaySec();
+    }
+    if (options.getProbDelayedEvent() != null) {
+      probDelayedEvent = options.getProbDelayedEvent();
+    }
+    if (options.getMaxLogEvents() != null) {
+      maxLogEvents = options.getMaxLogEvents();
+    }
+    if (options.getUsePubsubPublishTime() != null) {
+      usePubsubPublishTime = options.getUsePubsubPublishTime();
+    }
+    if (options.getOutOfOrderGroupSize() != null) {
+      outOfOrderGroupSize = options.getOutOfOrderGroupSize();
+    }
+  }
+
+  /**
+   * Return copy of configuration with given label.
+   */
+  public NexmarkConfiguration copy() {
+    NexmarkConfiguration result;
+    result = new NexmarkConfiguration();
+    result.debug = debug;
+    result.query = query;
+    result.sourceType = sourceType;
+    result.sinkType = sinkType;
+    result.pubSubMode = pubSubMode;
+    result.numEvents = numEvents;
+    result.numEventGenerators = numEventGenerators;
+    result.rateShape = rateShape;
+    result.firstEventRate = firstEventRate;
+    result.nextEventRate = nextEventRate;
+    result.rateUnit = rateUnit;
+    result.ratePeriodSec = ratePeriodSec;
+    result.preloadSeconds = preloadSeconds;
+    result.streamTimeout = streamTimeout;
+    result.isRateLimited = isRateLimited;
+    result.useWallclockEventTime = useWallclockEventTime;
+    result.avgPersonByteSize = avgPersonByteSize;
+    result.avgAuctionByteSize = avgAuctionByteSize;
+    result.avgBidByteSize = avgBidByteSize;
+    result.hotAuctionRatio = hotAuctionRatio;
+    result.hotSellersRatio = hotSellersRatio;
+    result.hotBiddersRatio = hotBiddersRatio;
+    result.windowSizeSec = windowSizeSec;
+    result.windowPeriodSec = windowPeriodSec;
+    result.watermarkHoldbackSec = watermarkHoldbackSec;
+    result.numInFlightAuctions = numInFlightAuctions;
+    result.numActivePeople = numActivePeople;
+    result.coderStrategy = coderStrategy;
+    result.cpuDelayMs = cpuDelayMs;
+    result.diskBusyBytes = diskBusyBytes;
+    result.auctionSkip = auctionSkip;
+    result.fanout = fanout;
+    result.maxAuctionsWaitingTime = maxAuctionsWaitingTime;
+    result.occasionalDelaySec = occasionalDelaySec;
+    result.probDelayedEvent = probDelayedEvent;
+    result.maxLogEvents = maxLogEvents;
+    result.usePubsubPublishTime = usePubsubPublishTime;
+    result.outOfOrderGroupSize = outOfOrderGroupSize;
+    return result;
+  }
+
+  /**
+   * Return short description of configuration (suitable for use in logging). We only render
+   * the core fields plus those which do not have default values.
+   */
+  public String toShortString() {
+    StringBuilder sb = new StringBuilder();
+    sb.append(String.format("query:%d", query));
+    if (debug != DEFAULT.debug) {
+      sb.append(String.format("; debug:%s", debug));
+    }
+    if (sourceType != DEFAULT.sourceType) {
+      sb.append(String.format("; sourceType:%s", sourceType));
+    }
+    if (sinkType != DEFAULT.sinkType) {
+      sb.append(String.format("; sinkType:%s", sinkType));
+    }
+    if (pubSubMode != DEFAULT.pubSubMode) {
+      sb.append(String.format("; pubSubMode:%s", pubSubMode));
+    }
+    if (numEvents != DEFAULT.numEvents) {
+      sb.append(String.format("; numEvents:%d", numEvents));
+    }
+    if (numEventGenerators != DEFAULT.numEventGenerators) {
+      sb.append(String.format("; numEventGenerators:%d", numEventGenerators));
+    }
+    if (rateShape != DEFAULT.rateShape) {
+      sb.append(String.format("; rateShape:%s", rateShape));
+    }
+    if (firstEventRate != DEFAULT.firstEventRate || nextEventRate != DEFAULT.nextEventRate) {
+      sb.append(String.format("; firstEventRate:%d", firstEventRate));
+      sb.append(String.format("; nextEventRate:%d", nextEventRate));
+    }
+    if (rateUnit != DEFAULT.rateUnit) {
+      sb.append(String.format("; rateUnit:%s", rateUnit));
+    }
+    if (ratePeriodSec != DEFAULT.ratePeriodSec) {
+      sb.append(String.format("; ratePeriodSec:%d", ratePeriodSec));
+    }
+    if (preloadSeconds != DEFAULT.preloadSeconds) {
+      sb.append(String.format("; preloadSeconds:%d", preloadSeconds));
+    }
+    if (streamTimeout != DEFAULT.streamTimeout) {
+      sb.append(String.format("; streamTimeout:%d", streamTimeout));
+    }
+    if (isRateLimited != DEFAULT.isRateLimited) {
+      sb.append(String.format("; isRateLimited:%s", isRateLimited));
+    }
+    if (useWallclockEventTime != DEFAULT.useWallclockEventTime) {
+      sb.append(String.format("; useWallclockEventTime:%s", useWallclockEventTime));
+    }
+    if (avgPersonByteSize != DEFAULT.avgPersonByteSize) {
+      sb.append(String.format("; avgPersonByteSize:%d", avgPersonByteSize));
+    }
+    if (avgAuctionByteSize != DEFAULT.avgAuctionByteSize) {
+      sb.append(String.format("; avgAuctionByteSize:%d", avgAuctionByteSize));
+    }
+    if (avgBidByteSize != DEFAULT.avgBidByteSize) {
+      sb.append(String.format("; avgBidByteSize:%d", avgBidByteSize));
+    }
+    if (hotAuctionRatio != DEFAULT.hotAuctionRatio) {
+      sb.append(String.format("; hotAuctionRatio:%d", hotAuctionRatio));
+    }
+    if (hotSellersRatio != DEFAULT.hotSellersRatio) {
+      sb.append(String.format("; hotSellersRatio:%d", hotSellersRatio));
+    }
+    if (hotBiddersRatio != DEFAULT.hotBiddersRatio) {
+      sb.append(String.format("; hotBiddersRatio:%d", hotBiddersRatio));
+    }
+    if (windowSizeSec != DEFAULT.windowSizeSec) {
+      sb.append(String.format("; windowSizeSec:%d", windowSizeSec));
+    }
+    if (windowPeriodSec != DEFAULT.windowPeriodSec) {
+      sb.append(String.format("; windowPeriodSec:%d", windowPeriodSec));
+    }
+    if (watermarkHoldbackSec != DEFAULT.watermarkHoldbackSec) {
+      sb.append(String.format("; watermarkHoldbackSec:%d", watermarkHoldbackSec));
+    }
+    if (numInFlightAuctions != DEFAULT.numInFlightAuctions) {
+      sb.append(String.format("; numInFlightAuctions:%d", numInFlightAuctions));
+    }
+    if (numActivePeople != DEFAULT.numActivePeople) {
+      sb.append(String.format("; numActivePeople:%d", numActivePeople));
+    }
+    if (coderStrategy != DEFAULT.coderStrategy) {
+      sb.append(String.format("; coderStrategy:%s", coderStrategy));
+    }
+    if (cpuDelayMs != DEFAULT.cpuDelayMs) {
+      sb.append(String.format("; cpuSlowdownMs:%d", cpuDelayMs));
+    }
+    if (diskBusyBytes != DEFAULT.diskBusyBytes) {
+      sb.append(String.format("; diskBuysBytes:%d", diskBusyBytes));
+    }
+    if (auctionSkip != DEFAULT.auctionSkip) {
+      sb.append(String.format("; auctionSkip:%d", auctionSkip));
+    }
+    if (fanout != DEFAULT.fanout) {
+      sb.append(String.format("; fanout:%d", fanout));
+    }
+    if (maxAuctionsWaitingTime != DEFAULT.maxAuctionsWaitingTime) {
+      sb.append(String.format("; maxAuctionsWaitingTime:%d", fanout));
+    }
+    if (occasionalDelaySec != DEFAULT.occasionalDelaySec) {
+      sb.append(String.format("; occasionalDelaySec:%d", occasionalDelaySec));
+    }
+    if (probDelayedEvent != DEFAULT.probDelayedEvent) {
+      sb.append(String.format("; probDelayedEvent:%f", probDelayedEvent));
+    }
+    if (maxLogEvents != DEFAULT.maxLogEvents) {
+      sb.append(String.format("; maxLogEvents:%d", maxLogEvents));
+    }
+    if (usePubsubPublishTime != DEFAULT.usePubsubPublishTime) {
+      sb.append(String.format("; usePubsubPublishTime:%s", usePubsubPublishTime));
+    }
+    if (outOfOrderGroupSize != DEFAULT.outOfOrderGroupSize) {
+      sb.append(String.format("; outOfOrderGroupSize:%d", outOfOrderGroupSize));
+    }
+    return sb.toString();
+  }
+
+  /**
+   * Return full description as a string.
+   */
+  @Override
+  public String toString() {
+    try {
+      return NexmarkUtils.MAPPER.writeValueAsString(this);
+    } catch (JsonProcessingException e) {
+      throw new RuntimeException(e);
+    }
+  }
+
+  /**
+   * Parse an object from {@code string}.
+   */
+  public static NexmarkConfiguration fromString(String string) {
+    try {
+      return NexmarkUtils.MAPPER.readValue(string, NexmarkConfiguration.class);
+    } catch (IOException e) {
+      throw new RuntimeException("Unable to parse nexmark configuration: ", e);
+    }
+  }
+
+  @Override
+  public int hashCode() {
+    return Objects.hash(
+        debug,
+        query,
+        sourceType,
+        sinkType,
+        pubSubMode,
+        numEvents,
+        numEventGenerators,
+        rateShape,
+        firstEventRate,
+        nextEventRate,
+        rateUnit,
+        ratePeriodSec,
+        preloadSeconds,
+        streamTimeout,
+        isRateLimited,
+        useWallclockEventTime,
+        avgPersonByteSize,
+        avgAuctionByteSize,
+        avgBidByteSize,
+        hotAuctionRatio,
+        hotSellersRatio,
+        hotBiddersRatio,
+        windowSizeSec,
+        windowPeriodSec,
+        watermarkHoldbackSec,
+        numInFlightAuctions,
+        numActivePeople,
+        coderStrategy,
+        cpuDelayMs,
+        diskBusyBytes,
+        auctionSkip,
+        fanout,
+        maxAuctionsWaitingTime,
+        occasionalDelaySec,
+        probDelayedEvent,
+        maxLogEvents,
+        usePubsubPublishTime,
+        outOfOrderGroupSize);
+  }
+
+  @Override
+  public boolean equals(Object obj) {
+    if (this == obj) {
+      return true;
+    }
+    if (obj == null) {
+      return false;
+    }
+    if (getClass() != obj.getClass()) {
+      return false;
+    }
+    NexmarkConfiguration other = (NexmarkConfiguration) obj;
+    if (debug != other.debug) {
+      return false;
+    }
+    if (auctionSkip != other.auctionSkip) {
+      return false;
+    }
+    if (avgAuctionByteSize != other.avgAuctionByteSize) {
+      return false;
+    }
+    if (avgBidByteSize != other.avgBidByteSize) {
+      return false;
+    }
+    if (avgPersonByteSize != other.avgPersonByteSize) {
+      return false;
+    }
+    if (coderStrategy != other.coderStrategy) {
+      return false;
+    }
+    if (cpuDelayMs != other.cpuDelayMs) {
+      return false;
+    }
+    if (diskBusyBytes != other.diskBusyBytes) {
+      return false;
+    }
+    if (fanout != other.fanout) {
+      return false;
+    }
+    if (maxAuctionsWaitingTime != other.maxAuctionsWaitingTime) {
+      return false;
+    }
+    if (firstEventRate != other.firstEventRate) {
+      return false;
+    }
+    if (hotAuctionRatio != other.hotAuctionRatio) {
+      return false;
+    }
+    if (hotBiddersRatio != other.hotBiddersRatio) {
+      return false;
+    }
+    if (hotSellersRatio != other.hotSellersRatio) {
+      return false;
+    }
+    if (isRateLimited != other.isRateLimited) {
+      return false;
+    }
+    if (maxLogEvents != other.maxLogEvents) {
+      return false;
+    }
+    if (nextEventRate != other.nextEventRate) {
+      return false;
+    }
+    if (rateUnit != other.rateUnit) {
+      return false;
+    }
+    if (numEventGenerators != other.numEventGenerators) {
+      return false;
+    }
+    if (numEvents != other.numEvents) {
+      return false;
+    }
+    if (numInFlightAuctions != other.numInFlightAuctions) {
+      return false;
+    }
+    if (numActivePeople != other.numActivePeople) {
+      return false;
+    }
+    if (occasionalDelaySec != other.occasionalDelaySec) {
+      return false;
+    }
+    if (preloadSeconds != other.preloadSeconds) {
+      return false;
+    }
+    if (streamTimeout != other.streamTimeout) {
+      return false;
+    }
+    if (Double.doubleToLongBits(probDelayedEvent)
+        != Double.doubleToLongBits(other.probDelayedEvent)) {
+      return false;
+    }
+    if (pubSubMode != other.pubSubMode) {
+      return false;
+    }
+    if (ratePeriodSec != other.ratePeriodSec) {
+      return false;
+    }
+    if (rateShape != other.rateShape) {
+      return false;
+    }
+    if (query != other.query) {
+      return false;
+    }
+    if (sinkType != other.sinkType) {
+      return false;
+    }
+    if (sourceType != other.sourceType) {
+      return false;
+    }
+    if (useWallclockEventTime != other.useWallclockEventTime) {
+      return false;
+    }
+    if (watermarkHoldbackSec != other.watermarkHoldbackSec) {
+      return false;
+    }
+    if (windowPeriodSec != other.windowPeriodSec) {
+      return false;
+    }
+    if (windowSizeSec != other.windowSizeSec) {
+      return false;
+    }
+    if (usePubsubPublishTime != other.usePubsubPublishTime) {
+      return false;
+    }
+    if (outOfOrderGroupSize != other.outOfOrderGroupSize) {
+      return false;
+    }
+    return true;
+  }
+}


[50/55] [abbrv] beam git commit: Move module beam-integration-java-nexmark to beam-sdks-java-nexmark

Posted by ie...@apache.org.
http://git-wip-us.apache.org/repos/asf/beam/blob/f4333df7/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkUtils.java
----------------------------------------------------------------------
diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkUtils.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkUtils.java
deleted file mode 100644
index 7926690..0000000
--- a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkUtils.java
+++ /dev/null
@@ -1,672 +0,0 @@
-/*
- * 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.beam.integration.nexmark;
-
-import com.fasterxml.jackson.databind.ObjectMapper;
-import com.google.common.hash.Hashing;
-import java.io.IOException;
-import java.io.InputStream;
-import java.io.OutputStream;
-import java.nio.charset.StandardCharsets;
-import java.util.Iterator;
-import org.apache.beam.integration.nexmark.model.Auction;
-import org.apache.beam.integration.nexmark.model.AuctionBid;
-import org.apache.beam.integration.nexmark.model.AuctionCount;
-import org.apache.beam.integration.nexmark.model.AuctionPrice;
-import org.apache.beam.integration.nexmark.model.Bid;
-import org.apache.beam.integration.nexmark.model.BidsPerSession;
-import org.apache.beam.integration.nexmark.model.CategoryPrice;
-import org.apache.beam.integration.nexmark.model.Done;
-import org.apache.beam.integration.nexmark.model.Event;
-import org.apache.beam.integration.nexmark.model.IdNameReserve;
-import org.apache.beam.integration.nexmark.model.KnownSize;
-import org.apache.beam.integration.nexmark.model.NameCityStateId;
-import org.apache.beam.integration.nexmark.model.Person;
-import org.apache.beam.integration.nexmark.model.SellerPrice;
-import org.apache.beam.integration.nexmark.sources.BoundedEventSource;
-import org.apache.beam.integration.nexmark.sources.Generator;
-import org.apache.beam.integration.nexmark.sources.GeneratorConfig;
-import org.apache.beam.integration.nexmark.sources.UnboundedEventSource;
-import org.apache.beam.sdk.Pipeline;
-import org.apache.beam.sdk.coders.AvroCoder;
-import org.apache.beam.sdk.coders.ByteArrayCoder;
-import org.apache.beam.sdk.coders.Coder;
-import org.apache.beam.sdk.coders.CoderException;
-import org.apache.beam.sdk.coders.CoderRegistry;
-import org.apache.beam.sdk.coders.CustomCoder;
-import org.apache.beam.sdk.coders.SerializableCoder;
-import org.apache.beam.sdk.io.Read;
-import org.apache.beam.sdk.metrics.Counter;
-import org.apache.beam.sdk.metrics.Metrics;
-import org.apache.beam.sdk.state.StateSpec;
-import org.apache.beam.sdk.state.StateSpecs;
-import org.apache.beam.sdk.state.ValueState;
-import org.apache.beam.sdk.transforms.Combine;
-import org.apache.beam.sdk.transforms.DoFn;
-import org.apache.beam.sdk.transforms.PTransform;
-import org.apache.beam.sdk.transforms.ParDo;
-import org.apache.beam.sdk.transforms.windowing.AfterPane;
-import org.apache.beam.sdk.transforms.windowing.BoundedWindow;
-import org.apache.beam.sdk.transforms.windowing.GlobalWindows;
-import org.apache.beam.sdk.transforms.windowing.Window;
-import org.apache.beam.sdk.values.KV;
-import org.apache.beam.sdk.values.PBegin;
-import org.apache.beam.sdk.values.PCollection;
-import org.apache.beam.sdk.values.TimestampedValue;
-import org.joda.time.Duration;
-import org.joda.time.Instant;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-/**
- * Odd's 'n Ends used throughout queries and driver.
- */
-public class NexmarkUtils {
-  private static final Logger LOG = LoggerFactory.getLogger(NexmarkUtils.class);
-
-  /**
-   * Mapper for (de)serializing JSON.
-   */
-  public static final ObjectMapper MAPPER = new ObjectMapper();
-
-  /**
-   * Possible sources for events.
-   */
-  public enum SourceType {
-    /**
-     * Produce events directly.
-     */
-    DIRECT,
-    /**
-     * Read events from an Avro file.
-     */
-    AVRO,
-    /**
-     * Read from a PubSub topic. It will be fed the same synthetic events by this pipeline.
-     */
-    PUBSUB
-  }
-
-  /**
-   * Possible sinks for query results.
-   */
-  public enum SinkType {
-    /**
-     * Discard all results.
-     */
-    COUNT_ONLY,
-    /**
-     * Discard all results after converting them to strings.
-     */
-    DEVNULL,
-    /**
-     * Write to a PubSub topic. It will be drained by this pipeline.
-     */
-    PUBSUB,
-    /**
-     * Write to a text file. Only works in batch mode.
-     */
-    TEXT,
-    /**
-     * Write raw Events to Avro. Only works in batch mode.
-     */
-    AVRO,
-    /**
-     * Write raw Events to BigQuery.
-     */
-    BIGQUERY,
-  }
-
-  /**
-   * Pub/sub mode to run in.
-   */
-  public enum PubSubMode {
-    /**
-     * Publish events to pub/sub, but don't run the query.
-     */
-    PUBLISH_ONLY,
-    /**
-     * Consume events from pub/sub and run the query, but don't publish.
-     */
-    SUBSCRIBE_ONLY,
-    /**
-     * Both publish and consume, but as separate jobs.
-     */
-    COMBINED
-  }
-
-  /**
-   * Coder strategies.
-   */
-  public enum CoderStrategy {
-    /**
-     * Hand-written.
-     */
-    HAND,
-    /**
-     * Avro.
-     */
-    AVRO,
-    /**
-     * Java serialization.
-     */
-    JAVA
-  }
-
-  /**
-   * How to determine resource names.
-   */
-  public enum ResourceNameMode {
-    /** Names are used as provided. */
-    VERBATIM,
-    /** Names are suffixed with the query being run. */
-    QUERY,
-    /** Names are suffixed with the query being run and a random number. */
-    QUERY_AND_SALT
-  }
-
-  /**
-   * Units for rates.
-   */
-  public enum RateUnit {
-    PER_SECOND(1_000_000L),
-    PER_MINUTE(60_000_000L);
-
-    RateUnit(long usPerUnit) {
-      this.usPerUnit = usPerUnit;
-    }
-
-    /**
-     * Number of microseconds per unit.
-     */
-    private final long usPerUnit;
-
-    /**
-     * Number of microseconds between events at given rate.
-     */
-    public long rateToPeriodUs(long rate) {
-      return (usPerUnit + rate / 2) / rate;
-    }
-  }
-
-  /**
-   * Shape of event rate.
-   */
-  public enum RateShape {
-    SQUARE,
-    SINE;
-
-    /**
-     * Number of steps used to approximate sine wave.
-     */
-    private static final int N = 10;
-
-    /**
-     * Return inter-event delay, in microseconds, for each generator
-     * to follow in order to achieve {@code rate} at {@code unit} using {@code numGenerators}.
-     */
-    public long interEventDelayUs(int rate, RateUnit unit, int numGenerators) {
-      return unit.rateToPeriodUs(rate) * numGenerators;
-    }
-
-    /**
-     * Return array of successive inter-event delays, in microseconds, for each generator
-     * to follow in order to achieve this shape with {@code firstRate/nextRate} at
-     * {@code unit} using {@code numGenerators}.
-     */
-    public long[] interEventDelayUs(
-        int firstRate, int nextRate, RateUnit unit, int numGenerators) {
-      if (firstRate == nextRate) {
-        long[] interEventDelayUs = new long[1];
-        interEventDelayUs[0] = unit.rateToPeriodUs(firstRate) * numGenerators;
-        return interEventDelayUs;
-      }
-
-      switch (this) {
-        case SQUARE: {
-          long[] interEventDelayUs = new long[2];
-          interEventDelayUs[0] = unit.rateToPeriodUs(firstRate) * numGenerators;
-          interEventDelayUs[1] = unit.rateToPeriodUs(nextRate) * numGenerators;
-          return interEventDelayUs;
-        }
-        case SINE: {
-          double mid = (firstRate + nextRate) / 2.0;
-          double amp = (firstRate - nextRate) / 2.0; // may be -ve
-          long[] interEventDelayUs = new long[N];
-          for (int i = 0; i < N; i++) {
-            double r = (2.0 * Math.PI * i) / N;
-            double rate = mid + amp * Math.cos(r);
-            interEventDelayUs[i] = unit.rateToPeriodUs(Math.round(rate)) * numGenerators;
-          }
-          return interEventDelayUs;
-        }
-      }
-      throw new RuntimeException(); // switch should be exhaustive
-    }
-
-    /**
-     * Return delay between steps, in seconds, for result of {@link #interEventDelayUs}, so
-     * as to cycle through the entire sequence every {@code ratePeriodSec}.
-     */
-    public int stepLengthSec(int ratePeriodSec) {
-      int n = 0;
-      switch (this) {
-        case SQUARE:
-          n = 2;
-          break;
-        case SINE:
-          n = N;
-          break;
-      }
-      return (ratePeriodSec + n - 1) / n;
-    }
-  }
-
-  /**
-   * Set to true to capture all info messages. The logging level flags don't currently work.
-   */
-  private static final boolean LOG_INFO = false;
-
-  /**
-   * Set to true to capture all error messages. The logging level flags don't currently work.
-   */
-  private static final boolean LOG_ERROR = true;
-
-  /**
-   * Set to true to log directly to stdout. If run using Google Dataflow, you can watch the results
-   * in real-time with: tail -f /var/log/dataflow/streaming-harness/harness-stdout.log
-   */
-  private static final boolean LOG_TO_CONSOLE = false;
-
-  /**
-   * Log info message.
-   */
-  public static void info(String format, Object... args) {
-    if (LOG_INFO) {
-      LOG.info(String.format(format, args));
-      if (LOG_TO_CONSOLE) {
-        System.out.println(String.format(format, args));
-      }
-    }
-  }
-
-  /**
-   * Log message to console. For client side only.
-   */
-  public static void console(String format, Object... args) {
-    System.out.printf("%s %s%n", Instant.now(), String.format(format, args));
-  }
-
-  /**
-   * Label to use for timestamps on pub/sub messages.
-   */
-  public static final String PUBSUB_TIMESTAMP = "timestamp";
-
-  /**
-   * Label to use for windmill ids on pub/sub messages.
-   */
-  public static final String PUBSUB_ID = "id";
-
-  /**
-   * All events will be given a timestamp relative to this time (ms since epoch).
-   */
-  private static final long BASE_TIME = Instant.parse("2015-07-15T00:00:00.000Z").getMillis();
-
-  /**
-   * Instants guaranteed to be strictly before and after all event timestamps, and which won't
-   * be subject to underflow/overflow.
-   */
-  public static final Instant BEGINNING_OF_TIME = new Instant(0).plus(Duration.standardDays(365));
-  public static final Instant END_OF_TIME =
-      BoundedWindow.TIMESTAMP_MAX_VALUE.minus(Duration.standardDays(365));
-
-  /**
-   * Setup pipeline with codes and some other options.
-   */
-  public static void setupPipeline(CoderStrategy coderStrategy, Pipeline p) {
-    CoderRegistry registry = p.getCoderRegistry();
-    switch (coderStrategy) {
-      case HAND:
-        registry.registerCoderForClass(Auction.class, Auction.CODER);
-        registry.registerCoderForClass(AuctionBid.class, AuctionBid.CODER);
-        registry.registerCoderForClass(AuctionCount.class, AuctionCount.CODER);
-        registry.registerCoderForClass(AuctionPrice.class, AuctionPrice.CODER);
-        registry.registerCoderForClass(Bid.class, Bid.CODER);
-        registry.registerCoderForClass(CategoryPrice.class, CategoryPrice.CODER);
-        registry.registerCoderForClass(Event.class, Event.CODER);
-        registry.registerCoderForClass(IdNameReserve.class, IdNameReserve.CODER);
-        registry.registerCoderForClass(NameCityStateId.class, NameCityStateId.CODER);
-        registry.registerCoderForClass(Person.class, Person.CODER);
-        registry.registerCoderForClass(SellerPrice.class, SellerPrice.CODER);
-        registry.registerCoderForClass(Done.class, Done.CODER);
-        registry.registerCoderForClass(BidsPerSession.class, BidsPerSession.CODER);
-        break;
-      case AVRO:
-        registry.registerCoderProvider(AvroCoder.getCoderProvider());
-        break;
-      case JAVA:
-        registry.registerCoderProvider(SerializableCoder.getCoderProvider());
-        break;
-    }
-  }
-
-  /**
-   * Return a generator config to match the given {@code options}.
-   */
-  private static GeneratorConfig standardGeneratorConfig(NexmarkConfiguration configuration) {
-    return new GeneratorConfig(configuration,
-                               configuration.useWallclockEventTime ? System.currentTimeMillis()
-                                                                   : BASE_TIME, 0,
-                               configuration.numEvents, 0);
-  }
-
-  /**
-   * Return an iterator of events using the 'standard' generator config.
-   */
-  public static Iterator<TimestampedValue<Event>> standardEventIterator(
-      NexmarkConfiguration configuration) {
-    return new Generator(standardGeneratorConfig(configuration));
-  }
-
-  /**
-   * Return a transform which yields a finite number of synthesized events generated
-   * as a batch.
-   */
-  public static PTransform<PBegin, PCollection<Event>> batchEventsSource(
-          NexmarkConfiguration configuration) {
-    return Read.from(new BoundedEventSource(standardGeneratorConfig(configuration),
-      configuration.numEventGenerators));
-  }
-
-  /**
-   * Return a transform which yields a finite number of synthesized events generated
-   * on-the-fly in real time.
-   */
-  public static PTransform<PBegin, PCollection<Event>> streamEventsSource(
-          NexmarkConfiguration configuration) {
-    return Read.from(new UnboundedEventSource(NexmarkUtils.standardGeneratorConfig(configuration),
-                                              configuration.numEventGenerators,
-                                              configuration.watermarkHoldbackSec,
-                                              configuration.isRateLimited));
-  }
-
-  /**
-   * Return a transform to pass-through events, but count them as they go by.
-   */
-  public static ParDo.SingleOutput<Event, Event> snoop(final String name) {
-    return ParDo.of(new DoFn<Event, Event>() {
-      final Counter eventCounter = Metrics.counter(name, "events");
-      final Counter newPersonCounter = Metrics.counter(name, "newPersons");
-      final Counter newAuctionCounter = Metrics.counter(name, "newAuctions");
-      final Counter bidCounter = Metrics.counter(name, "bids");
-      final Counter endOfStreamCounter = Metrics.counter(name, "endOfStream");
-
-      @ProcessElement
-      public void processElement(ProcessContext c) {
-        eventCounter.inc();
-        if (c.element().newPerson != null) {
-          newPersonCounter.inc();
-        } else if (c.element().newAuction != null) {
-          newAuctionCounter.inc();
-        } else if (c.element().bid != null) {
-          bidCounter.inc();
-        } else {
-          endOfStreamCounter.inc();
-        }
-        info("%s snooping element %s", name, c.element());
-        c.output(c.element());
-      }
-    });
-  }
-
-  /**
-   * Return a transform to count and discard each element.
-   */
-  public static <T> ParDo.SingleOutput<T, Void> devNull(final String name) {
-    return ParDo.of(new DoFn<T, Void>() {
-      final Counter discardedCounterMetric = Metrics.counter(name, "discarded");
-
-      @ProcessElement
-      public void processElement(ProcessContext c) {
-        discardedCounterMetric.inc();
-      }
-    });
-  }
-
-  /**
-   * Return a transform to log each element, passing it through unchanged.
-   */
-  public static <T> ParDo.SingleOutput<T, T> log(final String name) {
-    return ParDo.of(new DoFn<T, T>() {
-      @ProcessElement
-      public void processElement(ProcessContext c) {
-        LOG.info("%s: %s", name, c.element());
-        c.output(c.element());
-      }
-    });
-  }
-
-  /**
-   * Return a transform to format each element as a string.
-   */
-  public static <T> ParDo.SingleOutput<T, String> format(final String name) {
-    return ParDo.of(new DoFn<T, String>() {
-      final Counter recordCounterMetric = Metrics.counter(name, "records");
-
-      @ProcessElement
-      public void processElement(ProcessContext c) {
-        recordCounterMetric.inc();
-        c.output(c.element().toString());
-      }
-    });
-  }
-
-  /**
-   * Return a transform to make explicit the timestamp of each element.
-   */
-  public static <T> ParDo.SingleOutput<T, TimestampedValue<T>> stamp(String name) {
-    return ParDo.of(new DoFn<T, TimestampedValue<T>>() {
-      @ProcessElement
-      public void processElement(ProcessContext c) {
-        c.output(TimestampedValue.of(c.element(), c.timestamp()));
-      }
-    });
-  }
-
-  /**
-   * Return a transform to reduce a stream to a single, order-invariant long hash.
-   */
-  public static <T> PTransform<PCollection<T>, PCollection<Long>> hash(
-      final long numEvents, String name) {
-    return new PTransform<PCollection<T>, PCollection<Long>>(name) {
-      @Override
-      public PCollection<Long> expand(PCollection<T> input) {
-        return input.apply(Window.<T>into(new GlobalWindows())
-                               .triggering(AfterPane.elementCountAtLeast((int) numEvents))
-                               .withAllowedLateness(Duration.standardDays(1))
-                               .discardingFiredPanes())
-
-                    .apply(name + ".Hash", ParDo.of(new DoFn<T, Long>() {
-                      @ProcessElement
-                      public void processElement(ProcessContext c) {
-                        long hash =
-                            Hashing.murmur3_128()
-                                   .newHasher()
-                                   .putLong(c.timestamp().getMillis())
-                                   .putString(c.element().toString(), StandardCharsets.UTF_8)
-                                   .hash()
-                                   .asLong();
-                        c.output(hash);
-                      }
-                    }))
-
-                    .apply(Combine.globally(new Combine.BinaryCombineFn<Long>() {
-                      @Override
-                      public Long apply(Long left, Long right) {
-                        return left ^ right;
-                      }
-                    }));
-      }
-    };
-  }
-
-  private static final long MASK = (1L << 16) - 1L;
-  private static final long HASH = 0x243F6A8885A308D3L;
-  private static final long INIT_PLAINTEXT = 50000L;
-
-  /**
-   * Return a transform to keep the CPU busy for given milliseconds on every record.
-   */
-  public static <T> ParDo.SingleOutput<T, T> cpuDelay(String name, final long delayMs) {
-    return ParDo.of(new DoFn<T, T>() {
-                  @ProcessElement
-                  public void processElement(ProcessContext c) {
-                    long now = System.currentTimeMillis();
-                    long end = now + delayMs;
-                    while (now < end) {
-                      // Find plaintext which hashes to HASH in lowest MASK bits.
-                      // Values chosen to roughly take 1ms on typical workstation.
-                      long p = INIT_PLAINTEXT;
-                      while (true) {
-                        long t = Hashing.murmur3_128().hashLong(p).asLong();
-                        if ((t & MASK) == (HASH & MASK)) {
-                          break;
-                        }
-                        p++;
-                      }
-                      now = System.currentTimeMillis();
-                    }
-                    c.output(c.element());
-                  }
-                });
-  }
-
-  private static final int MAX_BUFFER_SIZE = 1 << 24;
-
-  private static class DiskBusyTransform<T> extends PTransform<PCollection<T>, PCollection<T>>{
-
-    private long bytes;
-
-    private DiskBusyTransform(long bytes) {
-      this.bytes = bytes;
-    }
-
-    @Override public PCollection<T> expand(PCollection<T> input) {
-      // Add dummy key to be able to use State API
-      PCollection<KV<Integer, T>> kvCollection = input.apply("diskBusy.keyElements", ParDo.of(new DoFn<T, KV<Integer, T>>() {
-
-        @ProcessElement public void processElement(ProcessContext context) {
-          context.output(KV.of(0, context.element()));
-        }
-      }));
-      // Apply actual transform that generates disk IO using state API
-      PCollection<T> output = kvCollection.apply("diskBusy.generateIO", ParDo.of(new DoFn<KV<Integer, T>, T>() {
-
-        private static final String DISK_BUSY = "diskBusy";
-
-        @StateId(DISK_BUSY) private final StateSpec<ValueState<byte[]>> spec = StateSpecs
-            .value(ByteArrayCoder.of());
-
-        @ProcessElement public void processElement(ProcessContext c,
-            @StateId(DISK_BUSY) ValueState<byte[]> state) {
-          long remain = bytes;
-          long now = System.currentTimeMillis();
-          while (remain > 0) {
-            long thisBytes = Math.min(remain, MAX_BUFFER_SIZE);
-            remain -= thisBytes;
-            byte[] arr = new byte[(int) thisBytes];
-            for (int i = 0; i < thisBytes; i++) {
-              arr[i] = (byte) now;
-            }
-            state.write(arr);
-            now = System.currentTimeMillis();
-          }
-          c.output(c.element().getValue());
-        }
-      }));
-      return output;
-    }
-  }
-
-
-  /**
-   * Return a transform to write given number of bytes to durable store on every record.
-   */
-  public static <T> PTransform<PCollection<T>, PCollection<T>> diskBusy(final long bytes) {
-    return new DiskBusyTransform<>(bytes);
-  }
-
-  /**
-   * Return a transform to cast each element to {@link KnownSize}.
-   */
-  private static <T extends KnownSize> ParDo.SingleOutput<T, KnownSize> castToKnownSize() {
-    return ParDo.of(new DoFn<T, KnownSize>() {
-                  @ProcessElement
-                  public void processElement(ProcessContext c) {
-                    c.output(c.element());
-                  }
-                });
-  }
-
-  /**
-   * A coder for instances of {@code T} cast up to {@link KnownSize}.
-   *
-   * @param <T> True type of object.
-   */
-  private static class CastingCoder<T extends KnownSize> extends CustomCoder<KnownSize> {
-    private final Coder<T> trueCoder;
-
-    public CastingCoder(Coder<T> trueCoder) {
-      this.trueCoder = trueCoder;
-    }
-
-    @Override
-    public void encode(KnownSize value, OutputStream outStream)
-        throws CoderException, IOException {
-      @SuppressWarnings("unchecked")
-      T typedValue = (T) value;
-      trueCoder.encode(typedValue, outStream);
-    }
-
-    @Override
-    public KnownSize decode(InputStream inStream)
-        throws CoderException, IOException {
-      return trueCoder.decode(inStream);
-    }
-  }
-
-  /**
-   * Return a coder for {@code KnownSize} that are known to be exactly of type {@code T}.
-   */
-  private static <T extends KnownSize> Coder<KnownSize> makeCastingCoder(Coder<T> trueCoder) {
-    return new CastingCoder<>(trueCoder);
-  }
-
-  /**
-   * Return {@code elements} as {@code KnownSize}s.
-   */
-  public static <T extends KnownSize> PCollection<KnownSize> castToKnownSize(
-      final String name, PCollection<T> elements) {
-    return elements.apply(name + ".Forget", castToKnownSize())
-            .setCoder(makeCastingCoder(elements.getCoder()));
-  }
-
-  // Do not instantiate.
-  private NexmarkUtils() {
-  }
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/f4333df7/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/model/Auction.java
----------------------------------------------------------------------
diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/model/Auction.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/model/Auction.java
deleted file mode 100644
index 9f5d7c0..0000000
--- a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/model/Auction.java
+++ /dev/null
@@ -1,187 +0,0 @@
-/*
- * 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.beam.integration.nexmark.model;
-
-import com.fasterxml.jackson.annotation.JsonProperty;
-import com.fasterxml.jackson.core.JsonProcessingException;
-import java.io.IOException;
-import java.io.InputStream;
-import java.io.OutputStream;
-import java.io.Serializable;
-import org.apache.beam.integration.nexmark.NexmarkUtils;
-import org.apache.beam.sdk.coders.Coder;
-import org.apache.beam.sdk.coders.CoderException;
-import org.apache.beam.sdk.coders.CustomCoder;
-import org.apache.beam.sdk.coders.StringUtf8Coder;
-import org.apache.beam.sdk.coders.VarLongCoder;
-
-/**
- * An auction submitted by a person.
- */
-public class Auction implements KnownSize, Serializable {
-  private static final Coder<Long> LONG_CODER = VarLongCoder.of();
-  private static final Coder<String> STRING_CODER = StringUtf8Coder.of();
-
-  public static final Coder<Auction> CODER = new CustomCoder<Auction>() {
-    @Override
-    public void encode(Auction value, OutputStream outStream)
-        throws CoderException, IOException {
-      LONG_CODER.encode(value.id, outStream);
-      STRING_CODER.encode(value.itemName, outStream);
-      STRING_CODER.encode(value.description, outStream);
-      LONG_CODER.encode(value.initialBid, outStream);
-      LONG_CODER.encode(value.reserve, outStream);
-      LONG_CODER.encode(value.dateTime, outStream);
-      LONG_CODER.encode(value.expires, outStream);
-      LONG_CODER.encode(value.seller, outStream);
-      LONG_CODER.encode(value.category, outStream);
-      STRING_CODER.encode(value.extra, outStream);
-    }
-
-    @Override
-    public Auction decode(
-        InputStream inStream)
-        throws CoderException, IOException {
-      long id = LONG_CODER.decode(inStream);
-      String itemName = STRING_CODER.decode(inStream);
-      String description = STRING_CODER.decode(inStream);
-      long initialBid = LONG_CODER.decode(inStream);
-      long reserve = LONG_CODER.decode(inStream);
-      long dateTime = LONG_CODER.decode(inStream);
-      long expires = LONG_CODER.decode(inStream);
-      long seller = LONG_CODER.decode(inStream);
-      long category = LONG_CODER.decode(inStream);
-      String extra = STRING_CODER.decode(inStream);
-      return new Auction(
-          id, itemName, description, initialBid, reserve, dateTime, expires, seller, category,
-          extra);
-    }
-  };
-
-
-  /** Id of auction. */
-  @JsonProperty
-  public final long id; // primary key
-
-  /** Extra auction properties. */
-  @JsonProperty
-  private final String itemName;
-
-  @JsonProperty
-  private final String description;
-
-  /** Initial bid price, in cents. */
-  @JsonProperty
-  private final long initialBid;
-
-  /** Reserve price, in cents. */
-  @JsonProperty
-  public final long reserve;
-
-  @JsonProperty
-  public final long dateTime;
-
-  /** When does auction expire? (ms since epoch). Bids at or after this time are ignored. */
-  @JsonProperty
-  public final long expires;
-
-  /** Id of person who instigated auction. */
-  @JsonProperty
-  public final long seller; // foreign key: Person.id
-
-  /** Id of category auction is listed under. */
-  @JsonProperty
-  public final long category; // foreign key: Category.id
-
-  /** Additional arbitrary payload for performance testing. */
-  @JsonProperty
-  private final String extra;
-
-
-  // For Avro only.
-  @SuppressWarnings("unused")
-  private Auction() {
-    id = 0;
-    itemName = null;
-    description = null;
-    initialBid = 0;
-    reserve = 0;
-    dateTime = 0;
-    expires = 0;
-    seller = 0;
-    category = 0;
-    extra = null;
-  }
-
-  public Auction(long id, String itemName, String description, long initialBid, long reserve,
-      long dateTime, long expires, long seller, long category, String extra) {
-    this.id = id;
-    this.itemName = itemName;
-    this.description = description;
-    this.initialBid = initialBid;
-    this.reserve = reserve;
-    this.dateTime = dateTime;
-    this.expires = expires;
-    this.seller = seller;
-    this.category = category;
-    this.extra = extra;
-  }
-
-  /**
-   * Return a copy of auction which capture the given annotation.
-   * (Used for debugging).
-   */
-  public Auction withAnnotation(String annotation) {
-    return new Auction(id, itemName, description, initialBid, reserve, dateTime, expires, seller,
-        category, annotation + ": " + extra);
-  }
-
-  /**
-   * Does auction have {@code annotation}? (Used for debugging.)
-   */
-  public boolean hasAnnotation(String annotation) {
-    return extra.startsWith(annotation + ": ");
-  }
-
-  /**
-   * Remove {@code annotation} from auction. (Used for debugging.)
-   */
-  public Auction withoutAnnotation(String annotation) {
-    if (hasAnnotation(annotation)) {
-      return new Auction(id, itemName, description, initialBid, reserve, dateTime, expires, seller,
-          category, extra.substring(annotation.length() + 2));
-    } else {
-      return this;
-    }
-  }
-
-  @Override
-  public long sizeInBytes() {
-    return 8 + itemName.length() + 1 + description.length() + 1 + 8 + 8 + 8 + 8 + 8 + 8
-        + extra.length() + 1;
-  }
-
-  @Override
-  public String toString() {
-    try {
-      return NexmarkUtils.MAPPER.writeValueAsString(this);
-    } catch (JsonProcessingException e) {
-      throw new RuntimeException(e);
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/f4333df7/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/model/AuctionBid.java
----------------------------------------------------------------------
diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/model/AuctionBid.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/model/AuctionBid.java
deleted file mode 100644
index b9d79db..0000000
--- a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/model/AuctionBid.java
+++ /dev/null
@@ -1,84 +0,0 @@
-/*
- * 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.beam.integration.nexmark.model;
-
-import com.fasterxml.jackson.annotation.JsonProperty;
-import com.fasterxml.jackson.core.JsonProcessingException;
-import java.io.IOException;
-import java.io.InputStream;
-import java.io.OutputStream;
-import java.io.Serializable;
-import org.apache.beam.integration.nexmark.NexmarkUtils;
-import org.apache.beam.sdk.coders.Coder;
-import org.apache.beam.sdk.coders.CoderException;
-import org.apache.beam.sdk.coders.CustomCoder;
-
-/**
- * Result of {@link org.apache.beam.integration.nexmark.queries.WinningBids} transform.
- */
-public class AuctionBid implements KnownSize, Serializable {
-  public static final Coder<AuctionBid> CODER = new CustomCoder<AuctionBid>() {
-    @Override
-    public void encode(AuctionBid value, OutputStream outStream)
-        throws CoderException, IOException {
-      Auction.CODER.encode(value.auction, outStream);
-      Bid.CODER.encode(value.bid, outStream);
-    }
-
-    @Override
-    public AuctionBid decode(
-        InputStream inStream)
-        throws CoderException, IOException {
-      Auction auction = Auction.CODER.decode(inStream);
-      Bid bid = Bid.CODER.decode(inStream);
-      return new AuctionBid(auction, bid);
-    }
-  };
-
-  @JsonProperty
-  public final Auction auction;
-
-  @JsonProperty
-  public final Bid bid;
-
-  // For Avro only.
-  @SuppressWarnings("unused")
-  private AuctionBid() {
-    auction = null;
-    bid = null;
-  }
-
-  public AuctionBid(Auction auction, Bid bid) {
-    this.auction = auction;
-    this.bid = bid;
-  }
-
-  @Override
-  public long sizeInBytes() {
-    return auction.sizeInBytes() + bid.sizeInBytes();
-  }
-
-  @Override
-  public String toString() {
-    try {
-      return NexmarkUtils.MAPPER.writeValueAsString(this);
-    } catch (JsonProcessingException e) {
-      throw new RuntimeException(e);
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/f4333df7/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/model/AuctionCount.java
----------------------------------------------------------------------
diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/model/AuctionCount.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/model/AuctionCount.java
deleted file mode 100644
index 0e643ff..0000000
--- a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/model/AuctionCount.java
+++ /dev/null
@@ -1,84 +0,0 @@
-/*
- * 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.beam.integration.nexmark.model;
-
-import com.fasterxml.jackson.annotation.JsonProperty;
-import com.fasterxml.jackson.core.JsonProcessingException;
-import java.io.IOException;
-import java.io.InputStream;
-import java.io.OutputStream;
-import java.io.Serializable;
-import org.apache.beam.integration.nexmark.NexmarkUtils;
-import org.apache.beam.sdk.coders.Coder;
-import org.apache.beam.sdk.coders.CoderException;
-import org.apache.beam.sdk.coders.CustomCoder;
-import org.apache.beam.sdk.coders.VarLongCoder;
-
-/**
- * Result of Query5.
- */
-public class AuctionCount implements KnownSize, Serializable {
-  private static final Coder<Long> LONG_CODER = VarLongCoder.of();
-
-  public static final Coder<AuctionCount> CODER = new CustomCoder<AuctionCount>() {
-    @Override
-    public void encode(AuctionCount value, OutputStream outStream)
-        throws CoderException, IOException {
-      LONG_CODER.encode(value.auction, outStream);
-      LONG_CODER.encode(value.count, outStream);
-    }
-
-    @Override
-    public AuctionCount decode(InputStream inStream)
-        throws CoderException, IOException {
-      long auction = LONG_CODER.decode(inStream);
-      long count = LONG_CODER.decode(inStream);
-      return new AuctionCount(auction, count);
-    }
-  };
-
-  @JsonProperty private final long auction;
-
-  @JsonProperty private final long count;
-
-  // For Avro only.
-  @SuppressWarnings("unused")
-  private AuctionCount() {
-    auction = 0;
-    count = 0;
-  }
-
-  public AuctionCount(long auction, long count) {
-    this.auction = auction;
-    this.count = count;
-  }
-
-  @Override
-  public long sizeInBytes() {
-    return 8 + 8;
-  }
-
-  @Override
-  public String toString() {
-    try {
-      return NexmarkUtils.MAPPER.writeValueAsString(this);
-    } catch (JsonProcessingException e) {
-      throw new RuntimeException(e);
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/f4333df7/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/model/AuctionPrice.java
----------------------------------------------------------------------
diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/model/AuctionPrice.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/model/AuctionPrice.java
deleted file mode 100644
index 7d51a21..0000000
--- a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/model/AuctionPrice.java
+++ /dev/null
@@ -1,88 +0,0 @@
-/*
- * 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.beam.integration.nexmark.model;
-
-import com.fasterxml.jackson.annotation.JsonProperty;
-import com.fasterxml.jackson.core.JsonProcessingException;
-import java.io.IOException;
-import java.io.InputStream;
-import java.io.OutputStream;
-import java.io.Serializable;
-import org.apache.beam.integration.nexmark.NexmarkUtils;
-import org.apache.beam.sdk.coders.Coder;
-import org.apache.beam.sdk.coders.CoderException;
-import org.apache.beam.sdk.coders.CustomCoder;
-import org.apache.beam.sdk.coders.VarLongCoder;
-
-/**
- * Result of Query2.
- */
-public class AuctionPrice implements KnownSize, Serializable {
-  private static final Coder<Long> LONG_CODER = VarLongCoder.of();
-
-  public static final Coder<AuctionPrice> CODER = new CustomCoder<AuctionPrice>() {
-    @Override
-    public void encode(AuctionPrice value, OutputStream outStream)
-        throws CoderException, IOException {
-      LONG_CODER.encode(value.auction, outStream);
-      LONG_CODER.encode(value.price, outStream);
-    }
-
-    @Override
-    public AuctionPrice decode(
-        InputStream inStream)
-        throws CoderException, IOException {
-      long auction = LONG_CODER.decode(inStream);
-      long price = LONG_CODER.decode(inStream);
-      return new AuctionPrice(auction, price);
-    }
-  };
-
-  @JsonProperty
-  private final long auction;
-
-  /** Price in cents. */
-  @JsonProperty
-  private final long price;
-
-  // For Avro only.
-  @SuppressWarnings("unused")
-  private AuctionPrice() {
-    auction = 0;
-    price = 0;
-  }
-
-  public AuctionPrice(long auction, long price) {
-    this.auction = auction;
-    this.price = price;
-  }
-
-  @Override
-  public long sizeInBytes() {
-    return 8 + 8;
-  }
-
-  @Override
-  public String toString() {
-    try {
-      return NexmarkUtils.MAPPER.writeValueAsString(this);
-    } catch (JsonProcessingException e) {
-      throw new RuntimeException(e);
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/f4333df7/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/model/Bid.java
----------------------------------------------------------------------
diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/model/Bid.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/model/Bid.java
deleted file mode 100644
index 4fa9ea0..0000000
--- a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/model/Bid.java
+++ /dev/null
@@ -1,177 +0,0 @@
-/*
- * 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.beam.integration.nexmark.model;
-
-import com.fasterxml.jackson.annotation.JsonProperty;
-import com.fasterxml.jackson.core.JsonProcessingException;
-import java.io.IOException;
-import java.io.InputStream;
-import java.io.OutputStream;
-import java.io.Serializable;
-import java.util.Comparator;
-import org.apache.beam.integration.nexmark.NexmarkUtils;
-import org.apache.beam.sdk.coders.Coder;
-import org.apache.beam.sdk.coders.CoderException;
-import org.apache.beam.sdk.coders.CustomCoder;
-import org.apache.beam.sdk.coders.StringUtf8Coder;
-import org.apache.beam.sdk.coders.VarLongCoder;
-
-/**
- * A bid for an item on auction.
- */
-public class Bid implements KnownSize, Serializable {
-  private static final Coder<Long> LONG_CODER = VarLongCoder.of();
-  private static final Coder<String> STRING_CODER = StringUtf8Coder.of();
-
-  public static final Coder<Bid> CODER = new CustomCoder<Bid>() {
-    @Override
-    public void encode(Bid value, OutputStream outStream)
-        throws CoderException, IOException {
-      LONG_CODER.encode(value.auction, outStream);
-      LONG_CODER.encode(value.bidder, outStream);
-      LONG_CODER.encode(value.price, outStream);
-      LONG_CODER.encode(value.dateTime, outStream);
-      STRING_CODER.encode(value.extra, outStream);
-    }
-
-    @Override
-    public Bid decode(
-        InputStream inStream)
-        throws CoderException, IOException {
-      long auction = LONG_CODER.decode(inStream);
-      long bidder = LONG_CODER.decode(inStream);
-      long price = LONG_CODER.decode(inStream);
-      long dateTime = LONG_CODER.decode(inStream);
-      String extra = STRING_CODER.decode(inStream);
-      return new Bid(auction, bidder, price, dateTime, extra);
-    }
-
-    @Override public void verifyDeterministic() throws NonDeterministicException {}
-  };
-
-  /**
-   * Comparator to order bids by ascending price then descending time
-   * (for finding winning bids).
-   */
-  public static final Comparator<Bid> PRICE_THEN_DESCENDING_TIME = new Comparator<Bid>() {
-    @Override
-    public int compare(Bid left, Bid right) {
-      int i = Double.compare(left.price, right.price);
-      if (i != 0) {
-        return i;
-      }
-      return Long.compare(right.dateTime, left.dateTime);
-    }
-  };
-
-  /**
-   * Comparator to order bids by ascending time then ascending price.
-   * (for finding most recent bids).
-   */
-  public static final Comparator<Bid> ASCENDING_TIME_THEN_PRICE = new Comparator<Bid>() {
-    @Override
-    public int compare(Bid left, Bid right) {
-      int i = Long.compare(left.dateTime, right.dateTime);
-      if (i != 0) {
-        return i;
-      }
-      return Double.compare(left.price, right.price);
-    }
-  };
-
-  /** Id of auction this bid is for. */
-  @JsonProperty
-  public final long auction; // foreign key: Auction.id
-
-  /** Id of person bidding in auction. */
-  @JsonProperty
-  public final long bidder; // foreign key: Person.id
-
-  /** Price of bid, in cents. */
-  @JsonProperty
-  public final long price;
-
-  /**
-   * Instant at which bid was made (ms since epoch).
-   * NOTE: This may be earlier than the system's event time.
-   */
-  @JsonProperty
-  public final long dateTime;
-
-  /** Additional arbitrary payload for performance testing. */
-  @JsonProperty
-  public final String extra;
-
-  // For Avro only.
-  @SuppressWarnings("unused")
-  private Bid() {
-    auction = 0;
-    bidder = 0;
-    price = 0;
-    dateTime = 0;
-    extra = null;
-  }
-
-  public Bid(long auction, long bidder, long price, long dateTime, String extra) {
-    this.auction = auction;
-    this.bidder = bidder;
-    this.price = price;
-    this.dateTime = dateTime;
-    this.extra = extra;
-  }
-
-  /**
-   * Return a copy of bid which capture the given annotation.
-   * (Used for debugging).
-   */
-  public Bid withAnnotation(String annotation) {
-    return new Bid(auction, bidder, price, dateTime, annotation + ": " + extra);
-  }
-
-  /**
-   * Does bid have {@code annotation}? (Used for debugging.)
-   */
-  public boolean hasAnnotation(String annotation) {
-    return extra.startsWith(annotation + ": ");
-  }
-
-  /**
-   * Remove {@code annotation} from bid. (Used for debugging.)
-   */
-  public Bid withoutAnnotation(String annotation) {
-    if (hasAnnotation(annotation)) {
-      return new Bid(auction, bidder, price, dateTime, extra.substring(annotation.length() + 2));
-    } else {
-      return this;
-    }
-  }
-
-  @Override
-  public long sizeInBytes() {
-    return 8 + 8 + 8 + 8 + extra.length() + 1;
-  }
-
-  @Override
-  public String toString() {
-    try {
-      return NexmarkUtils.MAPPER.writeValueAsString(this);
-    } catch (JsonProcessingException e) {
-      throw new RuntimeException(e);
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/f4333df7/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/model/BidsPerSession.java
----------------------------------------------------------------------
diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/model/BidsPerSession.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/model/BidsPerSession.java
deleted file mode 100644
index 3211456..0000000
--- a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/model/BidsPerSession.java
+++ /dev/null
@@ -1,87 +0,0 @@
-/*
- * 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.beam.integration.nexmark.model;
-
-import com.fasterxml.jackson.annotation.JsonProperty;
-import com.fasterxml.jackson.core.JsonProcessingException;
-import java.io.IOException;
-import java.io.InputStream;
-import java.io.OutputStream;
-import java.io.Serializable;
-import org.apache.beam.integration.nexmark.NexmarkUtils;
-import org.apache.beam.sdk.coders.Coder;
-import org.apache.beam.sdk.coders.CoderException;
-import org.apache.beam.sdk.coders.CustomCoder;
-import org.apache.beam.sdk.coders.VarLongCoder;
-
-/**
- * Result of query 11.
- */
-public class BidsPerSession implements KnownSize, Serializable {
-  private static final Coder<Long> LONG_CODER = VarLongCoder.of();
-
-  public static final Coder<BidsPerSession> CODER = new CustomCoder<BidsPerSession>() {
-    @Override
-    public void encode(BidsPerSession value, OutputStream outStream)
-        throws CoderException, IOException {
-      LONG_CODER.encode(value.personId, outStream);
-      LONG_CODER.encode(value.bidsPerSession, outStream);
-    }
-
-    @Override
-    public BidsPerSession decode(
-        InputStream inStream)
-        throws CoderException, IOException {
-      long personId = LONG_CODER.decode(inStream);
-      long bidsPerSession = LONG_CODER.decode(inStream);
-      return new BidsPerSession(personId, bidsPerSession);
-    }
-    @Override public void verifyDeterministic() throws NonDeterministicException {}
-  };
-
-  @JsonProperty
-  private final long personId;
-
-  @JsonProperty
-  private final long bidsPerSession;
-
-  public BidsPerSession() {
-    personId = 0;
-    bidsPerSession = 0;
-  }
-
-  public BidsPerSession(long personId, long bidsPerSession) {
-    this.personId = personId;
-    this.bidsPerSession = bidsPerSession;
-  }
-
-  @Override
-  public long sizeInBytes() {
-    // Two longs.
-    return 8 + 8;
-  }
-
-  @Override
-  public String toString() {
-    try {
-      return NexmarkUtils.MAPPER.writeValueAsString(this);
-    } catch (JsonProcessingException e) {
-      throw new RuntimeException(e);
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/f4333df7/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/model/CategoryPrice.java
----------------------------------------------------------------------
diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/model/CategoryPrice.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/model/CategoryPrice.java
deleted file mode 100644
index 2678198..0000000
--- a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/model/CategoryPrice.java
+++ /dev/null
@@ -1,97 +0,0 @@
-/*
- * 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.beam.integration.nexmark.model;
-
-import com.fasterxml.jackson.annotation.JsonProperty;
-import com.fasterxml.jackson.core.JsonProcessingException;
-import java.io.IOException;
-import java.io.InputStream;
-import java.io.OutputStream;
-import java.io.Serializable;
-import org.apache.beam.integration.nexmark.NexmarkUtils;
-import org.apache.beam.sdk.coders.Coder;
-import org.apache.beam.sdk.coders.CoderException;
-import org.apache.beam.sdk.coders.CustomCoder;
-import org.apache.beam.sdk.coders.VarIntCoder;
-import org.apache.beam.sdk.coders.VarLongCoder;
-
-/**
- * Result of Query4.
- */
-public class CategoryPrice implements KnownSize, Serializable {
-  private static final Coder<Long> LONG_CODER = VarLongCoder.of();
-  private static final Coder<Integer> INT_CODER = VarIntCoder.of();
-
-  public static final Coder<CategoryPrice> CODER = new CustomCoder<CategoryPrice>() {
-    @Override
-    public void encode(CategoryPrice value, OutputStream outStream)
-        throws CoderException, IOException {
-      LONG_CODER.encode(value.category, outStream);
-      LONG_CODER.encode(value.price, outStream);
-      INT_CODER.encode(value.isLast ? 1 : 0, outStream);
-    }
-
-    @Override
-    public CategoryPrice decode(InputStream inStream)
-        throws CoderException, IOException {
-      long category = LONG_CODER.decode(inStream);
-      long price = LONG_CODER.decode(inStream);
-      boolean isLast = INT_CODER.decode(inStream) != 0;
-      return new CategoryPrice(category, price, isLast);
-    }
-    @Override public void verifyDeterministic() throws NonDeterministicException {}
-  };
-
-  @JsonProperty
-  public final long category;
-
-  /** Price in cents. */
-  @JsonProperty
-  public final long price;
-
-  @JsonProperty
-  public final boolean isLast;
-
-  // For Avro only.
-  @SuppressWarnings("unused")
-  private CategoryPrice() {
-    category = 0;
-    price = 0;
-    isLast = false;
-  }
-
-  public CategoryPrice(long category, long price, boolean isLast) {
-    this.category = category;
-    this.price = price;
-    this.isLast = isLast;
-  }
-
-  @Override
-  public long sizeInBytes() {
-    return 8 + 8 + 1;
-  }
-
-  @Override
-  public String toString() {
-    try {
-      return NexmarkUtils.MAPPER.writeValueAsString(this);
-    } catch (JsonProcessingException e) {
-      throw new RuntimeException(e);
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/f4333df7/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/model/Done.java
----------------------------------------------------------------------
diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/model/Done.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/model/Done.java
deleted file mode 100644
index b0a88d4..0000000
--- a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/model/Done.java
+++ /dev/null
@@ -1,80 +0,0 @@
-/*
- * 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.beam.integration.nexmark.model;
-
-import com.fasterxml.jackson.annotation.JsonProperty;
-import com.fasterxml.jackson.core.JsonProcessingException;
-import java.io.IOException;
-import java.io.InputStream;
-import java.io.OutputStream;
-import java.io.Serializable;
-import org.apache.beam.integration.nexmark.NexmarkUtils;
-import org.apache.beam.sdk.coders.Coder;
-import org.apache.beam.sdk.coders.CoderException;
-import org.apache.beam.sdk.coders.CustomCoder;
-import org.apache.beam.sdk.coders.StringUtf8Coder;
-
-/**
- * Result of query 10.
- */
-public class Done implements KnownSize, Serializable {
-  private static final Coder<String> STRING_CODER = StringUtf8Coder.of();
-
-  public static final Coder<Done> CODER = new CustomCoder<Done>() {
-    @Override
-    public void encode(Done value, OutputStream outStream)
-        throws CoderException, IOException {
-      STRING_CODER.encode(value.message, outStream);
-    }
-
-    @Override
-    public Done decode(InputStream inStream)
-        throws CoderException, IOException {
-      String message = STRING_CODER.decode(inStream);
-      return new Done(message);
-    }
-    @Override public void verifyDeterministic() throws NonDeterministicException {}
-  };
-
-  @JsonProperty
-  private final String message;
-
-  // For Avro only.
-  @SuppressWarnings("unused")
-  public Done() {
-    message = null;
-  }
-
-  public Done(String message) {
-    this.message = message;
-  }
-
-  @Override
-  public long sizeInBytes() {
-    return message.length();
-  }
-
-  @Override
-  public String toString() {
-    try {
-      return NexmarkUtils.MAPPER.writeValueAsString(this);
-    } catch (JsonProcessingException e) {
-      throw new RuntimeException(e);
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/f4333df7/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/model/Event.java
----------------------------------------------------------------------
diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/model/Event.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/model/Event.java
deleted file mode 100644
index 0e1672e..0000000
--- a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/model/Event.java
+++ /dev/null
@@ -1,171 +0,0 @@
-/*
- * 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.beam.integration.nexmark.model;
-
-import java.io.IOException;
-import java.io.InputStream;
-import java.io.OutputStream;
-import java.io.Serializable;
-import javax.annotation.Nullable;
-import org.apache.beam.sdk.coders.Coder;
-import org.apache.beam.sdk.coders.CustomCoder;
-import org.apache.beam.sdk.coders.VarIntCoder;
-
-/**
- * An event in the auction system, either a (new) {@link Person}, a (new) {@link Auction}, or a
- * {@link Bid}.
- */
-public class Event implements KnownSize, Serializable {
-  private enum Tag {
-    PERSON(0),
-    AUCTION(1),
-    BID(2);
-
-    private int value = -1;
-
-    Tag(int value){
-      this.value = value;
-    }
-  }
-  private static final Coder<Integer> INT_CODER = VarIntCoder.of();
-
-  public static final Coder<Event> CODER =
-      new CustomCoder<Event>() {
-        @Override
-        public void encode(Event value, OutputStream outStream) throws IOException {
-          if (value.newPerson != null) {
-            INT_CODER.encode(Tag.PERSON.value, outStream);
-            Person.CODER.encode(value.newPerson, outStream);
-          } else if (value.newAuction != null) {
-            INT_CODER.encode(Tag.AUCTION.value, outStream);
-            Auction.CODER.encode(value.newAuction, outStream);
-          } else if (value.bid != null) {
-            INT_CODER.encode(Tag.BID.value, outStream);
-            Bid.CODER.encode(value.bid, outStream);
-          } else {
-            throw new RuntimeException("invalid event");
-          }
-        }
-
-        @Override
-        public Event decode(InputStream inStream) throws IOException {
-          int tag = INT_CODER.decode(inStream);
-          if (tag == Tag.PERSON.value) {
-            Person person = Person.CODER.decode(inStream);
-            return new Event(person);
-          } else if (tag == Tag.AUCTION.value) {
-            Auction auction = Auction.CODER.decode(inStream);
-            return new Event(auction);
-          } else if (tag == Tag.BID.value) {
-            Bid bid = Bid.CODER.decode(inStream);
-            return new Event(bid);
-          } else {
-            throw new RuntimeException("invalid event encoding");
-          }
-        }
-
-        @Override
-        public void verifyDeterministic() throws NonDeterministicException {}
-      };
-
-  @Nullable
-  @org.apache.avro.reflect.Nullable
-  public final Person newPerson;
-
-  @Nullable
-  @org.apache.avro.reflect.Nullable
-  public final Auction newAuction;
-
-  @Nullable
-  @org.apache.avro.reflect.Nullable
-  public final Bid bid;
-
-  // For Avro only.
-  @SuppressWarnings("unused")
-  private Event() {
-    newPerson = null;
-    newAuction = null;
-    bid = null;
-  }
-
-  public Event(Person newPerson) {
-    this.newPerson = newPerson;
-    newAuction = null;
-    bid = null;
-  }
-
-  public Event(Auction newAuction) {
-    newPerson = null;
-    this.newAuction = newAuction;
-    bid = null;
-  }
-
-  public Event(Bid bid) {
-    newPerson = null;
-    newAuction = null;
-    this.bid = bid;
-  }
-
-  /** Return a copy of event which captures {@code annotation}. (Used for debugging). */
-  public Event withAnnotation(String annotation) {
-    if (newPerson != null) {
-      return new Event(newPerson.withAnnotation(annotation));
-    } else if (newAuction != null) {
-      return new Event(newAuction.withAnnotation(annotation));
-    } else {
-      return new Event(bid.withAnnotation(annotation));
-    }
-  }
-
-  /** Does event have {@code annotation}? (Used for debugging.) */
-  public boolean hasAnnotation(String annotation) {
-    if (newPerson != null) {
-      return newPerson.hasAnnotation(annotation);
-    } else if (newAuction != null) {
-      return newAuction.hasAnnotation(annotation);
-    } else {
-      return bid.hasAnnotation(annotation);
-    }
-  }
-
-  @Override
-  public long sizeInBytes() {
-    if (newPerson != null) {
-      return 1 + newPerson.sizeInBytes();
-    } else if (newAuction != null) {
-      return 1 + newAuction.sizeInBytes();
-    } else if (bid != null) {
-      return 1 + bid.sizeInBytes();
-    } else {
-      throw new RuntimeException("invalid event");
-    }
-  }
-
-  @Override
-  public String toString() {
-    if (newPerson != null) {
-      return newPerson.toString();
-    } else if (newAuction != null) {
-      return newAuction.toString();
-    } else if (bid != null) {
-      return bid.toString();
-    } else {
-      throw new RuntimeException("invalid event");
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/f4333df7/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/model/IdNameReserve.java
----------------------------------------------------------------------
diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/model/IdNameReserve.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/model/IdNameReserve.java
deleted file mode 100644
index 8cade4e..0000000
--- a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/model/IdNameReserve.java
+++ /dev/null
@@ -1,98 +0,0 @@
-/*
- * 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.beam.integration.nexmark.model;
-
-import com.fasterxml.jackson.annotation.JsonProperty;
-import com.fasterxml.jackson.core.JsonProcessingException;
-import java.io.IOException;
-import java.io.InputStream;
-import java.io.OutputStream;
-import java.io.Serializable;
-import org.apache.beam.integration.nexmark.NexmarkUtils;
-import org.apache.beam.sdk.coders.Coder;
-import org.apache.beam.sdk.coders.CoderException;
-import org.apache.beam.sdk.coders.CustomCoder;
-import org.apache.beam.sdk.coders.StringUtf8Coder;
-import org.apache.beam.sdk.coders.VarLongCoder;
-
-/**
- * Result type of Query8.
- */
-public class IdNameReserve implements KnownSize, Serializable {
-  private static final Coder<Long> LONG_CODER = VarLongCoder.of();
-  private static final Coder<String> STRING_CODER = StringUtf8Coder.of();
-
-  public static final Coder<IdNameReserve> CODER = new CustomCoder<IdNameReserve>() {
-    @Override
-    public void encode(IdNameReserve value, OutputStream outStream)
-        throws CoderException, IOException {
-      LONG_CODER.encode(value.id, outStream);
-      STRING_CODER.encode(value.name, outStream);
-      LONG_CODER.encode(value.reserve, outStream);
-    }
-
-    @Override
-    public IdNameReserve decode(
-        InputStream inStream)
-        throws CoderException, IOException {
-      long id = LONG_CODER.decode(inStream);
-      String name = STRING_CODER.decode(inStream);
-      long reserve = LONG_CODER.decode(inStream);
-      return new IdNameReserve(id, name, reserve);
-    }
-    @Override public void verifyDeterministic() throws NonDeterministicException {}
-  };
-
-  @JsonProperty
-  private final long id;
-
-  @JsonProperty
-  private final String name;
-
-  /** Reserve price in cents. */
-  @JsonProperty
-  private final long reserve;
-
-  // For Avro only.
-  @SuppressWarnings("unused")
-  private IdNameReserve() {
-    id = 0;
-    name = null;
-    reserve = 0;
-  }
-
-  public IdNameReserve(long id, String name, long reserve) {
-    this.id = id;
-    this.name = name;
-    this.reserve = reserve;
-  }
-
-  @Override
-  public long sizeInBytes() {
-    return 8 + name.length() + 1 + 8;
-  }
-
-  @Override
-  public String toString() {
-    try {
-      return NexmarkUtils.MAPPER.writeValueAsString(this);
-    } catch (JsonProcessingException e) {
-      throw new RuntimeException(e);
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/f4333df7/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/model/KnownSize.java
----------------------------------------------------------------------
diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/model/KnownSize.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/model/KnownSize.java
deleted file mode 100644
index c742eac..0000000
--- a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/model/KnownSize.java
+++ /dev/null
@@ -1,26 +0,0 @@
-/*
- * 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.beam.integration.nexmark.model;
-
-/**
- * Interface for elements which can quickly estimate their encoded byte size.
- */
-public interface KnownSize {
-  long sizeInBytes();
-}
-

http://git-wip-us.apache.org/repos/asf/beam/blob/f4333df7/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/model/NameCityStateId.java
----------------------------------------------------------------------
diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/model/NameCityStateId.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/model/NameCityStateId.java
deleted file mode 100644
index 37bd3c6..0000000
--- a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/model/NameCityStateId.java
+++ /dev/null
@@ -1,103 +0,0 @@
-/*
- * 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.beam.integration.nexmark.model;
-
-import com.fasterxml.jackson.annotation.JsonProperty;
-import com.fasterxml.jackson.core.JsonProcessingException;
-import java.io.IOException;
-import java.io.InputStream;
-import java.io.OutputStream;
-import java.io.Serializable;
-import org.apache.beam.integration.nexmark.NexmarkUtils;
-import org.apache.beam.sdk.coders.Coder;
-import org.apache.beam.sdk.coders.CoderException;
-import org.apache.beam.sdk.coders.CustomCoder;
-import org.apache.beam.sdk.coders.StringUtf8Coder;
-import org.apache.beam.sdk.coders.VarLongCoder;
-
-/**
- * Result of Query3.
- */
-public class NameCityStateId implements KnownSize, Serializable {
-  private static final Coder<Long> LONG_CODER = VarLongCoder.of();
-  private static final Coder<String> STRING_CODER = StringUtf8Coder.of();
-
-  public static final Coder<NameCityStateId> CODER = new CustomCoder<NameCityStateId>() {
-    @Override
-    public void encode(NameCityStateId value, OutputStream outStream)
-        throws CoderException, IOException {
-      STRING_CODER.encode(value.name, outStream);
-      STRING_CODER.encode(value.city, outStream);
-      STRING_CODER.encode(value.state, outStream);
-      LONG_CODER.encode(value.id, outStream);
-    }
-
-    @Override
-    public NameCityStateId decode(InputStream inStream)
-        throws CoderException, IOException {
-      String name = STRING_CODER.decode(inStream);
-      String city = STRING_CODER.decode(inStream);
-      String state = STRING_CODER.decode(inStream);
-      long id = LONG_CODER.decode(inStream);
-      return new NameCityStateId(name, city, state, id);
-    }
-    @Override public void verifyDeterministic() throws NonDeterministicException {}
-  };
-
-  @JsonProperty
-  private final String name;
-
-  @JsonProperty
-  private final String city;
-
-  @JsonProperty
-  private final String state;
-
-  @JsonProperty
-  private final long id;
-
-  // For Avro only.
-  @SuppressWarnings("unused")
-  private NameCityStateId() {
-    name = null;
-    city = null;
-    state = null;
-    id = 0;
-  }
-
-  public NameCityStateId(String name, String city, String state, long id) {
-    this.name = name;
-    this.city = city;
-    this.state = state;
-    this.id = id;
-  }
-
-  @Override
-  public long sizeInBytes() {
-    return name.length() + 1 + city.length() + 1 + state.length() + 1 + 8;
-  }
-
-  @Override
-  public String toString() {
-    try {
-      return NexmarkUtils.MAPPER.writeValueAsString(this);
-    } catch (JsonProcessingException e) {
-      throw new RuntimeException(e);
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/f4333df7/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/model/Person.java
----------------------------------------------------------------------
diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/model/Person.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/model/Person.java
deleted file mode 100644
index bde587d..0000000
--- a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/model/Person.java
+++ /dev/null
@@ -1,163 +0,0 @@
-/*
- * 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.beam.integration.nexmark.model;
-
-import com.fasterxml.jackson.annotation.JsonProperty;
-import com.fasterxml.jackson.core.JsonProcessingException;
-import java.io.IOException;
-import java.io.InputStream;
-import java.io.OutputStream;
-import java.io.Serializable;
-import org.apache.beam.integration.nexmark.NexmarkUtils;
-import org.apache.beam.sdk.coders.Coder;
-import org.apache.beam.sdk.coders.CoderException;
-import org.apache.beam.sdk.coders.CustomCoder;
-import org.apache.beam.sdk.coders.StringUtf8Coder;
-import org.apache.beam.sdk.coders.VarLongCoder;
-
-/**
- * A person either creating an auction or making a bid.
- */
-public class Person implements KnownSize, Serializable {
-  private static final Coder<Long> LONG_CODER = VarLongCoder.of();
-  private static final Coder<String> STRING_CODER = StringUtf8Coder.of();
-  public static final Coder<Person> CODER = new CustomCoder<Person>() {
-    @Override
-    public void encode(Person value, OutputStream outStream)
-        throws CoderException, IOException {
-      LONG_CODER.encode(value.id, outStream);
-      STRING_CODER.encode(value.name, outStream);
-      STRING_CODER.encode(value.emailAddress, outStream);
-      STRING_CODER.encode(value.creditCard, outStream);
-      STRING_CODER.encode(value.city, outStream);
-      STRING_CODER.encode(value.state, outStream);
-      LONG_CODER.encode(value.dateTime, outStream);
-      STRING_CODER.encode(value.extra, outStream);
-    }
-
-    @Override
-    public Person decode(InputStream inStream)
-        throws CoderException, IOException {
-      long id = LONG_CODER.decode(inStream);
-      String name = STRING_CODER.decode(inStream);
-      String emailAddress = STRING_CODER.decode(inStream);
-      String creditCard = STRING_CODER.decode(inStream);
-      String city = STRING_CODER.decode(inStream);
-      String state = STRING_CODER.decode(inStream);
-      long dateTime = LONG_CODER.decode(inStream);
-      String extra = STRING_CODER.decode(inStream);
-      return new Person(id, name, emailAddress, creditCard, city, state, dateTime, extra);
-    }
-    @Override public void verifyDeterministic() throws NonDeterministicException {}
-  };
-
-  /** Id of person. */
-  @JsonProperty
-  public final long id; // primary key
-
-  /** Extra person properties. */
-  @JsonProperty
-  public final String name;
-
-  @JsonProperty
-  private final String emailAddress;
-
-  @JsonProperty
-  private final String creditCard;
-
-  @JsonProperty
-  public final String city;
-
-  @JsonProperty
-  public final String state;
-
-  @JsonProperty
-  public final long dateTime;
-
-  /** Additional arbitrary payload for performance testing. */
-  @JsonProperty
-  private final String extra;
-
-  // For Avro only.
-  @SuppressWarnings("unused")
-  private Person() {
-    id = 0;
-    name = null;
-    emailAddress = null;
-    creditCard = null;
-    city = null;
-    state = null;
-    dateTime = 0;
-    extra = null;
-  }
-
-  public Person(long id, String name, String emailAddress, String creditCard, String city,
-      String state, long dateTime, String extra) {
-    this.id = id;
-    this.name = name;
-    this.emailAddress = emailAddress;
-    this.creditCard = creditCard;
-    this.city = city;
-    this.state = state;
-    this.dateTime = dateTime;
-    this.extra = extra;
-  }
-
-  /**
-   * Return a copy of person which capture the given annotation.
-   * (Used for debugging).
-   */
-  public Person withAnnotation(String annotation) {
-    return new Person(id, name, emailAddress, creditCard, city, state, dateTime,
-        annotation + ": " + extra);
-  }
-
-  /**
-   * Does person have {@code annotation}? (Used for debugging.)
-   */
-  public boolean hasAnnotation(String annotation) {
-    return extra.startsWith(annotation + ": ");
-  }
-
-  /**
-   * Remove {@code annotation} from person. (Used for debugging.)
-   */
-  public Person withoutAnnotation(String annotation) {
-    if (hasAnnotation(annotation)) {
-      return new Person(id, name, emailAddress, creditCard, city, state, dateTime,
-          extra.substring(annotation.length() + 2));
-    } else {
-      return this;
-    }
-  }
-
-  @Override
-  public long sizeInBytes() {
-    return 8 + name.length() + 1 + emailAddress.length() + 1 + creditCard.length() + 1
-        + city.length() + 1 + state.length() + 8 + 1 + extra.length() + 1;
-  }
-
-  @Override
-  public String toString() {
-    try {
-      return NexmarkUtils.MAPPER.writeValueAsString(this);
-    } catch (JsonProcessingException e) {
-      throw new RuntimeException(e);
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/f4333df7/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/model/SellerPrice.java
----------------------------------------------------------------------
diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/model/SellerPrice.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/model/SellerPrice.java
deleted file mode 100644
index 61537f6..0000000
--- a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/model/SellerPrice.java
+++ /dev/null
@@ -1,89 +0,0 @@
-/*
- * 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.beam.integration.nexmark.model;
-
-import com.fasterxml.jackson.annotation.JsonProperty;
-import com.fasterxml.jackson.core.JsonProcessingException;
-import java.io.IOException;
-import java.io.InputStream;
-import java.io.OutputStream;
-import java.io.Serializable;
-import org.apache.beam.integration.nexmark.NexmarkUtils;
-import org.apache.beam.sdk.coders.Coder;
-import org.apache.beam.sdk.coders.CoderException;
-import org.apache.beam.sdk.coders.CustomCoder;
-import org.apache.beam.sdk.coders.VarLongCoder;
-
-/**
- * Result of Query6.
- */
-public class SellerPrice implements KnownSize, Serializable {
-  private static final Coder<Long> LONG_CODER = VarLongCoder.of();
-
-  public static final Coder<SellerPrice> CODER = new CustomCoder<SellerPrice>() {
-    @Override
-    public void encode(SellerPrice value, OutputStream outStream)
-        throws CoderException, IOException {
-      LONG_CODER.encode(value.seller, outStream);
-      LONG_CODER.encode(value.price, outStream);
-    }
-
-    @Override
-    public SellerPrice decode(
-        InputStream inStream)
-        throws CoderException, IOException {
-      long seller = LONG_CODER.decode(inStream);
-      long price = LONG_CODER.decode(inStream);
-      return new SellerPrice(seller, price);
-    }
-    @Override public void verifyDeterministic() throws NonDeterministicException {}
-  };
-
-  @JsonProperty
-  public final long seller;
-
-  /** Price in cents. */
-  @JsonProperty
-  private final long price;
-
-  // For Avro only.
-  @SuppressWarnings("unused")
-  private SellerPrice() {
-    seller = 0;
-    price = 0;
-  }
-
-  public SellerPrice(long seller, long price) {
-    this.seller = seller;
-    this.price = price;
-  }
-
-  @Override
-  public long sizeInBytes() {
-    return 8 + 8;
-  }
-
-  @Override
-  public String toString() {
-    try {
-      return NexmarkUtils.MAPPER.writeValueAsString(this);
-    } catch (JsonProcessingException e) {
-      throw new RuntimeException(e);
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/f4333df7/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/model/package-info.java
----------------------------------------------------------------------
diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/model/package-info.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/model/package-info.java
deleted file mode 100644
index e1d6113..0000000
--- a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/model/package-info.java
+++ /dev/null
@@ -1,22 +0,0 @@
-/*
- * 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.
- */
-
-/**
- * Nexmark Benchmark Model.
- */
-package org.apache.beam.integration.nexmark.model;

http://git-wip-us.apache.org/repos/asf/beam/blob/f4333df7/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/package-info.java
----------------------------------------------------------------------
diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/package-info.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/package-info.java
deleted file mode 100644
index df6f09f..0000000
--- a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/package-info.java
+++ /dev/null
@@ -1,21 +0,0 @@
-/*
- * 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.
- */
-/**
- * Nexmark.
- */
-package org.apache.beam.integration.nexmark;


[12/55] [abbrv] beam git commit: Refactor classes into packages

Posted by ie...@apache.org.
http://git-wip-us.apache.org/repos/asf/beam/blob/a7f9f7d0/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/UnboundedEventSource.java
----------------------------------------------------------------------
diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/UnboundedEventSource.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/UnboundedEventSource.java
deleted file mode 100644
index 9573ef7..0000000
--- a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/UnboundedEventSource.java
+++ /dev/null
@@ -1,328 +0,0 @@
-/*
- * 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.beam.integration.nexmark;
-
-import java.util.ArrayList;
-import java.util.List;
-import java.util.NoSuchElementException;
-import java.util.PriorityQueue;
-import java.util.Queue;
-import java.util.concurrent.ThreadLocalRandom;
-import javax.annotation.Nullable;
-
-import org.apache.beam.sdk.coders.Coder;
-import org.apache.beam.sdk.io.UnboundedSource;
-import org.apache.beam.sdk.options.PipelineOptions;
-import org.apache.beam.sdk.transforms.windowing.BoundedWindow;
-import org.apache.beam.sdk.values.TimestampedValue;
-import org.joda.time.Duration;
-import org.joda.time.Instant;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-/**
- * A custom, unbounded source of event records.
- *
- * <p>If {@code isRateLimited} is true, events become available for return from the reader such
- * that the overall rate respect the {@code interEventDelayUs} period if possible. Otherwise,
- * events are returned every time the system asks for one.
- */
-class UnboundedEventSource extends UnboundedSource<Event, Generator.Checkpoint> {
-  private static final Duration BACKLOG_PERIOD = Duration.standardSeconds(30);
-  private static final Logger LOG = LoggerFactory.getLogger(UnboundedEventSource.class);
-
-  /** Configuration for generator to use when reading synthetic events. May be split. */
-  private final GeneratorConfig config;
-
-  /** How many unbounded sources to create. */
-  private final int numEventGenerators;
-
-  /** How many seconds to hold back the watermark. */
-  private final long watermarkHoldbackSec;
-
-  /** Are we rate limiting the events? */
-  private final boolean isRateLimited;
-
-  public UnboundedEventSource(GeneratorConfig config, int numEventGenerators,
-      long watermarkHoldbackSec, boolean isRateLimited) {
-    this.config = config;
-    this.numEventGenerators = numEventGenerators;
-    this.watermarkHoldbackSec = watermarkHoldbackSec;
-    this.isRateLimited = isRateLimited;
-  }
-
-  /** A reader to pull events from the generator. */
-  private class EventReader extends UnboundedReader<Event> {
-    /** Generator we are reading from. */
-    private final Generator generator;
-
-    /**
-     * Current watermark (ms since epoch). Initially set to beginning of time.
-     * Then updated to be the time of the next generated event.
-     * Then, once all events have been generated, set to the end of time.
-     */
-    private long watermark;
-
-    /**
-     * Current backlog (ms), as delay between timestamp of last returned event and the timestamp
-     * we should be up to according to wall-clock time. Used only for logging.
-     */
-    private long backlogDurationMs;
-
-    /**
-     * Current backlog, as estimated number of event bytes we are behind, or null if
-     * unknown. Reported to callers.
-     */
-    @Nullable
-    private Long backlogBytes;
-
-    /**
-     * Wallclock time (ms since epoch) we last reported the backlog, or -1 if never reported.
-     */
-    private long lastReportedBacklogWallclock;
-
-    /**
-     * Event time (ms since epoch) of pending event at last reported backlog, or -1 if never
-     * calculated.
-     */
-    private long timestampAtLastReportedBacklogMs;
-
-    /** Next event to make 'current' when wallclock time has advanced sufficiently. */
-    @Nullable
-    private TimestampedValue<Event> pendingEvent;
-
-    /** Wallclock time when {@link #pendingEvent} is due, or -1 if no pending event. */
-    private long pendingEventWallclockTime;
-
-    /** Current event to return from getCurrent. */
-    @Nullable
-    private TimestampedValue<Event> currentEvent;
-
-    /** Events which have been held back so as to force them to be late. */
-    private Queue<Generator.NextEvent> heldBackEvents = new PriorityQueue<>();
-
-    public EventReader(Generator generator) {
-      this.generator = generator;
-      watermark = NexmarkUtils.BEGINNING_OF_TIME.getMillis();
-      lastReportedBacklogWallclock = -1;
-      pendingEventWallclockTime = -1;
-      timestampAtLastReportedBacklogMs = -1;
-    }
-
-    public EventReader(GeneratorConfig config) {
-      this(new Generator(config));
-    }
-
-    @Override
-    public boolean start() {
-      LOG.trace("starting unbounded generator {}", generator);
-      return advance();
-    }
-
-
-    @Override
-    public boolean advance() {
-      long now = System.currentTimeMillis();
-
-      while (pendingEvent == null) {
-        if (!generator.hasNext() && heldBackEvents.isEmpty()) {
-          // No more events, EVER.
-          if (isRateLimited) {
-            updateBacklog(System.currentTimeMillis(), 0);
-          }
-          if (watermark < BoundedWindow.TIMESTAMP_MAX_VALUE.getMillis()) {
-            watermark = BoundedWindow.TIMESTAMP_MAX_VALUE.getMillis();
-            LOG.trace("stopped unbounded generator {}", generator);
-          }
-          return false;
-        }
-
-        Generator.NextEvent next = heldBackEvents.peek();
-        if (next != null && next.wallclockTimestamp <= now) {
-          // Time to use the held-back event.
-          heldBackEvents.poll();
-          LOG.debug("replaying held-back event {}ms behind watermark",
-                             watermark - next.eventTimestamp);
-        } else if (generator.hasNext()) {
-          next = generator.nextEvent();
-          if (isRateLimited && config.configuration.probDelayedEvent > 0.0
-              && config.configuration.occasionalDelaySec > 0
-              && ThreadLocalRandom.current().nextDouble() < config.configuration.probDelayedEvent) {
-            // We'll hold back this event and go around again.
-            long delayMs =
-                ThreadLocalRandom.current().nextLong(config.configuration.occasionalDelaySec * 1000)
-                + 1L;
-            LOG.debug("delaying event by {}ms", delayMs);
-            heldBackEvents.add(next.withDelay(delayMs));
-            continue;
-          }
-        } else {
-          // Waiting for held-back event to fire.
-          if (isRateLimited) {
-            updateBacklog(now, 0);
-          }
-          return false;
-        }
-
-        pendingEventWallclockTime = next.wallclockTimestamp;
-        pendingEvent = TimestampedValue.of(next.event, new Instant(next.eventTimestamp));
-        long newWatermark =
-            next.watermark - Duration.standardSeconds(watermarkHoldbackSec).getMillis();
-        if (newWatermark > watermark) {
-          watermark = newWatermark;
-        }
-      }
-
-      if (isRateLimited) {
-        if (pendingEventWallclockTime > now) {
-          // We want this event to fire in the future. Try again later.
-          updateBacklog(now, 0);
-          return false;
-        }
-        updateBacklog(now, now - pendingEventWallclockTime);
-      }
-
-      // This event is ready to fire.
-      currentEvent = pendingEvent;
-      pendingEvent = null;
-      return true;
-    }
-
-    private void updateBacklog(long now, long newBacklogDurationMs) {
-      backlogDurationMs = newBacklogDurationMs;
-      long interEventDelayUs = generator.currentInterEventDelayUs();
-      if (interEventDelayUs != 0) {
-        long backlogEvents = (backlogDurationMs * 1000 + interEventDelayUs - 1) / interEventDelayUs;
-        backlogBytes = generator.getCurrentConfig().estimatedBytesForEvents(backlogEvents);
-      }
-      if (lastReportedBacklogWallclock < 0
-          || now - lastReportedBacklogWallclock > BACKLOG_PERIOD.getMillis()) {
-        double timeDialation = Double.NaN;
-        if (pendingEvent != null
-            && lastReportedBacklogWallclock >= 0
-            && timestampAtLastReportedBacklogMs >= 0) {
-          long wallclockProgressionMs = now - lastReportedBacklogWallclock;
-          long eventTimeProgressionMs =
-              pendingEvent.getTimestamp().getMillis() - timestampAtLastReportedBacklogMs;
-          timeDialation = (double) eventTimeProgressionMs / (double) wallclockProgressionMs;
-        }
-        LOG.debug(
-            "unbounded generator backlog now {}ms ({} bytes) at {}us interEventDelay "
-            + "with {} time dilation",
-            backlogDurationMs, backlogBytes, interEventDelayUs, timeDialation);
-        lastReportedBacklogWallclock = now;
-        if (pendingEvent != null) {
-          timestampAtLastReportedBacklogMs = pendingEvent.getTimestamp().getMillis();
-        }
-      }
-    }
-
-    @Override
-    public Event getCurrent() {
-      if (currentEvent == null) {
-        throw new NoSuchElementException();
-      }
-      return currentEvent.getValue();
-    }
-
-    @Override
-    public Instant getCurrentTimestamp() {
-      if (currentEvent == null) {
-        throw new NoSuchElementException();
-      }
-      return currentEvent.getTimestamp();
-    }
-
-    @Override
-    public void close() {
-      // Nothing to close.
-    }
-
-    @Override
-    public UnboundedEventSource getCurrentSource() {
-      return UnboundedEventSource.this;
-    }
-
-    @Override
-    public Instant getWatermark() {
-      return new Instant(watermark);
-    }
-
-    @Override
-    public Generator.Checkpoint getCheckpointMark() {
-      return generator.toCheckpoint();
-    }
-
-    @Override
-    public long getSplitBacklogBytes() {
-      return backlogBytes == null ? BACKLOG_UNKNOWN : backlogBytes;
-    }
-
-    @Override
-    public String toString() {
-      return String.format("EventReader(%d, %d, %d)",
-          generator.getCurrentConfig().getStartEventId(), generator.getNextEventId(),
-          generator.getCurrentConfig().getStopEventId());
-    }
-  }
-
-  @Override
-  public Coder<Generator.Checkpoint> getCheckpointMarkCoder() {
-    return Generator.Checkpoint.CODER_INSTANCE;
-  }
-
-  @Override
-  public List<UnboundedEventSource> generateInitialSplits(
-      int desiredNumSplits, PipelineOptions options) {
-    LOG.trace("splitting unbounded source into {} sub-sources", numEventGenerators);
-    List<UnboundedEventSource> results = new ArrayList<>();
-    // Ignore desiredNumSplits and use numEventGenerators instead.
-    for (GeneratorConfig subConfig : config.split(numEventGenerators)) {
-      results.add(new UnboundedEventSource(subConfig, 1, watermarkHoldbackSec, isRateLimited));
-    }
-    return results;
-  }
-
-  @Override
-  public EventReader createReader(
-      PipelineOptions options, @Nullable Generator.Checkpoint checkpoint) {
-    if (checkpoint == null) {
-      LOG.trace("creating initial unbounded reader for {}", config);
-      return new EventReader(config);
-    } else {
-      LOG.trace("resuming unbounded reader from {}", checkpoint);
-      return new EventReader(checkpoint.toGenerator(config));
-    }
-  }
-
-  @Override
-  public void validate() {
-    // Nothing to validate.
-  }
-
-  @Override
-  public Coder<Event> getDefaultOutputCoder() {
-    return Event.CODER;
-  }
-
-  @Override
-  public String toString() {
-    return String.format(
-        "UnboundedEventSource(%d, %d)", config.getStartEventId(), config.getStopEventId());
-  }
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/a7f9f7d0/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/WinningBids.java
----------------------------------------------------------------------
diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/WinningBids.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/WinningBids.java
index 594195a..9f1ddf8 100644
--- a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/WinningBids.java
+++ b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/WinningBids.java
@@ -20,7 +20,6 @@ package org.apache.beam.integration.nexmark;
 import static com.google.common.base.Preconditions.checkState;
 
 import com.fasterxml.jackson.annotation.JsonCreator;
-
 import java.io.IOException;
 import java.io.InputStream;
 import java.io.OutputStream;
@@ -31,7 +30,11 @@ import java.util.Collection;
 import java.util.List;
 import java.util.Map;
 import java.util.TreeMap;
-
+import org.apache.beam.integration.nexmark.model.Auction;
+import org.apache.beam.integration.nexmark.model.AuctionBid;
+import org.apache.beam.integration.nexmark.model.Bid;
+import org.apache.beam.integration.nexmark.model.Event;
+import org.apache.beam.integration.nexmark.sources.GeneratorConfig;
 import org.apache.beam.sdk.coders.AtomicCoder;
 import org.apache.beam.sdk.coders.Coder;
 import org.apache.beam.sdk.coders.CoderException;
@@ -45,10 +48,10 @@ import org.apache.beam.sdk.transforms.Sum;
 import org.apache.beam.sdk.transforms.join.CoGbkResult;
 import org.apache.beam.sdk.transforms.join.CoGroupByKey;
 import org.apache.beam.sdk.transforms.join.KeyedPCollectionTuple;
-import org.apache.beam.sdk.transforms.windowing.BoundedWindow;
 import org.apache.beam.sdk.transforms.windowing.IntervalWindow;
 import org.apache.beam.sdk.transforms.windowing.Window;
 import org.apache.beam.sdk.transforms.windowing.WindowFn;
+import org.apache.beam.sdk.transforms.windowing.WindowMappingFn;
 import org.apache.beam.sdk.values.KV;
 import org.apache.beam.sdk.values.PCollection;
 import org.joda.time.Instant;
@@ -249,7 +252,7 @@ public class WinningBids extends PTransform<PCollection<Event>, PCollection<Auct
     }
 
     @Override
-    public AuctionOrBidWindow getSideInputWindow(BoundedWindow window) {
+    public WindowMappingFn<AuctionOrBidWindow> getDefaultWindowMappingFn() {
       throw new UnsupportedOperationException("AuctionWindowFn not supported for side inputs");
     }
 

http://git-wip-us.apache.org/repos/asf/beam/blob/a7f9f7d0/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/WinningBidsSimulator.java
----------------------------------------------------------------------
diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/WinningBidsSimulator.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/WinningBidsSimulator.java
index dc8094b..e7f51b7 100644
--- a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/WinningBidsSimulator.java
+++ b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/WinningBidsSimulator.java
@@ -26,6 +26,10 @@ import java.util.TreeMap;
 import java.util.TreeSet;
 import javax.annotation.Nullable;
 
+import org.apache.beam.integration.nexmark.model.Auction;
+import org.apache.beam.integration.nexmark.model.AuctionBid;
+import org.apache.beam.integration.nexmark.model.Bid;
+import org.apache.beam.integration.nexmark.model.Event;
 import org.apache.beam.sdk.transforms.windowing.BoundedWindow;
 import org.apache.beam.sdk.values.TimestampedValue;
 import org.joda.time.Instant;

http://git-wip-us.apache.org/repos/asf/beam/blob/a7f9f7d0/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/drivers/NexmarkApexDriver.java
----------------------------------------------------------------------
diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/drivers/NexmarkApexDriver.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/drivers/NexmarkApexDriver.java
new file mode 100644
index 0000000..265ccf7
--- /dev/null
+++ b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/drivers/NexmarkApexDriver.java
@@ -0,0 +1,50 @@
+/*
+ * 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.beam.integration.nexmark.drivers;
+
+import org.apache.beam.integration.nexmark.NexmarkDriver;
+import org.apache.beam.integration.nexmark.NexmarkOptions;
+import org.apache.beam.runners.apex.ApexPipelineOptions;
+import org.apache.beam.runners.apex.ApexRunner;
+import org.apache.beam.sdk.options.PipelineOptionsFactory;
+
+/**
+ * Run NexMark queries using the Apex runner.
+ */
+public class NexmarkApexDriver extends NexmarkDriver<NexmarkApexDriver.NexmarkApexOptions> {
+  /**
+   * Command line flags.
+   */
+  public interface NexmarkApexOptions extends NexmarkOptions, ApexPipelineOptions {
+  }
+
+  /**
+   * Entry point.
+   */
+  public static void main(String[] args) {
+    // Gather command line args, baseline, configurations, etc.
+    NexmarkApexOptions options = PipelineOptionsFactory.fromArgs(args)
+                                                        .withValidation()
+                                                        .as(NexmarkApexOptions.class);
+    options.setRunner(ApexRunner.class);
+    NexmarkApexRunner runner = new NexmarkApexRunner(options);
+    new NexmarkApexDriver().runAll(options, runner);
+  }
+}
+
+

http://git-wip-us.apache.org/repos/asf/beam/blob/a7f9f7d0/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/drivers/NexmarkApexRunner.java
----------------------------------------------------------------------
diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/drivers/NexmarkApexRunner.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/drivers/NexmarkApexRunner.java
new file mode 100644
index 0000000..2bcf82d
--- /dev/null
+++ b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/drivers/NexmarkApexRunner.java
@@ -0,0 +1,65 @@
+/*
+ * 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.beam.integration.nexmark.drivers;
+
+import javax.annotation.Nullable;
+
+import org.apache.beam.integration.nexmark.NexmarkPerf;
+import org.apache.beam.integration.nexmark.NexmarkQuery;
+import org.apache.beam.integration.nexmark.NexmarkRunner;
+
+/**
+ * Run a query using the Apex runner.
+ */
+public class NexmarkApexRunner extends NexmarkRunner<NexmarkApexDriver.NexmarkApexOptions> {
+  @Override
+  protected boolean isStreaming() {
+    return options.isStreaming();
+  }
+
+  @Override
+  protected int coresPerWorker() {
+    return 4;
+  }
+
+  @Override
+  protected int maxNumWorkers() {
+    return 5;
+  }
+
+  @Override
+  protected void invokeBuilderForPublishOnlyPipeline(
+      PipelineBuilder builder) {
+    builder.build(options);
+  }
+
+  @Override
+  protected void waitForPublisherPreload() {
+    throw new UnsupportedOperationException();
+  }
+
+  @Override
+  @Nullable
+  protected NexmarkPerf monitor(NexmarkQuery query) {
+    return null;
+  }
+
+  public NexmarkApexRunner(NexmarkApexDriver.NexmarkApexOptions options) {
+    super(options);
+  }
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/a7f9f7d0/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/drivers/NexmarkDirectDriver.java
----------------------------------------------------------------------
diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/drivers/NexmarkDirectDriver.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/drivers/NexmarkDirectDriver.java
new file mode 100644
index 0000000..2b825f3
--- /dev/null
+++ b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/drivers/NexmarkDirectDriver.java
@@ -0,0 +1,49 @@
+/*
+ * 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.beam.integration.nexmark.drivers;
+
+import org.apache.beam.integration.nexmark.NexmarkDriver;
+import org.apache.beam.integration.nexmark.NexmarkOptions;
+import org.apache.beam.runners.direct.DirectOptions;
+import org.apache.beam.runners.direct.DirectRunner;
+import org.apache.beam.sdk.options.PipelineOptionsFactory;
+
+/**
+ * An implementation of the 'NEXMark queries' using the Direct Runner.
+ */
+class NexmarkDirectDriver extends NexmarkDriver<NexmarkDirectDriver.NexmarkDirectOptions> {
+  /**
+   * Command line flags.
+   */
+  public interface NexmarkDirectOptions extends NexmarkOptions, DirectOptions {
+  }
+
+  /**
+   * Entry point.
+   */
+  public static void main(String[] args) {
+    NexmarkDirectOptions options =
+        PipelineOptionsFactory.fromArgs(args)
+                              .withValidation()
+                              .as(NexmarkDirectOptions.class);
+    options.setRunner(DirectRunner.class);
+    NexmarkDirectRunner runner = new NexmarkDirectRunner(options);
+    new NexmarkDirectDriver().runAll(options, runner);
+  }
+}
+

http://git-wip-us.apache.org/repos/asf/beam/blob/a7f9f7d0/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/drivers/NexmarkDirectRunner.java
----------------------------------------------------------------------
diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/drivers/NexmarkDirectRunner.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/drivers/NexmarkDirectRunner.java
new file mode 100644
index 0000000..1391040
--- /dev/null
+++ b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/drivers/NexmarkDirectRunner.java
@@ -0,0 +1,60 @@
+/*
+ * 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.beam.integration.nexmark.drivers;
+
+import org.apache.beam.integration.nexmark.NexmarkRunner;
+
+/**
+ * Run a single query using the Direct Runner.
+ */
+class NexmarkDirectRunner extends NexmarkRunner<NexmarkDirectDriver.NexmarkDirectOptions> {
+  public NexmarkDirectRunner(NexmarkDirectDriver.NexmarkDirectOptions options) {
+    super(options);
+  }
+
+  @Override
+  protected boolean isStreaming() {
+    return options.isStreaming();
+  }
+
+  @Override
+  protected int coresPerWorker() {
+    return 4;
+  }
+
+  @Override
+  protected int maxNumWorkers() {
+    return 1;
+  }
+
+  @Override
+  protected void invokeBuilderForPublishOnlyPipeline(PipelineBuilder builder) {
+    throw new UnsupportedOperationException(
+        "Cannot use --pubSubMode=COMBINED with DirectRunner");
+  }
+
+  /**
+   * Monitor the progress of the publisher job. Return when it has been generating events for
+   * at least {@code configuration.preloadSeconds}.
+   */
+  @Override
+  protected void waitForPublisherPreload() {
+    throw new UnsupportedOperationException(
+        "Cannot use --pubSubMode=COMBINED with DirectRunner");
+  }
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/a7f9f7d0/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/drivers/NexmarkFlinkDriver.java
----------------------------------------------------------------------
diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/drivers/NexmarkFlinkDriver.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/drivers/NexmarkFlinkDriver.java
new file mode 100644
index 0000000..bf0b115
--- /dev/null
+++ b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/drivers/NexmarkFlinkDriver.java
@@ -0,0 +1,50 @@
+/*
+ * 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.beam.integration.nexmark.drivers;
+
+import org.apache.beam.integration.nexmark.NexmarkDriver;
+import org.apache.beam.integration.nexmark.NexmarkOptions;
+import org.apache.beam.runners.flink.FlinkPipelineOptions;
+import org.apache.beam.runners.flink.FlinkRunner;
+import org.apache.beam.sdk.options.PipelineOptionsFactory;
+
+/**
+ * Run NexMark queries using the Flink runner.
+ */
+public class NexmarkFlinkDriver extends NexmarkDriver<NexmarkFlinkDriver.NexmarkFlinkOptions> {
+  /**
+   * Command line flags.
+   */
+  public interface NexmarkFlinkOptions extends NexmarkOptions, FlinkPipelineOptions {
+  }
+
+  /**
+   * Entry point.
+   */
+  public static void main(String[] args) {
+    // Gather command line args, baseline, configurations, etc.
+    NexmarkFlinkOptions options = PipelineOptionsFactory.fromArgs(args)
+                                                        .withValidation()
+                                                        .as(NexmarkFlinkOptions.class);
+    options.setRunner(FlinkRunner.class);
+    NexmarkFlinkRunner runner = new NexmarkFlinkRunner(options);
+    new NexmarkFlinkDriver().runAll(options, runner);
+  }
+}
+
+

http://git-wip-us.apache.org/repos/asf/beam/blob/a7f9f7d0/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/drivers/NexmarkFlinkRunner.java
----------------------------------------------------------------------
diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/drivers/NexmarkFlinkRunner.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/drivers/NexmarkFlinkRunner.java
new file mode 100644
index 0000000..9d547ef
--- /dev/null
+++ b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/drivers/NexmarkFlinkRunner.java
@@ -0,0 +1,55 @@
+/*
+ * 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.beam.integration.nexmark.drivers;
+
+import org.apache.beam.integration.nexmark.NexmarkRunner;
+
+/**
+ * Run a query using the Flink runner.
+ */
+public class NexmarkFlinkRunner extends NexmarkRunner<NexmarkFlinkDriver.NexmarkFlinkOptions> {
+  @Override
+  protected boolean isStreaming() {
+    return options.isStreaming();
+  }
+
+  @Override
+  protected int coresPerWorker() {
+    return 4;
+  }
+
+  @Override
+  protected int maxNumWorkers() {
+    return 5;
+  }
+
+  @Override
+  protected void invokeBuilderForPublishOnlyPipeline(
+      PipelineBuilder builder) {
+    builder.build(options);
+  }
+
+  @Override
+  protected void waitForPublisherPreload() {
+    throw new UnsupportedOperationException();
+  }
+
+  public NexmarkFlinkRunner(NexmarkFlinkDriver.NexmarkFlinkOptions options) {
+    super(options);
+  }
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/a7f9f7d0/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/drivers/NexmarkGoogleDriver.java
----------------------------------------------------------------------
diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/drivers/NexmarkGoogleDriver.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/drivers/NexmarkGoogleDriver.java
new file mode 100644
index 0000000..f5a9751
--- /dev/null
+++ b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/drivers/NexmarkGoogleDriver.java
@@ -0,0 +1,67 @@
+/*
+ * 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.beam.integration.nexmark.drivers;
+
+import org.apache.beam.integration.nexmark.NexmarkDriver;
+import org.apache.beam.integration.nexmark.NexmarkOptions;
+import org.apache.beam.integration.nexmark.model.Auction;
+import org.apache.beam.integration.nexmark.model.Bid;
+import org.apache.beam.integration.nexmark.model.Person;
+import org.apache.beam.runners.dataflow.DataflowRunner;
+import org.apache.beam.runners.dataflow.options.DataflowPipelineOptions;
+import org.apache.beam.sdk.options.PipelineOptionsFactory;
+
+/**
+ * An implementation of the 'NEXMark queries' for Google Dataflow.
+ * These are multiple queries over a three table schema representing an online auction system:
+ * <ul>
+ * <li>{@link Person} represents a person submitting an item for auction and/or making a bid
+ * on an auction.
+ * <li>{@link Auction} represents an item under auction.
+ * <li>{@link Bid} represents a bid for an item under auction.
+ * </ul>
+ * The queries exercise many aspects of streaming dataflow.
+ *
+ * <p>We synthesize the creation of people, auctions and bids in real-time. The data is not
+ * particularly sensible.
+ *
+ * <p>See <a href="http://datalab.cs.pdx.edu/niagaraST/NEXMark/">
+ * http://datalab.cs.pdx.edu/niagaraST/NEXMark/</a>
+ */
+class NexmarkGoogleDriver extends NexmarkDriver<NexmarkGoogleDriver.NexmarkGoogleOptions> {
+  /**
+   * Command line flags.
+   */
+  public interface NexmarkGoogleOptions extends NexmarkOptions, DataflowPipelineOptions {
+
+  }
+
+  /**
+   * Entry point.
+   */
+  public static void main(String[] args) {
+    // Gather command line args, baseline, configurations, etc.
+    NexmarkGoogleOptions options = PipelineOptionsFactory.fromArgs(args)
+                                                         .withValidation()
+                                                         .as(NexmarkGoogleOptions.class);
+    options.setRunner(DataflowRunner.class);
+    NexmarkGoogleRunner runner = new NexmarkGoogleRunner(options);
+    new NexmarkGoogleDriver().runAll(options, runner);
+  }
+}
+

http://git-wip-us.apache.org/repos/asf/beam/blob/a7f9f7d0/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/drivers/NexmarkGoogleRunner.java
----------------------------------------------------------------------
diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/drivers/NexmarkGoogleRunner.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/drivers/NexmarkGoogleRunner.java
new file mode 100644
index 0000000..7ffd47a
--- /dev/null
+++ b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/drivers/NexmarkGoogleRunner.java
@@ -0,0 +1,163 @@
+/*
+ * 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.beam.integration.nexmark.drivers;
+
+import static com.google.common.base.Preconditions.checkNotNull;
+
+import org.apache.beam.integration.nexmark.Monitor;
+import org.apache.beam.integration.nexmark.NexmarkRunner;
+import org.apache.beam.integration.nexmark.NexmarkUtils;
+import org.apache.beam.integration.nexmark.model.Event;
+import org.apache.beam.runners.dataflow.DataflowPipelineJob;
+import org.apache.beam.sdk.PipelineResult;
+import org.joda.time.Duration;
+
+/**
+ * Run a singe Nexmark query using a given configuration on Google Dataflow.
+ */
+class NexmarkGoogleRunner extends NexmarkRunner<NexmarkGoogleDriver.NexmarkGoogleOptions> {
+
+  public NexmarkGoogleRunner(NexmarkGoogleDriver.NexmarkGoogleOptions options) {
+    super(options);
+  }
+
+  @Override
+  protected boolean isStreaming() {
+    return options.isStreaming();
+  }
+
+  @Override
+  protected int coresPerWorker() {
+    String machineType = options.getWorkerMachineType();
+    if (machineType == null || machineType.isEmpty()) {
+      return 1;
+    }
+    String[] split = machineType.split("-");
+    if (split.length != 3) {
+      return 1;
+    }
+    try {
+      return Integer.parseInt(split[2]);
+    } catch (NumberFormatException ex) {
+      return 1;
+    }
+  }
+
+  @Override
+  protected int maxNumWorkers() {
+    return Math.max(options.getNumWorkers(), options.getMaxNumWorkers());
+  }
+
+  @Override
+  protected String getJobId(PipelineResult job) {
+    return ((DataflowPipelineJob) job).getJobId();
+  }
+
+  @Override
+  protected void invokeBuilderForPublishOnlyPipeline(PipelineBuilder builder) {
+    String jobName = options.getJobName();
+    String appName = options.getAppName();
+    options.setJobName("p-" + jobName);
+    options.setAppName("p-" + appName);
+    int coresPerWorker = coresPerWorker();
+    int eventGeneratorWorkers = (configuration.numEventGenerators + coresPerWorker - 1)
+                                / coresPerWorker;
+    options.setMaxNumWorkers(Math.min(options.getMaxNumWorkers(), eventGeneratorWorkers));
+    options.setNumWorkers(Math.min(options.getNumWorkers(), eventGeneratorWorkers));
+    publisherMonitor = new Monitor<Event>(queryName, "publisher");
+    try {
+      builder.build(options);
+    } finally {
+      options.setJobName(jobName);
+      options.setAppName(appName);
+      options.setMaxNumWorkers(options.getMaxNumWorkers());
+      options.setNumWorkers(options.getNumWorkers());
+    }
+  }
+
+  /**
+   * Monitor the progress of the publisher job. Return when it has been generating events for
+   * at least {@code configuration.preloadSeconds}.
+   */
+  @Override
+  protected void waitForPublisherPreload() {
+    checkNotNull(publisherMonitor);
+    checkNotNull(publisherResult);
+    if (!options.getMonitorJobs()) {
+      return;
+    }
+    if (!(publisherResult instanceof DataflowPipelineJob)) {
+      return;
+    }
+    if (configuration.preloadSeconds <= 0) {
+      return;
+    }
+
+    NexmarkUtils.console("waiting for publisher to pre-load");
+
+    DataflowPipelineJob job = (DataflowPipelineJob) publisherResult;
+
+    long numEvents = 0;
+    long startMsSinceEpoch = -1;
+    long endMsSinceEpoch = -1;
+    while (true) {
+      PipelineResult.State state = job.getState();
+      switch (state) {
+        case UNKNOWN:
+          // Keep waiting.
+          NexmarkUtils.console("%s publisher (%d events)", state, numEvents);
+          break;
+        case STOPPED:
+        case DONE:
+        case CANCELLED:
+        case FAILED:
+        case UPDATED:
+          NexmarkUtils.console("%s publisher (%d events)", state, numEvents);
+          return;
+        case RUNNING:
+          numEvents = getLong(job, publisherMonitor.getElementCounter());
+          if (startMsSinceEpoch < 0 && numEvents > 0) {
+            startMsSinceEpoch = System.currentTimeMillis();
+            endMsSinceEpoch = startMsSinceEpoch
+                              + Duration.standardSeconds(configuration.preloadSeconds).getMillis();
+          }
+          if (endMsSinceEpoch < 0) {
+            NexmarkUtils.console("%s publisher (%d events)", state, numEvents);
+          } else {
+            long remainMs = endMsSinceEpoch - System.currentTimeMillis();
+            if (remainMs > 0) {
+              NexmarkUtils.console("%s publisher (%d events, waiting for %ds)", state, numEvents,
+                  remainMs / 1000);
+            } else {
+              NexmarkUtils.console("publisher preloaded %d events", numEvents);
+              return;
+            }
+          }
+          break;
+      }
+
+      try {
+        Thread.sleep(PERF_DELAY.getMillis());
+      } catch (InterruptedException e) {
+        Thread.interrupted();
+        throw new RuntimeException("Interrupted: publisher still running.");
+      }
+    }
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/a7f9f7d0/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/drivers/NexmarkSparkDriver.java
----------------------------------------------------------------------
diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/drivers/NexmarkSparkDriver.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/drivers/NexmarkSparkDriver.java
new file mode 100644
index 0000000..c7c32c2
--- /dev/null
+++ b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/drivers/NexmarkSparkDriver.java
@@ -0,0 +1,48 @@
+/*
+ * 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.beam.integration.nexmark.drivers;
+
+import org.apache.beam.integration.nexmark.NexmarkDriver;
+import org.apache.beam.integration.nexmark.NexmarkOptions;
+import org.apache.beam.runners.spark.SparkPipelineOptions;
+import org.apache.beam.runners.spark.SparkRunner;
+import org.apache.beam.sdk.options.PipelineOptionsFactory;
+
+/**
+ * Run NexMark queries using the Spark runner.
+ */
+class NexmarkSparkDriver extends NexmarkDriver<NexmarkSparkDriver.NexmarkSparkOptions> {
+    /**
+     * Command line flags.
+     */
+    public interface NexmarkSparkOptions extends NexmarkOptions, SparkPipelineOptions {
+    }
+
+    /**
+     * Entry point.
+     */
+    public static void main(String[] args) {
+        NexmarkSparkOptions options =
+                PipelineOptionsFactory.fromArgs(args)
+                        .withValidation()
+                        .as(NexmarkSparkOptions.class);
+        options.setRunner(SparkRunner.class);
+        NexmarkSparkRunner runner = new NexmarkSparkRunner(options);
+        new NexmarkSparkDriver().runAll(options, runner);
+    }
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/a7f9f7d0/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/drivers/NexmarkSparkRunner.java
----------------------------------------------------------------------
diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/drivers/NexmarkSparkRunner.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/drivers/NexmarkSparkRunner.java
new file mode 100644
index 0000000..1d49a3a
--- /dev/null
+++ b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/drivers/NexmarkSparkRunner.java
@@ -0,0 +1,56 @@
+/*
+ * 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.beam.integration.nexmark.drivers;
+
+import org.apache.beam.integration.nexmark.NexmarkRunner;
+
+/**
+ * Run a query using the Spark runner.
+ */
+public class NexmarkSparkRunner extends NexmarkRunner<NexmarkSparkDriver.NexmarkSparkOptions> {
+    @Override
+    protected boolean isStreaming() {
+        return options.isStreaming();
+    }
+
+    @Override
+    protected int coresPerWorker() {
+        return 4;
+    }
+
+    @Override
+    protected int maxNumWorkers() {
+        return 5;
+    }
+
+    @Override
+    protected void invokeBuilderForPublishOnlyPipeline(
+            PipelineBuilder builder) {
+        builder.build(options);
+    }
+
+    @Override
+    protected void waitForPublisherPreload() {
+        throw new UnsupportedOperationException();
+    }
+
+
+    public NexmarkSparkRunner(NexmarkSparkDriver.NexmarkSparkOptions options) {
+        super(options);
+    }
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/a7f9f7d0/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/drivers/package-info.java
----------------------------------------------------------------------
diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/drivers/package-info.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/drivers/package-info.java
new file mode 100644
index 0000000..c8aa144
--- /dev/null
+++ b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/drivers/package-info.java
@@ -0,0 +1,22 @@
+/*
+ * 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.
+ */
+
+/**
+ * Nexmark Benchmark Execution Drivers.
+ */
+package org.apache.beam.integration.nexmark.drivers;

http://git-wip-us.apache.org/repos/asf/beam/blob/a7f9f7d0/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/io/PubsubHelper.java
----------------------------------------------------------------------
diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/io/PubsubHelper.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/io/PubsubHelper.java
new file mode 100644
index 0000000..f5cfc2b
--- /dev/null
+++ b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/io/PubsubHelper.java
@@ -0,0 +1,217 @@
+/*
+ * 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.beam.integration.nexmark.io;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.List;
+
+import org.apache.beam.integration.nexmark.NexmarkUtils;
+import org.apache.beam.sdk.options.PubsubOptions;
+import org.apache.beam.sdk.util.PubsubClient;
+import org.apache.beam.sdk.util.PubsubJsonClient;
+
+/**
+ * Helper for working with pubsub.
+ */
+public class PubsubHelper implements AutoCloseable {
+  /**
+   * Underlying pub/sub client.
+   */
+  private final PubsubClient pubsubClient;
+
+  /**
+   * Project id.
+   */
+  private final String projectId;
+
+  /**
+   * Topics we should delete on close.
+   */
+  private final List<PubsubClient.TopicPath> createdTopics;
+
+  /**
+   * Subscriptions we should delete on close.
+   */
+  private final List<PubsubClient.SubscriptionPath> createdSubscriptions;
+
+  private PubsubHelper(PubsubClient pubsubClient, String projectId) {
+    this.pubsubClient = pubsubClient;
+    this.projectId = projectId;
+    createdTopics = new ArrayList<>();
+    createdSubscriptions = new ArrayList<>();
+  }
+
+  /**
+   * Create a helper.
+   */
+  public static PubsubHelper create(PubsubOptions options) {
+    try {
+      return new PubsubHelper(
+          PubsubJsonClient.FACTORY.newClient(null, null, options),
+          options.getProject());
+    } catch (IOException e) {
+      throw new RuntimeException("Unable to create Pubsub client: ", e);
+    }
+  }
+
+  /**
+   * Create a topic from short name. Delete it if it already exists. Ensure the topic will be
+   * deleted on cleanup. Return full topic name.
+   */
+  public PubsubClient.TopicPath createTopic(String shortTopic) {
+    PubsubClient.TopicPath topic = PubsubClient.topicPathFromName(projectId, shortTopic);
+    try {
+      if (topicExists(shortTopic)) {
+        NexmarkUtils.console("attempting to cleanup topic %s", topic);
+        pubsubClient.deleteTopic(topic);
+      }
+      NexmarkUtils.console("create topic %s", topic);
+      pubsubClient.createTopic(topic);
+      createdTopics.add(topic);
+      return topic;
+    } catch (IOException e) {
+      throw new RuntimeException("Unable to create Pubsub topic " + topic + ": ", e);
+    }
+  }
+
+  /**
+   * Create a topic from short name if it does not already exist. The topic will not be
+   * deleted on cleanup. Return full topic name.
+   */
+  public PubsubClient.TopicPath createOrReuseTopic(String shortTopic) {
+    PubsubClient.TopicPath topic = PubsubClient.topicPathFromName(projectId, shortTopic);
+    try {
+      if (topicExists(shortTopic)) {
+        NexmarkUtils.console("topic %s already exists", topic);
+        return topic;
+      }
+      NexmarkUtils.console("create topic %s", topic);
+      pubsubClient.createTopic(topic);
+      return topic;
+    } catch (IOException e) {
+      throw new RuntimeException("Unable to create or reuse Pubsub topic " + topic + ": ", e);
+    }
+  }
+
+  /**
+   * Check a topic corresponding to short name exists, and throw exception if not. The
+   * topic will not be deleted on cleanup. Return full topic name.
+   */
+  public PubsubClient.TopicPath reuseTopic(String shortTopic) {
+    PubsubClient.TopicPath topic = PubsubClient.topicPathFromName(projectId, shortTopic);
+    if (topicExists(shortTopic)) {
+      NexmarkUtils.console("reusing existing topic %s", topic);
+      return topic;
+    }
+    throw new RuntimeException("topic '" + topic + "' does not already exist");
+  }
+
+  /**
+   * Does topic corresponding to short name exist?
+   */
+  public boolean topicExists(String shortTopic) {
+    PubsubClient.ProjectPath project = PubsubClient.projectPathFromId(projectId);
+    PubsubClient.TopicPath topic = PubsubClient.topicPathFromName(projectId, shortTopic);
+    try {
+      Collection<PubsubClient.TopicPath> existingTopics = pubsubClient.listTopics(project);
+      return existingTopics.contains(topic);
+    } catch (IOException e) {
+      throw new RuntimeException("Unable to check Pubsub topic " + topic + ": ", e);
+    }
+  }
+
+  /**
+   * Create subscription from short name. Delete subscription if it already exists. Ensure the
+   * subscription will be deleted on cleanup. Return full subscription name.
+   */
+  public PubsubClient.SubscriptionPath createSubscription(
+      String shortTopic, String shortSubscription) {
+    PubsubClient.TopicPath topic = PubsubClient.topicPathFromName(projectId, shortTopic);
+    PubsubClient.SubscriptionPath subscription =
+        PubsubClient.subscriptionPathFromName(projectId, shortSubscription);
+    try {
+      if (subscriptionExists(shortTopic, shortSubscription)) {
+        NexmarkUtils.console("attempting to cleanup subscription %s", subscription);
+        pubsubClient.deleteSubscription(subscription);
+      }
+      NexmarkUtils.console("create subscription %s", subscription);
+      pubsubClient.createSubscription(topic, subscription, 60);
+      createdSubscriptions.add(subscription);
+    } catch (IOException e) {
+      throw new RuntimeException("Unable to create Pubsub subscription " + subscription + ": ", e);
+    }
+    return subscription;
+  }
+
+  /**
+   * Check a subscription corresponding to short name exists, and throw exception if not. The
+   * subscription will not be deleted on cleanup. Return full topic name.
+   */
+  public PubsubClient.SubscriptionPath reuseSubscription(
+      String shortTopic, String shortSubscription) {
+    PubsubClient.SubscriptionPath subscription =
+        PubsubClient.subscriptionPathFromName(projectId, shortSubscription);
+    if (subscriptionExists(shortTopic, shortSubscription)) {
+      NexmarkUtils.console("reusing existing subscription %s", subscription);
+      return subscription;
+    }
+    throw new RuntimeException("subscription'" + subscription + "' does not already exist");
+  }
+
+  /**
+   * Does subscription corresponding to short name exist?
+   */
+  public boolean subscriptionExists(String shortTopic, String shortSubscription) {
+    PubsubClient.ProjectPath project = PubsubClient.projectPathFromId(projectId);
+    PubsubClient.TopicPath topic = PubsubClient.topicPathFromName(projectId, shortTopic);
+    PubsubClient.SubscriptionPath subscription =
+        PubsubClient.subscriptionPathFromName(projectId, shortSubscription);
+    try {
+      Collection<PubsubClient.SubscriptionPath> existingSubscriptions =
+          pubsubClient.listSubscriptions(project, topic);
+      return existingSubscriptions.contains(subscription);
+    } catch (IOException e) {
+      throw new RuntimeException("Unable to check Pubsub subscription" + subscription + ": ", e);
+    }
+  }
+
+  /**
+   * Delete all the subscriptions and topics we created.
+   */
+  @Override
+  public void close() {
+    for (PubsubClient.SubscriptionPath subscription : createdSubscriptions) {
+      try {
+        NexmarkUtils.console("delete subscription %s", subscription);
+        pubsubClient.deleteSubscription(subscription);
+      } catch (IOException ex) {
+        NexmarkUtils.console("could not delete subscription %s", subscription);
+      }
+    }
+    for (PubsubClient.TopicPath topic : createdTopics) {
+      try {
+        NexmarkUtils.console("delete topic %s", topic);
+        pubsubClient.deleteTopic(topic);
+      } catch (IOException ex) {
+        NexmarkUtils.console("could not delete topic %s", topic);
+      }
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/a7f9f7d0/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/io/package-info.java
----------------------------------------------------------------------
diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/io/package-info.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/io/package-info.java
new file mode 100644
index 0000000..1161f3e
--- /dev/null
+++ b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/io/package-info.java
@@ -0,0 +1,22 @@
+/*
+ * 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.
+ */
+
+/**
+ * Nexmark Beam IO related utilities.
+ */
+package org.apache.beam.integration.nexmark.io;

http://git-wip-us.apache.org/repos/asf/beam/blob/a7f9f7d0/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/model/Auction.java
----------------------------------------------------------------------
diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/model/Auction.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/model/Auction.java
new file mode 100644
index 0000000..ac30568
--- /dev/null
+++ b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/model/Auction.java
@@ -0,0 +1,190 @@
+/*
+ * 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.beam.integration.nexmark.model;
+
+import com.fasterxml.jackson.annotation.JsonProperty;
+import com.fasterxml.jackson.core.JsonProcessingException;
+
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
+import java.io.Serializable;
+
+import org.apache.beam.integration.nexmark.NexmarkUtils;
+import org.apache.beam.sdk.coders.AtomicCoder;
+import org.apache.beam.sdk.coders.Coder;
+import org.apache.beam.sdk.coders.CoderException;
+import org.apache.beam.sdk.coders.StringUtf8Coder;
+import org.apache.beam.sdk.coders.VarLongCoder;
+
+/**
+ * An auction submitted by a person.
+ */
+public class Auction implements KnownSize, Serializable {
+  private static final Coder<Long> LONG_CODER = VarLongCoder.of();
+  private static final Coder<String> STRING_CODER = StringUtf8Coder.of();
+
+  public static final Coder<Auction> CODER = new AtomicCoder<Auction>() {
+    @Override
+    public void encode(Auction value, OutputStream outStream,
+        Coder.Context context)
+        throws CoderException, IOException {
+      LONG_CODER.encode(value.id, outStream, Context.NESTED);
+      STRING_CODER.encode(value.itemName, outStream, Context.NESTED);
+      STRING_CODER.encode(value.description, outStream, Context.NESTED);
+      LONG_CODER.encode(value.initialBid, outStream, Context.NESTED);
+      LONG_CODER.encode(value.reserve, outStream, Context.NESTED);
+      LONG_CODER.encode(value.dateTime, outStream, Context.NESTED);
+      LONG_CODER.encode(value.expires, outStream, Context.NESTED);
+      LONG_CODER.encode(value.seller, outStream, Context.NESTED);
+      LONG_CODER.encode(value.category, outStream, Context.NESTED);
+      STRING_CODER.encode(value.extra, outStream, Context.NESTED);
+    }
+
+    @Override
+    public Auction decode(
+        InputStream inStream, Coder.Context context)
+        throws CoderException, IOException {
+      long id = LONG_CODER.decode(inStream, Context.NESTED);
+      String itemName = STRING_CODER.decode(inStream, Context.NESTED);
+      String description = STRING_CODER.decode(inStream, Context.NESTED);
+      long initialBid = LONG_CODER.decode(inStream, Context.NESTED);
+      long reserve = LONG_CODER.decode(inStream, Context.NESTED);
+      long dateTime = LONG_CODER.decode(inStream, Context.NESTED);
+      long expires = LONG_CODER.decode(inStream, Context.NESTED);
+      long seller = LONG_CODER.decode(inStream, Context.NESTED);
+      long category = LONG_CODER.decode(inStream, Context.NESTED);
+      String extra = STRING_CODER.decode(inStream, Context.NESTED);
+      return new Auction(
+          id, itemName, description, initialBid, reserve, dateTime, expires, seller, category,
+          extra);
+    }
+  };
+
+
+  /** Id of auction. */
+  @JsonProperty
+  public final long id; // primary key
+
+  /** Extra auction properties. */
+  @JsonProperty
+  public final String itemName;
+
+  @JsonProperty
+  public final String description;
+
+  /** Initial bid price, in cents. */
+  @JsonProperty
+  public final long initialBid;
+
+  /** Reserve price, in cents. */
+  @JsonProperty
+  public final long reserve;
+
+  @JsonProperty
+  public final long dateTime;
+
+  /** When does auction expire? (ms since epoch). Bids at or after this time are ignored. */
+  @JsonProperty
+  public final long expires;
+
+  /** Id of person who instigated auction. */
+  @JsonProperty
+  public final long seller; // foreign key: Person.id
+
+  /** Id of category auction is listed under. */
+  @JsonProperty
+  public final long category; // foreign key: Category.id
+
+  /** Additional arbitrary payload for performance testing. */
+  @JsonProperty
+  public final String extra;
+
+
+  // For Avro only.
+  @SuppressWarnings("unused")
+  private Auction() {
+    id = 0;
+    itemName = null;
+    description = null;
+    initialBid = 0;
+    reserve = 0;
+    dateTime = 0;
+    expires = 0;
+    seller = 0;
+    category = 0;
+    extra = null;
+  }
+
+  public Auction(long id, String itemName, String description, long initialBid, long reserve,
+      long dateTime, long expires, long seller, long category, String extra) {
+    this.id = id;
+    this.itemName = itemName;
+    this.description = description;
+    this.initialBid = initialBid;
+    this.reserve = reserve;
+    this.dateTime = dateTime;
+    this.expires = expires;
+    this.seller = seller;
+    this.category = category;
+    this.extra = extra;
+  }
+
+  /**
+   * Return a copy of auction which capture the given annotation.
+   * (Used for debugging).
+   */
+  public Auction withAnnotation(String annotation) {
+    return new Auction(id, itemName, description, initialBid, reserve, dateTime, expires, seller,
+        category, annotation + ": " + extra);
+  }
+
+  /**
+   * Does auction have {@code annotation}? (Used for debugging.)
+   */
+  public boolean hasAnnotation(String annotation) {
+    return extra.startsWith(annotation + ": ");
+  }
+
+  /**
+   * Remove {@code annotation} from auction. (Used for debugging.)
+   */
+  public Auction withoutAnnotation(String annotation) {
+    if (hasAnnotation(annotation)) {
+      return new Auction(id, itemName, description, initialBid, reserve, dateTime, expires, seller,
+          category, extra.substring(annotation.length() + 2));
+    } else {
+      return this;
+    }
+  }
+
+  @Override
+  public long sizeInBytes() {
+    return 8 + itemName.length() + 1 + description.length() + 1 + 8 + 8 + 8 + 8 + 8 + 8
+        + extra.length() + 1;
+  }
+
+  @Override
+  public String toString() {
+    try {
+      return NexmarkUtils.MAPPER.writeValueAsString(this);
+    } catch (JsonProcessingException e) {
+      throw new RuntimeException(e);
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/a7f9f7d0/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/model/AuctionBid.java
----------------------------------------------------------------------
diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/model/AuctionBid.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/model/AuctionBid.java
new file mode 100644
index 0000000..c014257
--- /dev/null
+++ b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/model/AuctionBid.java
@@ -0,0 +1,88 @@
+/*
+ * 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.beam.integration.nexmark.model;
+
+import com.fasterxml.jackson.annotation.JsonProperty;
+import com.fasterxml.jackson.core.JsonProcessingException;
+
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
+import java.io.Serializable;
+
+import org.apache.beam.integration.nexmark.NexmarkUtils;
+import org.apache.beam.integration.nexmark.WinningBids;
+import org.apache.beam.sdk.coders.AtomicCoder;
+import org.apache.beam.sdk.coders.Coder;
+import org.apache.beam.sdk.coders.CoderException;
+
+/**
+ * Result of {@link WinningBids} transform.
+ */
+public class AuctionBid implements KnownSize, Serializable {
+  public static final Coder<AuctionBid> CODER = new AtomicCoder<AuctionBid>() {
+    @Override
+    public void encode(AuctionBid value, OutputStream outStream,
+        Coder.Context context)
+        throws CoderException, IOException {
+      Auction.CODER.encode(value.auction, outStream, Context.NESTED);
+      Bid.CODER.encode(value.bid, outStream, Context.NESTED);
+    }
+
+    @Override
+    public AuctionBid decode(
+        InputStream inStream, Coder.Context context)
+        throws CoderException, IOException {
+      Auction auction = Auction.CODER.decode(inStream, Context.NESTED);
+      Bid bid = Bid.CODER.decode(inStream, Context.NESTED);
+      return new AuctionBid(auction, bid);
+    }
+  };
+
+  @JsonProperty
+  public final Auction auction;
+
+  @JsonProperty
+  public final Bid bid;
+
+  // For Avro only.
+  @SuppressWarnings("unused")
+  private AuctionBid() {
+    auction = null;
+    bid = null;
+  }
+
+  public AuctionBid(Auction auction, Bid bid) {
+    this.auction = auction;
+    this.bid = bid;
+  }
+
+  @Override
+  public long sizeInBytes() {
+    return auction.sizeInBytes() + bid.sizeInBytes();
+  }
+
+  @Override
+  public String toString() {
+    try {
+      return NexmarkUtils.MAPPER.writeValueAsString(this);
+    } catch (JsonProcessingException e) {
+      throw new RuntimeException(e);
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/a7f9f7d0/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/model/AuctionCount.java
----------------------------------------------------------------------
diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/model/AuctionCount.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/model/AuctionCount.java
new file mode 100644
index 0000000..aa16629
--- /dev/null
+++ b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/model/AuctionCount.java
@@ -0,0 +1,90 @@
+/*
+ * 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.beam.integration.nexmark.model;
+
+import com.fasterxml.jackson.annotation.JsonProperty;
+import com.fasterxml.jackson.core.JsonProcessingException;
+
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
+import java.io.Serializable;
+
+import org.apache.beam.integration.nexmark.NexmarkUtils;
+import org.apache.beam.sdk.coders.AtomicCoder;
+import org.apache.beam.sdk.coders.Coder;
+import org.apache.beam.sdk.coders.CoderException;
+import org.apache.beam.sdk.coders.VarLongCoder;
+
+/**
+ * Result of Query5.
+ */
+public class AuctionCount implements KnownSize, Serializable {
+  private static final Coder<Long> LONG_CODER = VarLongCoder.of();
+
+  public static final Coder<AuctionCount> CODER = new AtomicCoder<AuctionCount>() {
+    @Override
+    public void encode(AuctionCount value, OutputStream outStream,
+        Coder.Context context)
+        throws CoderException, IOException {
+      LONG_CODER.encode(value.auction, outStream, Context.NESTED);
+      LONG_CODER.encode(value.count, outStream, Context.NESTED);
+    }
+
+    @Override
+    public AuctionCount decode(
+        InputStream inStream, Coder.Context context)
+        throws CoderException, IOException {
+      long auction = LONG_CODER.decode(inStream, Context.NESTED);
+      long count = LONG_CODER.decode(inStream, Context.NESTED);
+      return new AuctionCount(auction, count);
+    }
+  };
+
+  @JsonProperty
+  public final long auction;
+
+  @JsonProperty
+  public final long count;
+
+  // For Avro only.
+  @SuppressWarnings("unused")
+  private AuctionCount() {
+    auction = 0;
+    count = 0;
+  }
+
+  public AuctionCount(long auction, long count) {
+    this.auction = auction;
+    this.count = count;
+  }
+
+  @Override
+  public long sizeInBytes() {
+    return 8 + 8;
+  }
+
+  @Override
+  public String toString() {
+    try {
+      return NexmarkUtils.MAPPER.writeValueAsString(this);
+    } catch (JsonProcessingException e) {
+      throw new RuntimeException(e);
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/a7f9f7d0/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/model/AuctionPrice.java
----------------------------------------------------------------------
diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/model/AuctionPrice.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/model/AuctionPrice.java
new file mode 100644
index 0000000..f365cc8
--- /dev/null
+++ b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/model/AuctionPrice.java
@@ -0,0 +1,91 @@
+/*
+ * 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.beam.integration.nexmark.model;
+
+import com.fasterxml.jackson.annotation.JsonProperty;
+import com.fasterxml.jackson.core.JsonProcessingException;
+
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
+import java.io.Serializable;
+
+import org.apache.beam.integration.nexmark.NexmarkUtils;
+import org.apache.beam.sdk.coders.AtomicCoder;
+import org.apache.beam.sdk.coders.Coder;
+import org.apache.beam.sdk.coders.CoderException;
+import org.apache.beam.sdk.coders.VarLongCoder;
+
+/**
+ * Result of Query2.
+ */
+public class AuctionPrice implements KnownSize, Serializable {
+  private static final Coder<Long> LONG_CODER = VarLongCoder.of();
+
+  public static final Coder<AuctionPrice> CODER = new AtomicCoder<AuctionPrice>() {
+    @Override
+    public void encode(AuctionPrice value, OutputStream outStream,
+        Coder.Context context)
+        throws CoderException, IOException {
+      LONG_CODER.encode(value.auction, outStream, Context.NESTED);
+      LONG_CODER.encode(value.price, outStream, Context.NESTED);
+    }
+
+    @Override
+    public AuctionPrice decode(
+        InputStream inStream, Coder.Context context)
+        throws CoderException, IOException {
+      long auction = LONG_CODER.decode(inStream, Context.NESTED);
+      long price = LONG_CODER.decode(inStream, Context.NESTED);
+      return new AuctionPrice(auction, price);
+    }
+  };
+
+  @JsonProperty
+  public final long auction;
+
+  /** Price in cents. */
+  @JsonProperty
+  public final long price;
+
+  // For Avro only.
+  @SuppressWarnings("unused")
+  private AuctionPrice() {
+    auction = 0;
+    price = 0;
+  }
+
+  public AuctionPrice(long auction, long price) {
+    this.auction = auction;
+    this.price = price;
+  }
+
+  @Override
+  public long sizeInBytes() {
+    return 8 + 8;
+  }
+
+  @Override
+  public String toString() {
+    try {
+      return NexmarkUtils.MAPPER.writeValueAsString(this);
+    } catch (JsonProcessingException e) {
+      throw new RuntimeException(e);
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/a7f9f7d0/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/model/Bid.java
----------------------------------------------------------------------
diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/model/Bid.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/model/Bid.java
new file mode 100644
index 0000000..59a33c1
--- /dev/null
+++ b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/model/Bid.java
@@ -0,0 +1,178 @@
+/*
+ * 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.beam.integration.nexmark.model;
+
+import com.fasterxml.jackson.annotation.JsonProperty;
+import com.fasterxml.jackson.core.JsonProcessingException;
+
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
+import java.io.Serializable;
+import java.util.Comparator;
+
+import org.apache.beam.integration.nexmark.NexmarkUtils;
+import org.apache.beam.sdk.coders.AtomicCoder;
+import org.apache.beam.sdk.coders.Coder;
+import org.apache.beam.sdk.coders.CoderException;
+import org.apache.beam.sdk.coders.StringUtf8Coder;
+import org.apache.beam.sdk.coders.VarLongCoder;
+
+/**
+ * A bid for an item on auction.
+ */
+public class Bid implements KnownSize, Serializable {
+  private static final Coder<Long> LONG_CODER = VarLongCoder.of();
+  private static final Coder<String> STRING_CODER = StringUtf8Coder.of();
+
+  public static final Coder<Bid> CODER = new AtomicCoder<Bid>() {
+    @Override
+    public void encode(Bid value, OutputStream outStream,
+        Coder.Context context)
+        throws CoderException, IOException {
+      LONG_CODER.encode(value.auction, outStream, Context.NESTED);
+      LONG_CODER.encode(value.bidder, outStream, Context.NESTED);
+      LONG_CODER.encode(value.price, outStream, Context.NESTED);
+      LONG_CODER.encode(value.dateTime, outStream, Context.NESTED);
+      STRING_CODER.encode(value.extra, outStream, Context.NESTED);
+    }
+
+    @Override
+    public Bid decode(
+        InputStream inStream, Coder.Context context)
+        throws CoderException, IOException {
+      long auction = LONG_CODER.decode(inStream, Context.NESTED);
+      long bidder = LONG_CODER.decode(inStream, Context.NESTED);
+      long price = LONG_CODER.decode(inStream, Context.NESTED);
+      long dateTime = LONG_CODER.decode(inStream, Context.NESTED);
+      String extra = STRING_CODER.decode(inStream, Context.NESTED);
+      return new Bid(auction, bidder, price, dateTime, extra);
+    }
+  };
+
+  /**
+   * Comparator to order bids by ascending price then descending time
+   * (for finding winning bids).
+   */
+  public static final Comparator<Bid> PRICE_THEN_DESCENDING_TIME = new Comparator<Bid>() {
+    @Override
+    public int compare(Bid left, Bid right) {
+      int i = Double.compare(left.price, right.price);
+      if (i != 0) {
+        return i;
+      }
+      return Long.compare(right.dateTime, left.dateTime);
+    }
+  };
+
+  /**
+   * Comparator to order bids by ascending time then ascending price.
+   * (for finding most recent bids).
+   */
+  public static final Comparator<Bid> ASCENDING_TIME_THEN_PRICE = new Comparator<Bid>() {
+    @Override
+    public int compare(Bid left, Bid right) {
+      int i = Long.compare(left.dateTime, right.dateTime);
+      if (i != 0) {
+        return i;
+      }
+      return Double.compare(left.price, right.price);
+    }
+  };
+
+  /** Id of auction this bid is for. */
+  @JsonProperty
+  public final long auction; // foreign key: Auction.id
+
+  /** Id of person bidding in auction. */
+  @JsonProperty
+  public final long bidder; // foreign key: Person.id
+
+  /** Price of bid, in cents. */
+  @JsonProperty
+  public final long price;
+
+  /**
+   * Instant at which bid was made (ms since epoch).
+   * NOTE: This may be earlier than the system's event time.
+   */
+  @JsonProperty
+  public final long dateTime;
+
+  /** Additional arbitrary payload for performance testing. */
+  @JsonProperty
+  public final String extra;
+
+  // For Avro only.
+  @SuppressWarnings("unused")
+  private Bid() {
+    auction = 0;
+    bidder = 0;
+    price = 0;
+    dateTime = 0;
+    extra = null;
+  }
+
+  public Bid(long auction, long bidder, long price, long dateTime, String extra) {
+    this.auction = auction;
+    this.bidder = bidder;
+    this.price = price;
+    this.dateTime = dateTime;
+    this.extra = extra;
+  }
+
+  /**
+   * Return a copy of bid which capture the given annotation.
+   * (Used for debugging).
+   */
+  public Bid withAnnotation(String annotation) {
+    return new Bid(auction, bidder, price, dateTime, annotation + ": " + extra);
+  }
+
+  /**
+   * Does bid have {@code annotation}? (Used for debugging.)
+   */
+  public boolean hasAnnotation(String annotation) {
+    return extra.startsWith(annotation + ": ");
+  }
+
+  /**
+   * Remove {@code annotation} from bid. (Used for debugging.)
+   */
+  public Bid withoutAnnotation(String annotation) {
+    if (hasAnnotation(annotation)) {
+      return new Bid(auction, bidder, price, dateTime, extra.substring(annotation.length() + 2));
+    } else {
+      return this;
+    }
+  }
+
+  @Override
+  public long sizeInBytes() {
+    return 8 + 8 + 8 + 8 + extra.length() + 1;
+  }
+
+  @Override
+  public String toString() {
+    try {
+      return NexmarkUtils.MAPPER.writeValueAsString(this);
+    } catch (JsonProcessingException e) {
+      throw new RuntimeException(e);
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/a7f9f7d0/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/model/BidsPerSession.java
----------------------------------------------------------------------
diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/model/BidsPerSession.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/model/BidsPerSession.java
new file mode 100644
index 0000000..7c4dfae
--- /dev/null
+++ b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/model/BidsPerSession.java
@@ -0,0 +1,89 @@
+/*
+ * 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.beam.integration.nexmark.model;
+
+import com.fasterxml.jackson.annotation.JsonProperty;
+import com.fasterxml.jackson.core.JsonProcessingException;
+
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
+import java.io.Serializable;
+
+import org.apache.beam.integration.nexmark.NexmarkUtils;
+import org.apache.beam.sdk.coders.AtomicCoder;
+import org.apache.beam.sdk.coders.Coder;
+import org.apache.beam.sdk.coders.CoderException;
+import org.apache.beam.sdk.coders.VarLongCoder;
+
+/**
+ * Result of query 11.
+ */
+public class BidsPerSession implements KnownSize, Serializable {
+  private static final Coder<Long> LONG_CODER = VarLongCoder.of();
+
+  public static final Coder<BidsPerSession> CODER = new AtomicCoder<BidsPerSession>() {
+    @Override
+    public void encode(BidsPerSession value, OutputStream outStream,
+        Coder.Context context)
+        throws CoderException, IOException {
+      LONG_CODER.encode(value.personId, outStream, Context.NESTED);
+      LONG_CODER.encode(value.bidsPerSession, outStream, Context.NESTED);
+    }
+
+    @Override
+    public BidsPerSession decode(
+        InputStream inStream, Coder.Context context)
+        throws CoderException, IOException {
+      long personId = LONG_CODER.decode(inStream, Context.NESTED);
+      long bidsPerSession = LONG_CODER.decode(inStream, Context.NESTED);
+      return new BidsPerSession(personId, bidsPerSession);
+    }
+  };
+
+  @JsonProperty
+  public final long personId;
+
+  @JsonProperty
+  public final long bidsPerSession;
+
+  public BidsPerSession() {
+    personId = 0;
+    bidsPerSession = 0;
+  }
+
+  public BidsPerSession(long personId, long bidsPerSession) {
+    this.personId = personId;
+    this.bidsPerSession = bidsPerSession;
+  }
+
+  @Override
+  public long sizeInBytes() {
+    // Two longs.
+    return 8 + 8;
+  }
+
+  @Override
+  public String toString() {
+    try {
+      return NexmarkUtils.MAPPER.writeValueAsString(this);
+    } catch (JsonProcessingException e) {
+      throw new RuntimeException(e);
+    }
+  }
+}


[15/55] [abbrv] beam git commit: Refactor classes into packages

Posted by ie...@apache.org.
http://git-wip-us.apache.org/repos/asf/beam/blob/a7f9f7d0/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/Generator.java
----------------------------------------------------------------------
diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/Generator.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/Generator.java
deleted file mode 100644
index 7adb1b2..0000000
--- a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/Generator.java
+++ /dev/null
@@ -1,589 +0,0 @@
-/*
- * 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.beam.integration.nexmark;
-
-import static com.google.common.base.Preconditions.checkNotNull;
-
-import java.io.IOException;
-import java.io.InputStream;
-import java.io.OutputStream;
-import java.io.Serializable;
-import java.util.Arrays;
-import java.util.Iterator;
-import java.util.List;
-import java.util.Random;
-
-import org.apache.beam.sdk.coders.AtomicCoder;
-import org.apache.beam.sdk.coders.Coder;
-import org.apache.beam.sdk.coders.CoderException;
-import org.apache.beam.sdk.coders.VarLongCoder;
-import org.apache.beam.sdk.io.UnboundedSource;
-import org.apache.beam.sdk.values.TimestampedValue;
-import org.joda.time.Instant;
-
-/**
- * A generator for synthetic events. We try to make the data vaguely reasonable. We also ensure
- * most primary key/foreign key relations are correct. Eg: a {@link Bid} event will usually have
- * valid auction and bidder ids which can be joined to already-generated Auction and Person events.
- *
- * <p>To help with testing, we generate timestamps relative to a given {@code baseTime}. Each new
- * event is given a timestamp advanced from the previous timestamp by {@code interEventDelayUs}
- * (in microseconds). The event stream is thus fully deterministic and does not depend on
- * wallclock time.
- *
- * <p>This class implements {@link org.apache.beam.sdk.io.UnboundedSource.CheckpointMark}
- * so that we can resume generating events from a saved snapshot.
- */
-public class Generator implements Iterator<TimestampedValue<Event>>, Serializable {
-  /**
-   * Keep the number of categories small so the example queries will find results even with
-   * a small batch of events.
-   */
-  private static final int NUM_CATEGORIES = 5;
-
-  /** Smallest random string size. */
-  private static final int MIN_STRING_LENGTH = 3;
-
-  /**
-   * Keep the number of states small so that the example queries will find results even with
-   * a small batch of events.
-   */
-  private static final List<String> US_STATES = Arrays.asList(("AZ,CA,ID,OR,WA,WY").split(","));
-
-  private static final List<String> US_CITIES =
-      Arrays.asList(
-          ("Phoenix,Los Angeles,San Francisco,Boise,Portland,Bend,Redmond,Seattle,Kent,Cheyenne")
-              .split(","));
-
-  private static final List<String> FIRST_NAMES =
-      Arrays.asList(("Peter,Paul,Luke,John,Saul,Vicky,Kate,Julie,Sarah,Deiter,Walter").split(","));
-
-  private static final List<String> LAST_NAMES =
-      Arrays.asList(("Shultz,Abrams,Spencer,White,Bartels,Walton,Smith,Jones,Noris").split(","));
-
-  /**
-   * Number of yet-to-be-created people and auction ids allowed.
-   */
-  private static final int PERSON_ID_LEAD = 10;
-  private static final int AUCTION_ID_LEAD = 10;
-
-  /**
-   * Fraction of people/auctions which may be 'hot' sellers/bidders/auctions are 1
-   * over these values.
-   */
-  private static final int HOT_AUCTION_RATIO = 100;
-  private static final int HOT_SELLER_RATIO = 100;
-  private static final int HOT_BIDDER_RATIO = 100;
-
-  /**
-   * Just enough state to be able to restore a generator back to where it was checkpointed.
-   */
-  public static class Checkpoint implements UnboundedSource.CheckpointMark {
-    private static final Coder<Long> LONG_CODER = VarLongCoder.of();
-
-    /** Coder for this class. */
-    public static final Coder<Checkpoint> CODER_INSTANCE =
-        new AtomicCoder<Checkpoint>() {
-          @Override
-          public void encode(
-              Checkpoint value,
-              OutputStream outStream,
-              Coder.Context context)
-              throws CoderException, IOException {
-            LONG_CODER.encode(value.numEvents, outStream, Context.NESTED);
-            LONG_CODER.encode(value.wallclockBaseTime, outStream, Context.NESTED);
-          }
-
-          @Override
-          public Checkpoint decode(
-              InputStream inStream, Coder.Context context)
-              throws CoderException, IOException {
-            long numEvents = LONG_CODER.decode(inStream, Context.NESTED);
-            long wallclockBaseTime = LONG_CODER.decode(inStream, Context.NESTED);
-            return new Checkpoint(numEvents, wallclockBaseTime);
-          }
-        };
-
-    private long numEvents;
-    private long wallclockBaseTime;
-
-    private Checkpoint(long numEvents, long wallclockBaseTime) {
-      this.numEvents = numEvents;
-      this.wallclockBaseTime = wallclockBaseTime;
-    }
-
-    public Generator toGenerator(GeneratorConfig config) {
-      return new Generator(config, numEvents, wallclockBaseTime);
-    }
-
-    @Override
-    public void finalizeCheckpoint() throws IOException {
-      // Nothing to finalize.
-    }
-
-    @Override
-    public String toString() {
-      return String.format("Generator.Checkpoint{numEvents:%d;wallclockBaseTime:%d}",
-          numEvents, wallclockBaseTime);
-    }
-  }
-
-  /**
-   * The next event and its various timestamps. Ordered by increasing wallclock timestamp, then
-   * (arbitrary but stable) event hash order.
-   */
-  public static class NextEvent implements Comparable<NextEvent> {
-    /** When, in wallclock time, should this event be emitted? */
-    public final long wallclockTimestamp;
-
-    /** When, in event time, should this event be considered to have occured? */
-    public final long eventTimestamp;
-
-    /** The event itself. */
-    public final Event event;
-
-    /** The minimum of this and all future event timestamps. */
-    public final long watermark;
-
-    public NextEvent(long wallclockTimestamp, long eventTimestamp, Event event, long watermark) {
-      this.wallclockTimestamp = wallclockTimestamp;
-      this.eventTimestamp = eventTimestamp;
-      this.event = event;
-      this.watermark = watermark;
-    }
-
-    /**
-     * Return a deep clone of next event with delay added to wallclock timestamp and
-     * event annotate as 'LATE'.
-     */
-    public NextEvent withDelay(long delayMs) {
-      return new NextEvent(
-          wallclockTimestamp + delayMs, eventTimestamp, event.withAnnotation("LATE"), watermark);
-    }
-
-    @Override
-    public int compareTo(NextEvent other) {
-      int i = Long.compare(wallclockTimestamp, other.wallclockTimestamp);
-      if (i != 0) {
-        return i;
-      }
-      return Integer.compare(event.hashCode(), other.event.hashCode());
-    }
-  }
-
-  /**
-   * Configuration to generate events against. Note that it may be replaced by a call to
-   * {@link #splitAtEventId}.
-   */
-  private GeneratorConfig config;
-
-  /** Number of events generated by this generator. */
-  private long numEvents;
-
-  /**
-   * Wallclock time at which we emitted the first event (ms since epoch). Initially -1.
-   */
-  private long wallclockBaseTime;
-
-  private Generator(GeneratorConfig config, long numEvents, long wallclockBaseTime) {
-    checkNotNull(config);
-    this.config = config;
-    this.numEvents = numEvents;
-    this.wallclockBaseTime = wallclockBaseTime;
-  }
-
-  /**
-   * Create a fresh generator according to {@code config}.
-   */
-  public Generator(GeneratorConfig config) {
-    this(config, 0, -1);
-  }
-
-  /**
-   * Return a checkpoint for the current generator.
-   */
-  public Checkpoint toCheckpoint() {
-    return new Checkpoint(numEvents, wallclockBaseTime);
-  }
-
-  /**
-   * Return a deep clone of this generator.
-   */
-  @Override
-  public Generator clone() {
-    return new Generator(config.clone(), numEvents, wallclockBaseTime);
-  }
-
-  /**
-   * Return the current config for this generator. Note that configs may be replaced by {@link
-   * #splitAtEventId}.
-   */
-  public GeneratorConfig getCurrentConfig() {
-    return config;
-  }
-
-  /**
-   * Mutate this generator so that it will only generate events up to but not including
-   * {@code eventId}. Return a config to represent the events this generator will no longer yield.
-   * The generators will run in on a serial timeline.
-   */
-  public GeneratorConfig splitAtEventId(long eventId) {
-    long newMaxEvents = eventId - (config.firstEventId + config.firstEventNumber);
-    GeneratorConfig remainConfig = config.cloneWith(config.firstEventId,
-        config.maxEvents - newMaxEvents, config.firstEventNumber + newMaxEvents);
-    config = config.cloneWith(config.firstEventId, newMaxEvents, config.firstEventNumber);
-    return remainConfig;
-  }
-
-  /**
-   * Return the next 'event id'. Though events don't have ids we can simulate them to
-   * help with bookkeeping.
-   */
-  public long getNextEventId() {
-    return config.firstEventId + config.nextAdjustedEventNumber(numEvents);
-  }
-
-  /**
-   * Return the last valid person id (ignoring FIRST_PERSON_ID). Will be the current person id if
-   * due to generate a person.
-   */
-  private long lastBase0PersonId() {
-    long eventId = getNextEventId();
-    long epoch = eventId / GeneratorConfig.PROPORTION_DENOMINATOR;
-    long offset = eventId % GeneratorConfig.PROPORTION_DENOMINATOR;
-    if (offset >= GeneratorConfig.PERSON_PROPORTION) {
-      // About to generate an auction or bid.
-      // Go back to the last person generated in this epoch.
-      offset = GeneratorConfig.PERSON_PROPORTION - 1;
-    }
-    // About to generate a person.
-    return epoch * GeneratorConfig.PERSON_PROPORTION + offset;
-  }
-
-  /**
-   * Return the last valid auction id (ignoring FIRST_AUCTION_ID). Will be the current auction id if
-   * due to generate an auction.
-   */
-  private long lastBase0AuctionId() {
-    long eventId = getNextEventId();
-    long epoch = eventId / GeneratorConfig.PROPORTION_DENOMINATOR;
-    long offset = eventId % GeneratorConfig.PROPORTION_DENOMINATOR;
-    if (offset < GeneratorConfig.PERSON_PROPORTION) {
-      // About to generate a person.
-      // Go back to the last auction in the last epoch.
-      epoch--;
-      offset = GeneratorConfig.AUCTION_PROPORTION - 1;
-    } else if (offset >= GeneratorConfig.PERSON_PROPORTION + GeneratorConfig.AUCTION_PROPORTION) {
-      // About to generate a bid.
-      // Go back to the last auction generated in this epoch.
-      offset = GeneratorConfig.AUCTION_PROPORTION - 1;
-    } else {
-      // About to generate an auction.
-      offset -= GeneratorConfig.PERSON_PROPORTION;
-    }
-    return epoch * GeneratorConfig.AUCTION_PROPORTION + offset;
-  }
-
-  /** return a random US state. */
-  private static String nextUSState(Random random) {
-    return US_STATES.get(random.nextInt(US_STATES.size()));
-  }
-
-  /** Return a random US city. */
-  private static String nextUSCity(Random random) {
-    return US_CITIES.get(random.nextInt(US_CITIES.size()));
-  }
-
-  /** Return a random person name. */
-  private static String nextPersonName(Random random) {
-    return FIRST_NAMES.get(random.nextInt(FIRST_NAMES.size())) + " "
-        + LAST_NAMES.get(random.nextInt(LAST_NAMES.size()));
-  }
-
-  /** Return a random string of up to {@code maxLength}. */
-  private static String nextString(Random random, int maxLength) {
-    int len = MIN_STRING_LENGTH + random.nextInt(maxLength - MIN_STRING_LENGTH);
-    StringBuilder sb = new StringBuilder();
-    while (len-- > 0) {
-      if (random.nextInt(13) == 0) {
-        sb.append(' ');
-      } else {
-        sb.append((char) ('a' + random.nextInt(26)));
-      }
-    }
-    return sb.toString().trim();
-  }
-
-  /** Return a random string of exactly {@code length}. */
-  private static String nextExactString(Random random, int length) {
-    StringBuilder sb = new StringBuilder();
-    while (length-- > 0) {
-      sb.append((char) ('a' + random.nextInt(26)));
-    }
-    return sb.toString();
-  }
-
-  /** Return a random email address. */
-  private static String nextEmail(Random random) {
-    return nextString(random, 7) + "@" + nextString(random, 5) + ".com";
-  }
-
-  /** Return a random credit card number. */
-  private static String nextCreditCard(Random random) {
-    StringBuilder sb = new StringBuilder();
-    for (int i = 0; i < 4; i++) {
-      if (i > 0) {
-        sb.append(' ');
-      }
-      sb.append(String.format("%04d", random.nextInt(10000)));
-    }
-    return sb.toString();
-  }
-
-  /** Return a random price. */
-  private static long nextPrice(Random random) {
-    return Math.round(Math.pow(10.0, random.nextDouble() * 6.0) * 100.0);
-  }
-
-  /** Return a random time delay, in milliseconds, for length of auctions. */
-  private long nextAuctionLengthMs(Random random, long timestamp) {
-    // What's our current event number?
-    long currentEventNumber = config.nextAdjustedEventNumber(numEvents);
-    // How many events till we've generated numInFlightAuctions?
-    long numEventsForAuctions =
-        (config.configuration.numInFlightAuctions * GeneratorConfig.PROPORTION_DENOMINATOR)
-        / GeneratorConfig.AUCTION_PROPORTION;
-    // When will the auction numInFlightAuctions beyond now be generated?
-    long futureAuction =
-        config.timestampAndInterEventDelayUsForEvent(currentEventNumber + numEventsForAuctions)
-            .getKey();
-    // System.out.printf("*** auction will be for %dms (%d events ahead) ***\n",
-    //     futureAuction - timestamp, numEventsForAuctions);
-    // Choose a length with average horizonMs.
-    long horizonMs = futureAuction - timestamp;
-    return 1L + nextLong(random, Math.max(horizonMs * 2, 1L));
-  }
-
-  /**
-   * Return a random {@code string} such that {@code currentSize + string.length()} is on average
-   * {@code averageSize}.
-   */
-  private static String nextExtra(Random random, int currentSize, int desiredAverageSize) {
-    if (currentSize > desiredAverageSize) {
-      return "";
-    }
-    desiredAverageSize -= currentSize;
-    int delta = (int) Math.round(desiredAverageSize * 0.2);
-    int minSize = desiredAverageSize - delta;
-    int desiredSize = minSize + (delta == 0 ? 0 : random.nextInt(2 * delta));
-    return nextExactString(random, desiredSize);
-  }
-
-  /** Return a random long from {@code [0, n)}. */
-  private static long nextLong(Random random, long n) {
-    if (n < Integer.MAX_VALUE) {
-      return random.nextInt((int) n);
-    } else {
-      // TODO: Very skewed distribution! Bad!
-      return Math.abs(random.nextLong()) % n;
-    }
-  }
-
-  /**
-   * Generate and return a random person with next available id.
-   */
-  private Person nextPerson(Random random, long timestamp) {
-    long id = lastBase0PersonId() + GeneratorConfig.FIRST_PERSON_ID;
-    String name = nextPersonName(random);
-    String email = nextEmail(random);
-    String creditCard = nextCreditCard(random);
-    String city = nextUSCity(random);
-    String state = nextUSState(random);
-    int currentSize =
-        8 + name.length() + email.length() + creditCard.length() + city.length() + state.length();
-    String extra = nextExtra(random, currentSize, config.configuration.avgPersonByteSize);
-    return new Person(id, name, email, creditCard, city, state, timestamp, extra);
-  }
-
-  /**
-   * Return a random person id (base 0).
-   */
-  private long nextBase0PersonId(Random random) {
-    // Choose a random person from any of the 'active' people, plus a few 'leads'.
-    // By limiting to 'active' we ensure the density of bids or auctions per person
-    // does not decrease over time for long running jobs.
-    // By choosing a person id ahead of the last valid person id we will make
-    // newPerson and newAuction events appear to have been swapped in time.
-    long numPeople = lastBase0PersonId() + 1;
-    long activePeople = Math.min(numPeople, config.configuration.numActivePeople);
-    long n = nextLong(random, activePeople + PERSON_ID_LEAD);
-    return numPeople - activePeople + n;
-  }
-
-  /**
-   * Return a random auction id (base 0).
-   */
-  private long nextBase0AuctionId(Random random) {
-    // Choose a random auction for any of those which are likely to still be in flight,
-    // plus a few 'leads'.
-    // Note that ideally we'd track non-expired auctions exactly, but that state
-    // is difficult to split.
-    long minAuction = Math.max(lastBase0AuctionId() - config.configuration.numInFlightAuctions, 0);
-    long maxAuction = lastBase0AuctionId();
-    return minAuction + nextLong(random, maxAuction - minAuction + 1 + AUCTION_ID_LEAD);
-  }
-
-  /**
-   * Generate and return a random auction with next available id.
-   */
-  private Auction nextAuction(Random random, long timestamp) {
-    long id = lastBase0AuctionId() + GeneratorConfig.FIRST_AUCTION_ID;
-
-    long seller;
-    // Here P(auction will be for a hot seller) = 1 - 1/hotSellersRatio.
-    if (random.nextInt(config.configuration.hotSellersRatio) > 0) {
-      // Choose the first person in the batch of last HOT_SELLER_RATIO people.
-      seller = (lastBase0PersonId() / HOT_SELLER_RATIO) * HOT_SELLER_RATIO;
-    } else {
-      seller = nextBase0PersonId(random);
-    }
-    seller += GeneratorConfig.FIRST_PERSON_ID;
-
-    long category = GeneratorConfig.FIRST_CATEGORY_ID + random.nextInt(NUM_CATEGORIES);
-    long initialBid = nextPrice(random);
-    long dateTime = timestamp;
-    long expires = timestamp + nextAuctionLengthMs(random, timestamp);
-    String name = nextString(random, 20);
-    String desc = nextString(random, 100);
-    long reserve = initialBid + nextPrice(random);
-    int currentSize = 8 + name.length() + desc.length() + 8 + 8 + 8 + 8 + 8;
-    String extra = nextExtra(random, currentSize, config.configuration.avgAuctionByteSize);
-    return new Auction(id, name, desc, initialBid, reserve, dateTime, expires, seller, category,
-        extra);
-  }
-
-  /**
-   * Generate and return a random bid with next available id.
-   */
-  private Bid nextBid(Random random, long timestamp) {
-    long auction;
-    // Here P(bid will be for a hot auction) = 1 - 1/hotAuctionRatio.
-    if (random.nextInt(config.configuration.hotAuctionRatio) > 0) {
-      // Choose the first auction in the batch of last HOT_AUCTION_RATIO auctions.
-      auction = (lastBase0AuctionId() / HOT_AUCTION_RATIO) * HOT_AUCTION_RATIO;
-    } else {
-      auction = nextBase0AuctionId(random);
-    }
-    auction += GeneratorConfig.FIRST_AUCTION_ID;
-
-    long bidder;
-    // Here P(bid will be by a hot bidder) = 1 - 1/hotBiddersRatio
-    if (random.nextInt(config.configuration.hotBiddersRatio) > 0) {
-      // Choose the second person (so hot bidders and hot sellers don't collide) in the batch of
-      // last HOT_BIDDER_RATIO people.
-      bidder = (lastBase0PersonId() / HOT_BIDDER_RATIO) * HOT_BIDDER_RATIO + 1;
-    } else {
-      bidder = nextBase0PersonId(random);
-    }
-    bidder += GeneratorConfig.FIRST_PERSON_ID;
-
-    long price = nextPrice(random);
-    int currentSize = 8 + 8 + 8 + 8;
-    String extra = nextExtra(random, currentSize, config.configuration.avgBidByteSize);
-    return new Bid(auction, bidder, price, timestamp, extra);
-  }
-
-  @Override
-  public boolean hasNext() {
-    return numEvents < config.maxEvents;
-  }
-
-  /**
-   * Return the next event. The outer timestamp is in wallclock time and corresponds to
-   * when the event should fire. The inner timestamp is in event-time and represents the
-   * time the event is purported to have taken place in the simulation.
-   */
-  public NextEvent nextEvent() {
-    if (wallclockBaseTime < 0) {
-      wallclockBaseTime = System.currentTimeMillis();
-    }
-    // When, in event time, we should generate the event. Monotonic.
-    long eventTimestamp =
-        config.timestampAndInterEventDelayUsForEvent(config.nextEventNumber(numEvents)).getKey();
-    // When, in event time, the event should say it was generated. Depending on outOfOrderGroupSize
-    // may have local jitter.
-    long adjustedEventTimestamp =
-        config.timestampAndInterEventDelayUsForEvent(config.nextAdjustedEventNumber(numEvents))
-            .getKey();
-    // The minimum of this and all future adjusted event timestamps. Accounts for jitter in
-    // the event timestamp.
-    long watermark =
-        config.timestampAndInterEventDelayUsForEvent(config.nextEventNumberForWatermark(numEvents))
-            .getKey();
-    // When, in wallclock time, we should emit the event.
-    long wallclockTimestamp = wallclockBaseTime + (eventTimestamp - getCurrentConfig().baseTime);
-
-    // Seed the random number generator with the next 'event id'.
-    Random random = new Random(getNextEventId());
-    long rem = getNextEventId() % GeneratorConfig.PROPORTION_DENOMINATOR;
-
-    Event event;
-    if (rem < GeneratorConfig.PERSON_PROPORTION) {
-      event = new Event(nextPerson(random, adjustedEventTimestamp));
-    } else if (rem < GeneratorConfig.PERSON_PROPORTION + GeneratorConfig.AUCTION_PROPORTION) {
-      event = new Event(nextAuction(random, adjustedEventTimestamp));
-    } else {
-      event = new Event(nextBid(random, adjustedEventTimestamp));
-    }
-
-    numEvents++;
-    return new NextEvent(wallclockTimestamp, adjustedEventTimestamp, event, watermark);
-  }
-
-  @Override
-  public TimestampedValue<Event> next() {
-    NextEvent next = nextEvent();
-    return TimestampedValue.of(next.event, new Instant(next.eventTimestamp));
-  }
-
-  @Override
-  public void remove() {
-    throw new UnsupportedOperationException();
-  }
-
-  /**
-   * Return how many microseconds till we emit the next event.
-   */
-  public long currentInterEventDelayUs() {
-    return config.timestampAndInterEventDelayUsForEvent(config.nextEventNumber(numEvents))
-        .getValue();
-  }
-
-  /**
-   * Return an estimate of fraction of output consumed.
-   */
-  public double getFractionConsumed() {
-    return (double) numEvents / config.maxEvents;
-  }
-
-  @Override
-  public String toString() {
-    return String.format("Generator{config:%s; numEvents:%d; wallclockBaseTime:%d}", config,
-        numEvents, wallclockBaseTime);
-  }
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/a7f9f7d0/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/GeneratorConfig.java
----------------------------------------------------------------------
diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/GeneratorConfig.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/GeneratorConfig.java
deleted file mode 100644
index dceff4f..0000000
--- a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/GeneratorConfig.java
+++ /dev/null
@@ -1,294 +0,0 @@
-/*
- * 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.beam.integration.nexmark;
-
-import java.io.Serializable;
-import java.util.ArrayList;
-import java.util.List;
-
-import org.apache.beam.sdk.values.KV;
-
-/**
- * Parameters controlling how {@link Generator} synthesizes {@link Event} elements.
- */
-class GeneratorConfig implements Serializable {
-  /**
-   * We start the ids at specific values to help ensure the queries find a match even on
-   * small synthesized dataset sizes.
-   */
-  public static final long FIRST_AUCTION_ID = 1000L;
-  public static final long FIRST_PERSON_ID = 1000L;
-  public static final long FIRST_CATEGORY_ID = 10L;
-
-  /**
-   * Proportions of people/auctions/bids to synthesize.
-   */
-  public static final int PERSON_PROPORTION = 1;
-  public static final int AUCTION_PROPORTION = 3;
-  public static final int BID_PROPORTION = 46;
-  public static final int PROPORTION_DENOMINATOR =
-      PERSON_PROPORTION + AUCTION_PROPORTION + BID_PROPORTION;
-
-  /**
-   * Environment options.
-   */
-  public final NexmarkConfiguration configuration;
-
-  /**
-   * Delay between events, in microseconds. If the array has more than one entry then
-   * the rate is changed every {@link #stepLengthSec}, and wraps around.
-   */
-  public final long[] interEventDelayUs;
-
-  /**
-   * Delay before changing the current inter-event delay.
-   */
-  public final long stepLengthSec;
-
-  /**
-   * Time for first event (ms since epoch).
-   */
-  public final long baseTime;
-
-  /**
-   * Event id of first event to be generated. Event ids are unique over all generators, and
-   * are used as a seed to generate each event's data.
-   */
-  public final long firstEventId;
-
-  /**
-   * Maximum number of events to generate.
-   */
-  public final long maxEvents;
-
-  /**
-   * First event number. Generators running in parallel time may share the same event number,
-   * and the event number is used to determine the event timestamp.
-   */
-  public final long firstEventNumber;
-
-  /**
-   * True period of epoch in milliseconds. Derived from above.
-   * (Ie time to run through cycle for all interEventDelayUs entries).
-   */
-  public final long epochPeriodMs;
-
-  /**
-   * Number of events per epoch. Derived from above.
-   * (Ie number of events to run through cycle for all interEventDelayUs entries).
-   */
-  public final long eventsPerEpoch;
-
-  public GeneratorConfig(
-      NexmarkConfiguration configuration, long baseTime, long firstEventId,
-      long maxEventsOrZero, long firstEventNumber) {
-    this.configuration = configuration;
-    this.interEventDelayUs = configuration.rateShape.interEventDelayUs(
-        configuration.firstEventRate, configuration.nextEventRate,
-        configuration.rateUnit, configuration.numEventGenerators);
-    this.stepLengthSec = configuration.rateShape.stepLengthSec(configuration.ratePeriodSec);
-    this.baseTime = baseTime;
-    this.firstEventId = firstEventId;
-    if (maxEventsOrZero == 0) {
-      // Scale maximum down to avoid overflow in getEstimatedSizeBytes.
-      this.maxEvents =
-          Long.MAX_VALUE / (PROPORTION_DENOMINATOR
-                            * Math.max(
-              Math.max(configuration.avgPersonByteSize, configuration.avgAuctionByteSize),
-              configuration.avgBidByteSize));
-    } else {
-      this.maxEvents = maxEventsOrZero;
-    }
-    this.firstEventNumber = firstEventNumber;
-
-    long eventsPerEpoch = 0;
-    long epochPeriodMs = 0;
-    if (interEventDelayUs.length > 1) {
-      for (int i = 0; i < interEventDelayUs.length; i++) {
-        long numEventsForThisCycle = (stepLengthSec * 1_000_000L) / interEventDelayUs[i];
-        eventsPerEpoch += numEventsForThisCycle;
-        epochPeriodMs += (numEventsForThisCycle * interEventDelayUs[i]) / 1000L;
-      }
-    }
-    this.eventsPerEpoch = eventsPerEpoch;
-    this.epochPeriodMs = epochPeriodMs;
-  }
-
-  /**
-   * Return a clone of this config.
-   */
-  @Override
-  public GeneratorConfig clone() {
-    return new GeneratorConfig(configuration, baseTime, firstEventId, maxEvents, firstEventNumber);
-  }
-
-  /**
-   * Return clone of this config except with given parameters.
-   */
-  public GeneratorConfig cloneWith(long firstEventId, long maxEvents, long firstEventNumber) {
-    return new GeneratorConfig(configuration, baseTime, firstEventId, maxEvents, firstEventNumber);
-  }
-
-  /**
-   * Split this config into {@code n} sub-configs with roughly equal number of
-   * possible events, but distinct value spaces. The generators will run on parallel timelines.
-   * This config should no longer be used.
-   */
-  public List<GeneratorConfig> split(int n) {
-    List<GeneratorConfig> results = new ArrayList<>();
-    if (n == 1) {
-      // No split required.
-      results.add(this);
-    } else {
-      long subMaxEvents = maxEvents / n;
-      long subFirstEventId = firstEventId;
-      for (int i = 0; i < n; i++) {
-        if (i == n - 1) {
-          // Don't loose any events to round-down.
-          subMaxEvents = maxEvents - subMaxEvents * (n - 1);
-        }
-        results.add(cloneWith(subFirstEventId, subMaxEvents, firstEventNumber));
-        subFirstEventId += subMaxEvents;
-      }
-    }
-    return results;
-  }
-
-  /**
-   * Return an estimate of the bytes needed by {@code numEvents}.
-   */
-  public long estimatedBytesForEvents(long numEvents) {
-    long numPersons =
-        (numEvents * GeneratorConfig.PERSON_PROPORTION) / GeneratorConfig.PROPORTION_DENOMINATOR;
-    long numAuctions = (numEvents * AUCTION_PROPORTION) / PROPORTION_DENOMINATOR;
-    long numBids = (numEvents * BID_PROPORTION) / PROPORTION_DENOMINATOR;
-    return numPersons * configuration.avgPersonByteSize
-           + numAuctions * configuration.avgAuctionByteSize
-           + numBids * configuration.avgBidByteSize;
-  }
-
-  /**
-   * Return an estimate of the byte-size of all events a generator for this config would yield.
-   */
-  public long getEstimatedSizeBytes() {
-    return estimatedBytesForEvents(maxEvents);
-  }
-
-  /**
-   * Return the first 'event id' which could be generated from this config. Though events don't
-   * have ids we can simulate them to help bookkeeping.
-   */
-  public long getStartEventId() {
-    return firstEventId + firstEventNumber;
-  }
-
-  /**
-   * Return one past the last 'event id' which could be generated from this config.
-   */
-  public long getStopEventId() {
-    return firstEventId + firstEventNumber + maxEvents;
-  }
-
-  /**
-   * Return the next event number for a generator which has so far emitted {@code numEvents}.
-   */
-  public long nextEventNumber(long numEvents) {
-    return firstEventNumber + numEvents;
-  }
-
-  /**
-   * Return the next event number for a generator which has so far emitted {@code numEvents},
-   * but adjusted to account for {@code outOfOrderGroupSize}.
-   */
-  public long nextAdjustedEventNumber(long numEvents) {
-    long n = configuration.outOfOrderGroupSize;
-    long eventNumber = nextEventNumber(numEvents);
-    long base = (eventNumber / n) * n;
-    long offset = (eventNumber * 953) % n;
-    return base + offset;
-  }
-
-  /**
-   * Return the event number who's event time will be a suitable watermark for
-   * a generator which has so far emitted {@code numEvents}.
-   */
-  public long nextEventNumberForWatermark(long numEvents) {
-    long n = configuration.outOfOrderGroupSize;
-    long eventNumber = nextEventNumber(numEvents);
-    return (eventNumber / n) * n;
-  }
-
-  /**
-   * What timestamp should the event with {@code eventNumber} have for this generator? And
-   * what inter-event delay (in microseconds) is current?
-   */
-  public KV<Long, Long> timestampAndInterEventDelayUsForEvent(long eventNumber) {
-    if (interEventDelayUs.length == 1) {
-      long timestamp = baseTime + (eventNumber * interEventDelayUs[0]) / 1000L;
-      return KV.of(timestamp, interEventDelayUs[0]);
-    }
-
-    long epoch = eventNumber / eventsPerEpoch;
-    long n = eventNumber % eventsPerEpoch;
-    long offsetInEpochMs = 0;
-    for (int i = 0; i < interEventDelayUs.length; i++) {
-      long numEventsForThisCycle = (stepLengthSec * 1_000_000L) / interEventDelayUs[i];
-      if (n < numEventsForThisCycle) {
-        long offsetInCycleUs = n * interEventDelayUs[i];
-        long timestamp =
-            baseTime + epoch * epochPeriodMs + offsetInEpochMs + (offsetInCycleUs / 1000L);
-        return KV.of(timestamp, interEventDelayUs[i]);
-      }
-      n -= numEventsForThisCycle;
-      offsetInEpochMs += (numEventsForThisCycle * interEventDelayUs[i]) / 1000L;
-    }
-    throw new RuntimeException("internal eventsPerEpoch incorrect"); // can't reach
-  }
-
-  @Override
-  public String toString() {
-    StringBuilder sb = new StringBuilder();
-    sb.append("GeneratorConfig");
-    sb.append("{configuration:");
-    sb.append(configuration.toString());
-    sb.append(";interEventDelayUs=[");
-    for (int i = 0; i < interEventDelayUs.length; i++) {
-      if (i > 0) {
-        sb.append(",");
-      }
-      sb.append(interEventDelayUs[i]);
-    }
-    sb.append("]");
-    sb.append(";stepLengthSec:");
-    sb.append(stepLengthSec);
-    sb.append(";baseTime:");
-    sb.append(baseTime);
-    sb.append(";firstEventId:");
-    sb.append(firstEventId);
-    sb.append(";maxEvents:");
-    sb.append(maxEvents);
-    sb.append(";firstEventNumber:");
-    sb.append(firstEventNumber);
-    sb.append(";epochPeriodMs:");
-    sb.append(epochPeriodMs);
-    sb.append(";eventsPerEpoch:");
-    sb.append(eventsPerEpoch);
-    sb.append("}");
-    return sb.toString();
-  }
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/a7f9f7d0/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/IdNameReserve.java
----------------------------------------------------------------------
diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/IdNameReserve.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/IdNameReserve.java
deleted file mode 100644
index 21fa3f4..0000000
--- a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/IdNameReserve.java
+++ /dev/null
@@ -1,99 +0,0 @@
-/*
- * 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.beam.integration.nexmark;
-
-import com.fasterxml.jackson.annotation.JsonProperty;
-import com.fasterxml.jackson.core.JsonProcessingException;
-
-import java.io.IOException;
-import java.io.InputStream;
-import java.io.OutputStream;
-import java.io.Serializable;
-
-import org.apache.beam.sdk.coders.AtomicCoder;
-import org.apache.beam.sdk.coders.Coder;
-import org.apache.beam.sdk.coders.CoderException;
-import org.apache.beam.sdk.coders.StringUtf8Coder;
-import org.apache.beam.sdk.coders.VarLongCoder;
-
-/**
- * Result type of {@link Query8}.
- */
-public class IdNameReserve implements KnownSize, Serializable {
-  private static final Coder<Long> LONG_CODER = VarLongCoder.of();
-  private static final Coder<String> STRING_CODER = StringUtf8Coder.of();
-
-  public static final Coder<IdNameReserve> CODER = new AtomicCoder<IdNameReserve>() {
-    @Override
-    public void encode(IdNameReserve value, OutputStream outStream,
-        Coder.Context context)
-        throws CoderException, IOException {
-      LONG_CODER.encode(value.id, outStream, Context.NESTED);
-      STRING_CODER.encode(value.name, outStream, Context.NESTED);
-      LONG_CODER.encode(value.reserve, outStream, Context.NESTED);
-    }
-
-    @Override
-    public IdNameReserve decode(
-        InputStream inStream, Coder.Context context)
-        throws CoderException, IOException {
-      long id = LONG_CODER.decode(inStream, Context.NESTED);
-      String name = STRING_CODER.decode(inStream, Context.NESTED);
-      long reserve = LONG_CODER.decode(inStream, Context.NESTED);
-      return new IdNameReserve(id, name, reserve);
-    }
-  };
-
-  @JsonProperty
-  public final long id;
-
-  @JsonProperty
-  public final String name;
-
-  /** Reserve price in cents. */
-  @JsonProperty
-  public final long reserve;
-
-  // For Avro only.
-  @SuppressWarnings("unused")
-  private IdNameReserve() {
-    id = 0;
-    name = null;
-    reserve = 0;
-  }
-
-  public IdNameReserve(long id, String name, long reserve) {
-    this.id = id;
-    this.name = name;
-    this.reserve = reserve;
-  }
-
-  @Override
-  public long sizeInBytes() {
-    return 8 + name.length() + 1 + 8;
-  }
-
-  @Override
-  public String toString() {
-    try {
-      return NexmarkUtils.MAPPER.writeValueAsString(this);
-    } catch (JsonProcessingException e) {
-      throw new RuntimeException(e);
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/a7f9f7d0/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/KnownSize.java
----------------------------------------------------------------------
diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/KnownSize.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/KnownSize.java
deleted file mode 100644
index 2093c48..0000000
--- a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/KnownSize.java
+++ /dev/null
@@ -1,26 +0,0 @@
-/*
- * 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.beam.integration.nexmark;
-
-/**
- * Interface for elements which can quickly estimate their encoded byte size.
- */
-public interface KnownSize {
-  long sizeInBytes();
-}
-

http://git-wip-us.apache.org/repos/asf/beam/blob/a7f9f7d0/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/Monitor.java
----------------------------------------------------------------------
diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/Monitor.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/Monitor.java
index 02660bf..6370e41 100644
--- a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/Monitor.java
+++ b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/Monitor.java
@@ -19,6 +19,7 @@ package org.apache.beam.integration.nexmark;
 
 import java.io.Serializable;
 
+import org.apache.beam.integration.nexmark.model.KnownSize;
 import org.apache.beam.sdk.transforms.Aggregator;
 import org.apache.beam.sdk.transforms.DoFn;
 import org.apache.beam.sdk.transforms.Max;

http://git-wip-us.apache.org/repos/asf/beam/blob/a7f9f7d0/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NameCityStateId.java
----------------------------------------------------------------------
diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NameCityStateId.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NameCityStateId.java
deleted file mode 100644
index fe4687b..0000000
--- a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NameCityStateId.java
+++ /dev/null
@@ -1,105 +0,0 @@
-/*
- * 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.beam.integration.nexmark;
-
-import com.fasterxml.jackson.annotation.JsonProperty;
-import com.fasterxml.jackson.core.JsonProcessingException;
-
-import java.io.IOException;
-import java.io.InputStream;
-import java.io.OutputStream;
-import java.io.Serializable;
-
-import org.apache.beam.sdk.coders.AtomicCoder;
-import org.apache.beam.sdk.coders.Coder;
-import org.apache.beam.sdk.coders.CoderException;
-import org.apache.beam.sdk.coders.StringUtf8Coder;
-import org.apache.beam.sdk.coders.VarLongCoder;
-
-/**
- * Result of {@link Query3}.
- */
-public class NameCityStateId implements KnownSize, Serializable {
-  private static final Coder<Long> LONG_CODER = VarLongCoder.of();
-  private static final Coder<String> STRING_CODER = StringUtf8Coder.of();
-
-  public static final Coder<NameCityStateId> CODER = new AtomicCoder<NameCityStateId>() {
-    @Override
-    public void encode(NameCityStateId value, OutputStream outStream,
-        Coder.Context context)
-        throws CoderException, IOException {
-      STRING_CODER.encode(value.name, outStream, Context.NESTED);
-      STRING_CODER.encode(value.city, outStream, Context.NESTED);
-      STRING_CODER.encode(value.state, outStream, Context.NESTED);
-      LONG_CODER.encode(value.id, outStream, Context.NESTED);
-    }
-
-    @Override
-    public NameCityStateId decode(
-        InputStream inStream, Coder.Context context)
-        throws CoderException, IOException {
-      String name = STRING_CODER.decode(inStream, Context.NESTED);
-      String city = STRING_CODER.decode(inStream, Context.NESTED);
-      String state = STRING_CODER.decode(inStream, Context.NESTED);
-      long id = LONG_CODER.decode(inStream, Context.NESTED);
-      return new NameCityStateId(name, city, state, id);
-    }
-  };
-
-  @JsonProperty
-  public final String name;
-
-  @JsonProperty
-  public final String city;
-
-  @JsonProperty
-  public final String state;
-
-  @JsonProperty
-  public final long id;
-
-  // For Avro only.
-  @SuppressWarnings("unused")
-  private NameCityStateId() {
-    name = null;
-    city = null;
-    state = null;
-    id = 0;
-  }
-
-  public NameCityStateId(String name, String city, String state, long id) {
-    this.name = name;
-    this.city = city;
-    this.state = state;
-    this.id = id;
-  }
-
-  @Override
-  public long sizeInBytes() {
-    return name.length() + 1 + city.length() + 1 + state.length() + 1 + 8;
-  }
-
-  @Override
-  public String toString() {
-    try {
-      return NexmarkUtils.MAPPER.writeValueAsString(this);
-    } catch (JsonProcessingException e) {
-      throw new RuntimeException(e);
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/a7f9f7d0/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkApexDriver.java
----------------------------------------------------------------------
diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkApexDriver.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkApexDriver.java
deleted file mode 100644
index 4c2721e..0000000
--- a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkApexDriver.java
+++ /dev/null
@@ -1,48 +0,0 @@
-/*
- * 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.beam.integration.nexmark;
-
-import org.apache.beam.runners.apex.ApexPipelineOptions;
-import org.apache.beam.runners.apex.ApexRunner;
-import org.apache.beam.sdk.options.PipelineOptionsFactory;
-
-/**
- * Run NexMark queries using the Apex runner.
- */
-public class NexmarkApexDriver extends NexmarkDriver<NexmarkApexDriver.NexmarkApexOptions> {
-  /**
-   * Command line flags.
-   */
-  public interface NexmarkApexOptions extends Options, ApexPipelineOptions {
-  }
-
-  /**
-   * Entry point.
-   */
-  public static void main(String[] args) {
-    // Gather command line args, baseline, configurations, etc.
-    NexmarkApexOptions options = PipelineOptionsFactory.fromArgs(args)
-                                                        .withValidation()
-                                                        .as(NexmarkApexOptions.class);
-    options.setRunner(ApexRunner.class);
-    NexmarkApexRunner runner = new NexmarkApexRunner(options);
-    new NexmarkApexDriver().runAll(options, runner);
-  }
-}
-
-

http://git-wip-us.apache.org/repos/asf/beam/blob/a7f9f7d0/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkApexRunner.java
----------------------------------------------------------------------
diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkApexRunner.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkApexRunner.java
deleted file mode 100644
index 3b8993a..0000000
--- a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkApexRunner.java
+++ /dev/null
@@ -1,61 +0,0 @@
-/*
- * 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.beam.integration.nexmark;
-
-import javax.annotation.Nullable;
-
-/**
- * Run a query using the Apex runner.
- */
-public class NexmarkApexRunner extends NexmarkRunner<NexmarkApexDriver.NexmarkApexOptions> {
-  @Override
-  protected boolean isStreaming() {
-    return options.isStreaming();
-  }
-
-  @Override
-  protected int coresPerWorker() {
-    return 4;
-  }
-
-  @Override
-  protected int maxNumWorkers() {
-    return 5;
-  }
-
-  @Override
-  protected void invokeBuilderForPublishOnlyPipeline(
-      PipelineBuilder builder) {
-    builder.build(options);
-  }
-
-  @Override
-  protected void waitForPublisherPreload() {
-    throw new UnsupportedOperationException();
-  }
-
-  @Override
-  @Nullable
-  protected NexmarkPerf monitor(NexmarkQuery query) {
-    return null;
-  }
-
-  public NexmarkApexRunner(NexmarkApexDriver.NexmarkApexOptions options) {
-    super(options);
-  }
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/a7f9f7d0/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkConfiguration.java
----------------------------------------------------------------------
diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkConfiguration.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkConfiguration.java
index 0943664..e2890ed 100644
--- a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkConfiguration.java
+++ b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkConfiguration.java
@@ -29,7 +29,7 @@ import java.util.Objects;
  * programmatically. We only capture properties which may influence the resulting
  * pipeline performance, as captured by {@link NexmarkPerf}.
  */
-class NexmarkConfiguration implements Serializable {
+public class NexmarkConfiguration implements Serializable {
   public static final NexmarkConfiguration DEFAULT = new NexmarkConfiguration();
 
   /** If {@literal true}, include additional debugging and monitoring stats. */
@@ -228,7 +228,7 @@ class NexmarkConfiguration implements Serializable {
   /**
    * Replace any properties of this configuration which have been supplied by the command line.
    */
-  public void overrideFromOptions(Options options) {
+  public void overrideFromOptions(NexmarkOptions options) {
     if (options.getDebug() != null) {
       debug = options.getDebug();
     }
@@ -511,8 +511,6 @@ class NexmarkConfiguration implements Serializable {
 
   /**
    * Parse an object from {@code string}.
-   *
-   * @throws IOException
    */
   public static NexmarkConfiguration fromString(String string) {
     try {

http://git-wip-us.apache.org/repos/asf/beam/blob/a7f9f7d0/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkDirectDriver.java
----------------------------------------------------------------------
diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkDirectDriver.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkDirectDriver.java
deleted file mode 100644
index 24fcc01..0000000
--- a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkDirectDriver.java
+++ /dev/null
@@ -1,47 +0,0 @@
-/*
- * 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.beam.integration.nexmark;
-
-import org.apache.beam.runners.direct.DirectOptions;
-import org.apache.beam.runners.direct.DirectRunner;
-import org.apache.beam.sdk.options.PipelineOptionsFactory;
-
-/**
- * An implementation of the 'NEXMark queries' using the Direct Runner.
- */
-class NexmarkDirectDriver extends NexmarkDriver<NexmarkDirectDriver.NexmarkDirectOptions> {
-  /**
-   * Command line flags.
-   */
-  public interface NexmarkDirectOptions extends Options, DirectOptions {
-  }
-
-  /**
-   * Entry point.
-   */
-  public static void main(String[] args) {
-    NexmarkDirectOptions options =
-        PipelineOptionsFactory.fromArgs(args)
-                              .withValidation()
-                              .as(NexmarkDirectOptions.class);
-    options.setRunner(DirectRunner.class);
-    NexmarkDirectRunner runner = new NexmarkDirectRunner(options);
-    new NexmarkDirectDriver().runAll(options, runner);
-  }
-}
-

http://git-wip-us.apache.org/repos/asf/beam/blob/a7f9f7d0/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkDirectRunner.java
----------------------------------------------------------------------
diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkDirectRunner.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkDirectRunner.java
deleted file mode 100644
index 0119bbc..0000000
--- a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkDirectRunner.java
+++ /dev/null
@@ -1,58 +0,0 @@
-/*
- * 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.beam.integration.nexmark;
-
-/**
- * Run a single query using the Direct Runner.
- */
-class NexmarkDirectRunner extends NexmarkRunner<NexmarkDirectDriver.NexmarkDirectOptions> {
-  public NexmarkDirectRunner(NexmarkDirectDriver.NexmarkDirectOptions options) {
-    super(options);
-  }
-
-  @Override
-  protected boolean isStreaming() {
-    return options.isStreaming();
-  }
-
-  @Override
-  protected int coresPerWorker() {
-    return 4;
-  }
-
-  @Override
-  protected int maxNumWorkers() {
-    return 1;
-  }
-
-  @Override
-  protected void invokeBuilderForPublishOnlyPipeline(PipelineBuilder builder) {
-    throw new UnsupportedOperationException(
-        "Cannot use --pubSubMode=COMBINED with DirectRunner");
-  }
-
-  /**
-   * Monitor the progress of the publisher job. Return when it has been generating events for
-   * at least {@code configuration.preloadSeconds}.
-   */
-  @Override
-  protected void waitForPublisherPreload() {
-    throw new UnsupportedOperationException(
-        "Cannot use --pubSubMode=COMBINED with DirectRunner");
-  }
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/a7f9f7d0/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkDriver.java
----------------------------------------------------------------------
diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkDriver.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkDriver.java
index e6a7b0b..4714124 100644
--- a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkDriver.java
+++ b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkDriver.java
@@ -28,6 +28,9 @@ import java.util.List;
 import java.util.Map;
 import javax.annotation.Nullable;
 
+import org.apache.beam.integration.nexmark.model.Auction;
+import org.apache.beam.integration.nexmark.model.Bid;
+import org.apache.beam.integration.nexmark.model.Person;
 import org.joda.time.Duration;
 import org.joda.time.Instant;
 
@@ -48,7 +51,7 @@ import org.joda.time.Instant;
  * <p>See <a href="http://datalab.cs.pdx.edu/niagaraST/NEXMark/">
  * http://datalab.cs.pdx.edu/niagaraST/NEXMark/</a>
  */
-public class NexmarkDriver<OptionT extends Options> {
+public class NexmarkDriver<OptionT extends NexmarkOptions> {
 
   /**
    * Entry point.

http://git-wip-us.apache.org/repos/asf/beam/blob/a7f9f7d0/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkFlinkDriver.java
----------------------------------------------------------------------
diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkFlinkDriver.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkFlinkDriver.java
deleted file mode 100644
index 61a7d29..0000000
--- a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkFlinkDriver.java
+++ /dev/null
@@ -1,48 +0,0 @@
-/*
- * 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.beam.integration.nexmark;
-
-import org.apache.beam.runners.flink.FlinkPipelineOptions;
-import org.apache.beam.runners.flink.FlinkRunner;
-import org.apache.beam.sdk.options.PipelineOptionsFactory;
-
-/**
- * Run NexMark queries using the Flink runner.
- */
-public class NexmarkFlinkDriver extends NexmarkDriver<NexmarkFlinkDriver.NexmarkFlinkOptions> {
-  /**
-   * Command line flags.
-   */
-  public interface NexmarkFlinkOptions extends Options, FlinkPipelineOptions {
-  }
-
-  /**
-   * Entry point.
-   */
-  public static void main(String[] args) {
-    // Gather command line args, baseline, configurations, etc.
-    NexmarkFlinkOptions options = PipelineOptionsFactory.fromArgs(args)
-                                                        .withValidation()
-                                                        .as(NexmarkFlinkOptions.class);
-    options.setRunner(FlinkRunner.class);
-    NexmarkFlinkRunner runner = new NexmarkFlinkRunner(options);
-    new NexmarkFlinkDriver().runAll(options, runner);
-  }
-}
-
-

http://git-wip-us.apache.org/repos/asf/beam/blob/a7f9f7d0/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkFlinkRunner.java
----------------------------------------------------------------------
diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkFlinkRunner.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkFlinkRunner.java
deleted file mode 100644
index 95ab1ad..0000000
--- a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkFlinkRunner.java
+++ /dev/null
@@ -1,53 +0,0 @@
-/*
- * 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.beam.integration.nexmark;
-
-/**
- * Run a query using the Flink runner.
- */
-public class NexmarkFlinkRunner extends NexmarkRunner<NexmarkFlinkDriver.NexmarkFlinkOptions> {
-  @Override
-  protected boolean isStreaming() {
-    return options.isStreaming();
-  }
-
-  @Override
-  protected int coresPerWorker() {
-    return 4;
-  }
-
-  @Override
-  protected int maxNumWorkers() {
-    return 5;
-  }
-
-  @Override
-  protected void invokeBuilderForPublishOnlyPipeline(
-      PipelineBuilder builder) {
-    builder.build(options);
-  }
-
-  @Override
-  protected void waitForPublisherPreload() {
-    throw new UnsupportedOperationException();
-  }
-
-  public NexmarkFlinkRunner(NexmarkFlinkDriver.NexmarkFlinkOptions options) {
-    super(options);
-  }
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/a7f9f7d0/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkGoogleDriver.java
----------------------------------------------------------------------
diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkGoogleDriver.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkGoogleDriver.java
deleted file mode 100644
index 50c2a7c..0000000
--- a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkGoogleDriver.java
+++ /dev/null
@@ -1,62 +0,0 @@
-/*
- * 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.beam.integration.nexmark;
-
-import org.apache.beam.runners.dataflow.DataflowRunner;
-import org.apache.beam.runners.dataflow.options.DataflowPipelineOptions;
-import org.apache.beam.sdk.options.PipelineOptionsFactory;
-
-/**
- * An implementation of the 'NEXMark queries' for Google Dataflow.
- * These are multiple queries over a three table schema representing an online auction system:
- * <ul>
- * <li>{@link Person} represents a person submitting an item for auction and/or making a bid
- * on an auction.
- * <li>{@link Auction} represents an item under auction.
- * <li>{@link Bid} represents a bid for an item under auction.
- * </ul>
- * The queries exercise many aspects of streaming dataflow.
- *
- * <p>We synthesize the creation of people, auctions and bids in real-time. The data is not
- * particularly sensible.
- *
- * <p>See <a href="http://datalab.cs.pdx.edu/niagaraST/NEXMark/">
- * http://datalab.cs.pdx.edu/niagaraST/NEXMark/</a>
- */
-class NexmarkGoogleDriver extends NexmarkDriver<NexmarkGoogleDriver.NexmarkGoogleOptions> {
-  /**
-   * Command line flags.
-   */
-  public interface NexmarkGoogleOptions extends Options, DataflowPipelineOptions {
-
-  }
-
-  /**
-   * Entry point.
-   */
-  public static void main(String[] args) {
-    // Gather command line args, baseline, configurations, etc.
-    NexmarkGoogleOptions options = PipelineOptionsFactory.fromArgs(args)
-                                                         .withValidation()
-                                                         .as(NexmarkGoogleOptions.class);
-    options.setRunner(DataflowRunner.class);
-    NexmarkGoogleRunner runner = new NexmarkGoogleRunner(options);
-    new NexmarkGoogleDriver().runAll(options, runner);
-  }
-}
-

http://git-wip-us.apache.org/repos/asf/beam/blob/a7f9f7d0/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkGoogleRunner.java
----------------------------------------------------------------------
diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkGoogleRunner.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkGoogleRunner.java
deleted file mode 100644
index f4bfb1e..0000000
--- a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkGoogleRunner.java
+++ /dev/null
@@ -1,159 +0,0 @@
-/*
- * 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.beam.integration.nexmark;
-
-import static com.google.common.base.Preconditions.checkNotNull;
-
-import org.apache.beam.runners.dataflow.DataflowPipelineJob;
-import org.apache.beam.sdk.PipelineResult;
-import org.joda.time.Duration;
-
-/**
- * Run a singe Nexmark query using a given configuration on Google Dataflow.
- */
-class NexmarkGoogleRunner extends NexmarkRunner<NexmarkGoogleDriver.NexmarkGoogleOptions> {
-
-  public NexmarkGoogleRunner(NexmarkGoogleDriver.NexmarkGoogleOptions options) {
-    super(options);
-  }
-
-  @Override
-  protected boolean isStreaming() {
-    return options.isStreaming();
-  }
-
-  @Override
-  protected int coresPerWorker() {
-    String machineType = options.getWorkerMachineType();
-    if (machineType == null || machineType.isEmpty()) {
-      return 1;
-    }
-    String[] split = machineType.split("-");
-    if (split.length != 3) {
-      return 1;
-    }
-    try {
-      return Integer.parseInt(split[2]);
-    } catch (NumberFormatException ex) {
-      return 1;
-    }
-  }
-
-  @Override
-  protected int maxNumWorkers() {
-    return Math.max(options.getNumWorkers(), options.getMaxNumWorkers());
-  }
-
-  @Override
-  protected String getJobId(PipelineResult job) {
-    return ((DataflowPipelineJob) job).getJobId();
-  }
-
-  @Override
-  protected void invokeBuilderForPublishOnlyPipeline(PipelineBuilder builder) {
-    String jobName = options.getJobName();
-    String appName = options.getAppName();
-    options.setJobName("p-" + jobName);
-    options.setAppName("p-" + appName);
-    int coresPerWorker = coresPerWorker();
-    int eventGeneratorWorkers = (configuration.numEventGenerators + coresPerWorker - 1)
-                                / coresPerWorker;
-    options.setMaxNumWorkers(Math.min(options.getMaxNumWorkers(), eventGeneratorWorkers));
-    options.setNumWorkers(Math.min(options.getNumWorkers(), eventGeneratorWorkers));
-    publisherMonitor = new Monitor<Event>(queryName, "publisher");
-    try {
-      builder.build(options);
-    } finally {
-      options.setJobName(jobName);
-      options.setAppName(appName);
-      options.setMaxNumWorkers(options.getMaxNumWorkers());
-      options.setNumWorkers(options.getNumWorkers());
-    }
-  }
-
-  /**
-   * Monitor the progress of the publisher job. Return when it has been generating events for
-   * at least {@code configuration.preloadSeconds}.
-   */
-  @Override
-  protected void waitForPublisherPreload() {
-    checkNotNull(publisherMonitor);
-    checkNotNull(publisherResult);
-    if (!options.getMonitorJobs()) {
-      return;
-    }
-    if (!(publisherResult instanceof DataflowPipelineJob)) {
-      return;
-    }
-    if (configuration.preloadSeconds <= 0) {
-      return;
-    }
-
-    NexmarkUtils.console("waiting for publisher to pre-load");
-
-    DataflowPipelineJob job = (DataflowPipelineJob) publisherResult;
-
-    long numEvents = 0;
-    long startMsSinceEpoch = -1;
-    long endMsSinceEpoch = -1;
-    while (true) {
-      PipelineResult.State state = job.getState();
-      switch (state) {
-        case UNKNOWN:
-          // Keep waiting.
-          NexmarkUtils.console("%s publisher (%d events)", state, numEvents);
-          break;
-        case STOPPED:
-        case DONE:
-        case CANCELLED:
-        case FAILED:
-        case UPDATED:
-          NexmarkUtils.console("%s publisher (%d events)", state, numEvents);
-          return;
-        case RUNNING:
-          numEvents = getLong(job, publisherMonitor.getElementCounter());
-          if (startMsSinceEpoch < 0 && numEvents > 0) {
-            startMsSinceEpoch = System.currentTimeMillis();
-            endMsSinceEpoch = startMsSinceEpoch
-                              + Duration.standardSeconds(configuration.preloadSeconds).getMillis();
-          }
-          if (endMsSinceEpoch < 0) {
-            NexmarkUtils.console("%s publisher (%d events)", state, numEvents);
-          } else {
-            long remainMs = endMsSinceEpoch - System.currentTimeMillis();
-            if (remainMs > 0) {
-              NexmarkUtils.console("%s publisher (%d events, waiting for %ds)", state, numEvents,
-                  remainMs / 1000);
-            } else {
-              NexmarkUtils.console("publisher preloaded %d events", numEvents);
-              return;
-            }
-          }
-          break;
-      }
-
-      try {
-        Thread.sleep(PERF_DELAY.getMillis());
-      } catch (InterruptedException e) {
-        Thread.interrupted();
-        throw new RuntimeException("Interrupted: publisher still running.");
-      }
-    }
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/a7f9f7d0/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkOptions.java
----------------------------------------------------------------------
diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkOptions.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkOptions.java
new file mode 100644
index 0000000..1be974f
--- /dev/null
+++ b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkOptions.java
@@ -0,0 +1,386 @@
+/*
+ * 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.beam.integration.nexmark;
+
+import javax.annotation.Nullable;
+
+import org.apache.beam.sdk.options.Default;
+import org.apache.beam.sdk.options.Description;
+import org.apache.beam.sdk.options.PubsubOptions;
+
+/**
+ * Command line flags.
+ */
+public interface NexmarkOptions extends PubsubOptions {
+  @Description("Which suite to run. Default is to use command line arguments for one job.")
+  @Default.Enum("DEFAULT")
+  NexmarkSuite getSuite();
+
+  void setSuite(NexmarkSuite suite);
+
+  @Description("If true, and using the DataflowPipelineRunner, monitor the jobs as they run.")
+  @Default.Boolean(false)
+  boolean getMonitorJobs();
+
+  void setMonitorJobs(boolean monitorJobs);
+
+  @Description("Where the events come from.")
+  @Nullable
+  NexmarkUtils.SourceType getSourceType();
+
+  void setSourceType(NexmarkUtils.SourceType sourceType);
+
+  @Description("Prefix for input files if using avro input")
+  @Nullable
+  String getInputPath();
+
+  void setInputPath(String inputPath);
+
+  @Description("Where results go.")
+  @Nullable
+  NexmarkUtils.SinkType getSinkType();
+
+  void setSinkType(NexmarkUtils.SinkType sinkType);
+
+  @Description("Which mode to run in when source is PUBSUB.")
+  @Nullable
+  NexmarkUtils.PubSubMode getPubSubMode();
+
+  void setPubSubMode(NexmarkUtils.PubSubMode pubSubMode);
+
+  @Description("Which query to run.")
+  @Nullable
+  Integer getQuery();
+
+  void setQuery(Integer query);
+
+  @Description("Prefix for output files if using text output for results or running Query 10.")
+  @Nullable
+  String getOutputPath();
+
+  void setOutputPath(String outputPath);
+
+  @Description("Base name of pubsub topic to publish to in streaming mode.")
+  @Nullable
+  @Default.String("nexmark")
+  String getPubsubTopic();
+
+  void setPubsubTopic(String pubsubTopic);
+
+  @Description("Base name of pubsub subscription to read from in streaming mode.")
+  @Nullable
+  @Default.String("nexmark")
+  String getPubsubSubscription();
+
+  void setPubsubSubscription(String pubsubSubscription);
+
+  @Description("Base name of BigQuery table name if using BigQuery output.")
+  @Nullable
+  @Default.String("nexmark")
+  String getBigQueryTable();
+
+  void setBigQueryTable(String bigQueryTable);
+
+  @Description("Approximate number of events to generate. "
+               + "Zero for effectively unlimited in streaming mode.")
+  @Nullable
+  Long getNumEvents();
+
+  void setNumEvents(Long numEvents);
+
+  @Description("Time in seconds to preload the subscription with data, at the initial input rate "
+               + "of the pipeline.")
+  @Nullable
+  Integer getPreloadSeconds();
+
+  void setPreloadSeconds(Integer preloadSeconds);
+
+  @Description("Number of unbounded sources to create events.")
+  @Nullable
+  Integer getNumEventGenerators();
+
+  void setNumEventGenerators(Integer numEventGenerators);
+
+  @Description("Shape of event rate curve.")
+  @Nullable
+  NexmarkUtils.RateShape getRateShape();
+
+  void setRateShape(NexmarkUtils.RateShape rateShape);
+
+  @Description("Initial overall event rate (in --rateUnit).")
+  @Nullable
+  Integer getFirstEventRate();
+
+  void setFirstEventRate(Integer firstEventRate);
+
+  @Description("Next overall event rate (in --rateUnit).")
+  @Nullable
+  Integer getNextEventRate();
+
+  void setNextEventRate(Integer nextEventRate);
+
+  @Description("Unit for rates.")
+  @Nullable
+  NexmarkUtils.RateUnit getRateUnit();
+
+  void setRateUnit(NexmarkUtils.RateUnit rateUnit);
+
+  @Description("Overall period of rate shape, in seconds.")
+  @Nullable
+  Integer getRatePeriodSec();
+
+  void setRatePeriodSec(Integer ratePeriodSec);
+
+  @Description("If true, relay events in real time in streaming mode.")
+  @Nullable
+  Boolean getIsRateLimited();
+
+  void setIsRateLimited(Boolean isRateLimited);
+
+  @Description("If true, use wallclock time as event time. Otherwise, use a deterministic"
+               + " time in the past so that multiple runs will see exactly the same event streams"
+               + " and should thus have exactly the same results.")
+  @Nullable
+  Boolean getUseWallclockEventTime();
+
+  void setUseWallclockEventTime(Boolean useWallclockEventTime);
+
+  @Description("Assert pipeline results match model results.")
+  @Nullable
+  boolean getAssertCorrectness();
+
+  void setAssertCorrectness(boolean assertCorrectness);
+
+  @Description("Log all input events.")
+  @Nullable
+  boolean getLogEvents();
+
+  void setLogEvents(boolean logEvents);
+
+  @Description("Log all query results.")
+  @Nullable
+  boolean getLogResults();
+
+  void setLogResults(boolean logResults);
+
+  @Description("Average size in bytes for a person record.")
+  @Nullable
+  Integer getAvgPersonByteSize();
+
+  void setAvgPersonByteSize(Integer avgPersonByteSize);
+
+  @Description("Average size in bytes for an auction record.")
+  @Nullable
+  Integer getAvgAuctionByteSize();
+
+  void setAvgAuctionByteSize(Integer avgAuctionByteSize);
+
+  @Description("Average size in bytes for a bid record.")
+  @Nullable
+  Integer getAvgBidByteSize();
+
+  void setAvgBidByteSize(Integer avgBidByteSize);
+
+  @Description("Ratio of bids for 'hot' auctions above the background.")
+  @Nullable
+  Integer getHotAuctionRatio();
+
+  void setHotAuctionRatio(Integer hotAuctionRatio);
+
+  @Description("Ratio of auctions for 'hot' sellers above the background.")
+  @Nullable
+  Integer getHotSellersRatio();
+
+  void setHotSellersRatio(Integer hotSellersRatio);
+
+  @Description("Ratio of auctions for 'hot' bidders above the background.")
+  @Nullable
+  Integer getHotBiddersRatio();
+
+  void setHotBiddersRatio(Integer hotBiddersRatio);
+
+  @Description("Window size in seconds.")
+  @Nullable
+  Long getWindowSizeSec();
+
+  void setWindowSizeSec(Long windowSizeSec);
+
+  @Description("Window period in seconds.")
+  @Nullable
+  Long getWindowPeriodSec();
+
+  void setWindowPeriodSec(Long windowPeriodSec);
+
+  @Description("If in streaming mode, the holdback for watermark in seconds.")
+  @Nullable
+  Long getWatermarkHoldbackSec();
+
+  void setWatermarkHoldbackSec(Long watermarkHoldbackSec);
+
+  @Description("Roughly how many auctions should be in flight for each generator.")
+  @Nullable
+  Integer getNumInFlightAuctions();
+
+  void setNumInFlightAuctions(Integer numInFlightAuctions);
+
+
+  @Description("Maximum number of people to consider as active for placing auctions or bids.")
+  @Nullable
+  Integer getNumActivePeople();
+
+  void setNumActivePeople(Integer numActivePeople);
+
+  @Description("Filename of perf data to append to.")
+  @Nullable
+  String getPerfFilename();
+
+  void setPerfFilename(String perfFilename);
+
+  @Description("Filename of baseline perf data to read from.")
+  @Nullable
+  String getBaselineFilename();
+
+  void setBaselineFilename(String baselineFilename);
+
+  @Description("Filename of summary perf data to append to.")
+  @Nullable
+  String getSummaryFilename();
+
+  void setSummaryFilename(String summaryFilename);
+
+  @Description("Filename for javascript capturing all perf data and any baselines.")
+  @Nullable
+  String getJavascriptFilename();
+
+  void setJavascriptFilename(String javascriptFilename);
+
+  @Description("If true, don't run the actual query. Instead, calculate the distribution "
+               + "of number of query results per (event time) minute according to the query model.")
+  @Nullable
+  boolean getJustModelResultRate();
+
+  void setJustModelResultRate(boolean justModelResultRate);
+
+  @Description("Coder strategy to use.")
+  @Nullable
+  NexmarkUtils.CoderStrategy getCoderStrategy();
+
+  void setCoderStrategy(NexmarkUtils.CoderStrategy coderStrategy);
+
+  @Description("Delay, in milliseconds, for each event. We will peg one core for this "
+               + "number of milliseconds to simulate CPU-bound computation.")
+  @Nullable
+  Long getCpuDelayMs();
+
+  void setCpuDelayMs(Long cpuDelayMs);
+
+  @Description("Extra data, in bytes, to save to persistent state for each event. "
+               + "This will force I/O all the way to durable storage to simulate an "
+               + "I/O-bound computation.")
+  @Nullable
+  Long getDiskBusyBytes();
+
+  void setDiskBusyBytes(Long diskBusyBytes);
+
+  @Description("Skip factor for query 2. We select bids for every {@code auctionSkip}'th auction")
+  @Nullable
+  Integer getAuctionSkip();
+
+  void setAuctionSkip(Integer auctionSkip);
+
+  @Description("Fanout for queries 4 (groups by category id) and 7 (finds a global maximum).")
+  @Nullable
+  Integer getFanout();
+
+  void setFanout(Integer fanout);
+
+  @Description("Length of occasional delay to impose on events (in seconds).")
+  @Nullable
+  Long getOccasionalDelaySec();
+
+  void setOccasionalDelaySec(Long occasionalDelaySec);
+
+  @Description("Probability that an event will be delayed by delayS.")
+  @Nullable
+  Double getProbDelayedEvent();
+
+  void setProbDelayedEvent(Double probDelayedEvent);
+
+  @Description("Maximum size of each log file (in events). For Query10 only.")
+  @Nullable
+  Integer getMaxLogEvents();
+
+  void setMaxLogEvents(Integer maxLogEvents);
+
+  @Description("How to derive names of resources.")
+  @Default.Enum("QUERY_AND_SALT")
+  NexmarkUtils.ResourceNameMode getResourceNameMode();
+
+  void setResourceNameMode(NexmarkUtils.ResourceNameMode mode);
+
+  @Description("If true, manage the creation and cleanup of topics, subscriptions and gcs files.")
+  @Default.Boolean(true)
+  boolean getManageResources();
+
+  void setManageResources(boolean manageResources);
+
+  @Description("If true, use pub/sub publish time instead of event time.")
+  @Nullable
+  Boolean getUsePubsubPublishTime();
+
+  void setUsePubsubPublishTime(Boolean usePubsubPublishTime);
+
+  @Description("Number of events in out-of-order groups. 1 implies no out-of-order events. "
+               + "1000 implies every 1000 events per generator are emitted in pseudo-random order.")
+  @Nullable
+  Long getOutOfOrderGroupSize();
+
+  void setOutOfOrderGroupSize(Long outOfOrderGroupSize);
+
+  @Description("If false, do not add the Monitor and Snoop transforms.")
+  @Nullable
+  Boolean getDebug();
+
+  void setDebug(Boolean value);
+
+  @Description("If set, cancel running pipelines after this long")
+  @Nullable
+  Long getRunningTimeMinutes();
+
+  void setRunningTimeMinutes(Long value);
+
+  @Description("If set and --monitorJobs is true, check that the system watermark is never more "
+               + "than this far behind real time")
+  @Nullable
+  Long getMaxSystemLagSeconds();
+
+  void setMaxSystemLagSeconds(Long value);
+
+  @Description("If set and --monitorJobs is true, check that the data watermark is never more "
+               + "than this far behind real time")
+  @Nullable
+  Long getMaxDataLagSeconds();
+
+  void setMaxDataLagSeconds(Long value);
+
+  @Description("Only start validating watermarks after this many seconds")
+  @Nullable
+  Long getWatermarkValidationDelaySeconds();
+
+  void setWatermarkValidationDelaySeconds(Long value);
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/a7f9f7d0/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkPerf.java
----------------------------------------------------------------------
diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkPerf.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkPerf.java
index 37b6213..e7f59c8 100644
--- a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkPerf.java
+++ b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkPerf.java
@@ -27,7 +27,7 @@ import javax.annotation.Nullable;
 /**
  * Summary of performance for a particular run of a configuration.
  */
-class NexmarkPerf {
+public class NexmarkPerf {
   /**
    * A sample of the number of events and number of results (if known) generated at
    * a particular time.
@@ -177,8 +177,6 @@ class NexmarkPerf {
 
   /**
    * Parse a {@link NexmarkPerf} object from JSON {@code string}.
-   *
-   * @throws IOException
    */
   public static NexmarkPerf fromString(String string) {
     try {

http://git-wip-us.apache.org/repos/asf/beam/blob/a7f9f7d0/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkQuery.java
----------------------------------------------------------------------
diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkQuery.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkQuery.java
index 5ef4191..c268a3b 100644
--- a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkQuery.java
+++ b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkQuery.java
@@ -18,7 +18,11 @@
 package org.apache.beam.integration.nexmark;
 
 import javax.annotation.Nullable;
-
+import org.apache.beam.integration.nexmark.model.Auction;
+import org.apache.beam.integration.nexmark.model.Bid;
+import org.apache.beam.integration.nexmark.model.Event;
+import org.apache.beam.integration.nexmark.model.KnownSize;
+import org.apache.beam.integration.nexmark.model.Person;
 import org.apache.beam.sdk.transforms.Aggregator;
 import org.apache.beam.sdk.transforms.DoFn;
 import org.apache.beam.sdk.transforms.Filter;
@@ -29,7 +33,6 @@ import org.apache.beam.sdk.values.KV;
 import org.apache.beam.sdk.values.PCollection;
 import org.apache.beam.sdk.values.TimestampedValue;
 import org.apache.beam.sdk.values.TupleTag;
-
 import org.joda.time.Instant;
 
 /**

http://git-wip-us.apache.org/repos/asf/beam/blob/a7f9f7d0/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkQueryModel.java
----------------------------------------------------------------------
diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkQueryModel.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkQueryModel.java
index f265e0d..b2b1826 100644
--- a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkQueryModel.java
+++ b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkQueryModel.java
@@ -17,11 +17,6 @@
  */
 package org.apache.beam.integration.nexmark;
 
-import static org.hamcrest.CoreMatchers.containsString;
-import static org.hamcrest.CoreMatchers.equalTo;
-import static org.hamcrest.CoreMatchers.hasItems;
-import org.hamcrest.collection.IsIterableContainingInAnyOrder;
-
 import java.io.Serializable;
 import java.util.ArrayList;
 import java.util.Collection;
@@ -30,10 +25,11 @@ import java.util.Iterator;
 import java.util.List;
 import java.util.Set;
 
+import org.apache.beam.integration.nexmark.model.KnownSize;
 import org.apache.beam.sdk.transforms.SerializableFunction;
 import org.apache.beam.sdk.values.TimestampedValue;
 
-import org.hamcrest.core.IsCollectionContaining;
+
 import org.hamcrest.core.IsEqual;
 import org.joda.time.Duration;
 import org.joda.time.Instant;
@@ -107,15 +103,18 @@ public abstract class NexmarkQueryModel implements Serializable {
   /** Return assertion to use on results of pipeline for this query. */
   public SerializableFunction<Iterable<TimestampedValue<KnownSize>>, Void> assertionFor() {
     final Collection<String> expectedStrings = toCollection(simulator().results());
-    final String[] expectedStringsArray = expectedStrings.toArray(new String[expectedStrings.size()]);
+    final String[] expectedStringsArray =
+      expectedStrings.toArray(new String[expectedStrings.size()]);
 
     return new SerializableFunction<Iterable<TimestampedValue<KnownSize>>, Void>() {
       @Override
       public Void apply(Iterable<TimestampedValue<KnownSize>> actual) {
-        Collection<String> actualStrings = toCollection(relevantResults(actual).iterator());
-                Assert.assertThat("wrong pipeline output", actualStrings, IsEqual.equalTo(expectedStrings));
+      Collection<String> actualStrings = toCollection(relevantResults(actual).iterator());
+        Assert.assertThat("wrong pipeline output", actualStrings,
+          IsEqual.equalTo(expectedStrings));
 //compare without order
-//        Assert.assertThat("wrong pipeline output", actualStrings, IsIterableContainingInAnyOrder.containsInAnyOrder(expectedStringsArray));
+//      Assert.assertThat("wrong pipeline output", actualStrings,
+//        IsIterableContainingInAnyOrder.containsInAnyOrder(expectedStringsArray));
         return null;
       }
     };


[38/55] [abbrv] beam git commit: Remove references to dataflow in generic classes

Posted by ie...@apache.org.
Remove references to dataflow in generic classes


Project: http://git-wip-us.apache.org/repos/asf/beam/repo
Commit: http://git-wip-us.apache.org/repos/asf/beam/commit/69953a0b
Tree: http://git-wip-us.apache.org/repos/asf/beam/tree/69953a0b
Diff: http://git-wip-us.apache.org/repos/asf/beam/diff/69953a0b

Branch: refs/heads/master
Commit: 69953a0b803896a982347b6bb821a922f6970d2b
Parents: dbd1b15
Author: Etienne Chauchot <ec...@gmail.com>
Authored: Tue May 9 12:08:10 2017 +0200
Committer: Ismaël Mejía <ie...@gmail.com>
Committed: Wed Aug 23 19:07:28 2017 +0200

----------------------------------------------------------------------
 .../src/main/java/org/apache/beam/integration/nexmark/Main.java  | 4 ++--
 .../java/org/apache/beam/integration/nexmark/NexmarkOptions.java | 2 +-
 .../java/org/apache/beam/integration/nexmark/NexmarkUtils.java   | 2 +-
 3 files changed, 4 insertions(+), 4 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/beam/blob/69953a0b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/Main.java
----------------------------------------------------------------------
diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/Main.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/Main.java
index da4d446..4c23651 100644
--- a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/Main.java
+++ b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/Main.java
@@ -36,7 +36,7 @@ import org.joda.time.Duration;
 import org.joda.time.Instant;
 
 /**
- * An implementation of the 'NEXMark queries' for Google Dataflow.
+ * An implementation of the 'NEXMark queries' for Beam.
  * These are multiple queries over a three table schema representing an online auction system:
  * <ul>
  * <li>{@link Person} represents a person submitting an item for auction and/or making a bid
@@ -44,7 +44,7 @@ import org.joda.time.Instant;
  * <li>{@link Auction} represents an item under auction.
  * <li>{@link Bid} represents a bid for an item under auction.
  * </ul>
- * The queries exercise many aspects of streaming dataflow.
+ * The queries exercise many aspects of the Beam model.
  *
  * <p>We synthesize the creation of people, auctions and bids in real-time. The data is not
  * particularly sensible.

http://git-wip-us.apache.org/repos/asf/beam/blob/69953a0b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkOptions.java
----------------------------------------------------------------------
diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkOptions.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkOptions.java
index f162fd6..9afffaa 100644
--- a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkOptions.java
+++ b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkOptions.java
@@ -37,7 +37,7 @@ public interface NexmarkOptions
 
   void setSuite(NexmarkSuite suite);
 
-  @Description("If true, and using the DataflowPipelineRunner, monitor the jobs as they run.")
+  @Description("If true, monitor the jobs as they run.")
   @Default.Boolean(false)
   boolean getMonitorJobs();
 

http://git-wip-us.apache.org/repos/asf/beam/blob/69953a0b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkUtils.java
----------------------------------------------------------------------
diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkUtils.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkUtils.java
index f6215e9..ea851af 100644
--- a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkUtils.java
+++ b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkUtils.java
@@ -289,7 +289,7 @@ public class NexmarkUtils {
   private static final boolean LOG_ERROR = true;
 
   /**
-   * Set to true to log directly to stdout on VM. You can watch the results in real-time with:
+   * Set to true to log directly to stdout. If run using Google Dataflow, you can watch the results in real-time with:
    * tail -f /var/log/dataflow/streaming-harness/harness-stdout.log
    */
   private static final boolean LOG_TO_CONSOLE = false;


[45/55] [abbrv] beam git commit: Move module beam-integration-java-nexmark to beam-sdks-java-nexmark

Posted by ie...@apache.org.
http://git-wip-us.apache.org/repos/asf/beam/blob/f4333df7/sdks/java/nexmark/src/main/java/org/apache/beam/sdk/nexmark/NexmarkLauncher.java
----------------------------------------------------------------------
diff --git a/sdks/java/nexmark/src/main/java/org/apache/beam/sdk/nexmark/NexmarkLauncher.java b/sdks/java/nexmark/src/main/java/org/apache/beam/sdk/nexmark/NexmarkLauncher.java
new file mode 100644
index 0000000..550fbd2
--- /dev/null
+++ b/sdks/java/nexmark/src/main/java/org/apache/beam/sdk/nexmark/NexmarkLauncher.java
@@ -0,0 +1,1157 @@
+/*
+ * 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.beam.sdk.nexmark;
+
+import static com.google.common.base.Preconditions.checkState;
+
+import com.google.api.services.bigquery.model.TableFieldSchema;
+import com.google.api.services.bigquery.model.TableRow;
+import com.google.api.services.bigquery.model.TableSchema;
+import com.google.common.base.Strings;
+import com.google.common.collect.ImmutableList;
+import com.google.common.collect.Lists;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.NoSuchElementException;
+import java.util.concurrent.ThreadLocalRandom;
+import javax.annotation.Nullable;
+import org.apache.beam.sdk.Pipeline;
+import org.apache.beam.sdk.PipelineResult;
+import org.apache.beam.sdk.coders.CoderException;
+import org.apache.beam.sdk.io.AvroIO;
+import org.apache.beam.sdk.io.TextIO;
+import org.apache.beam.sdk.io.gcp.bigquery.BigQueryIO;
+import org.apache.beam.sdk.io.gcp.pubsub.PubsubIO;
+import org.apache.beam.sdk.io.gcp.pubsub.PubsubMessage;
+import org.apache.beam.sdk.metrics.DistributionResult;
+import org.apache.beam.sdk.metrics.MetricNameFilter;
+import org.apache.beam.sdk.metrics.MetricQueryResults;
+import org.apache.beam.sdk.metrics.MetricResult;
+import org.apache.beam.sdk.metrics.MetricsFilter;
+import org.apache.beam.sdk.nexmark.model.Auction;
+import org.apache.beam.sdk.nexmark.model.Bid;
+import org.apache.beam.sdk.nexmark.model.Event;
+import org.apache.beam.sdk.nexmark.model.KnownSize;
+import org.apache.beam.sdk.nexmark.model.Person;
+import org.apache.beam.sdk.nexmark.queries.NexmarkQuery;
+import org.apache.beam.sdk.nexmark.queries.NexmarkQueryModel;
+import org.apache.beam.sdk.nexmark.queries.Query0;
+import org.apache.beam.sdk.nexmark.queries.Query0Model;
+import org.apache.beam.sdk.nexmark.queries.Query1;
+import org.apache.beam.sdk.nexmark.queries.Query10;
+import org.apache.beam.sdk.nexmark.queries.Query11;
+import org.apache.beam.sdk.nexmark.queries.Query12;
+import org.apache.beam.sdk.nexmark.queries.Query1Model;
+import org.apache.beam.sdk.nexmark.queries.Query2;
+import org.apache.beam.sdk.nexmark.queries.Query2Model;
+import org.apache.beam.sdk.nexmark.queries.Query3;
+import org.apache.beam.sdk.nexmark.queries.Query3Model;
+import org.apache.beam.sdk.nexmark.queries.Query4;
+import org.apache.beam.sdk.nexmark.queries.Query4Model;
+import org.apache.beam.sdk.nexmark.queries.Query5;
+import org.apache.beam.sdk.nexmark.queries.Query5Model;
+import org.apache.beam.sdk.nexmark.queries.Query6;
+import org.apache.beam.sdk.nexmark.queries.Query6Model;
+import org.apache.beam.sdk.nexmark.queries.Query7;
+import org.apache.beam.sdk.nexmark.queries.Query7Model;
+import org.apache.beam.sdk.nexmark.queries.Query8;
+import org.apache.beam.sdk.nexmark.queries.Query8Model;
+import org.apache.beam.sdk.nexmark.queries.Query9;
+import org.apache.beam.sdk.nexmark.queries.Query9Model;
+import org.apache.beam.sdk.testing.PAssert;
+import org.apache.beam.sdk.transforms.DoFn;
+import org.apache.beam.sdk.transforms.ParDo;
+import org.apache.beam.sdk.util.CoderUtils;
+import org.apache.beam.sdk.values.PCollection;
+import org.apache.beam.sdk.values.PCollectionTuple;
+import org.apache.beam.sdk.values.TimestampedValue;
+import org.apache.beam.sdk.values.TupleTag;
+import org.apache.beam.sdk.values.TupleTagList;
+import org.joda.time.Duration;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Run a single Nexmark query using a given configuration.
+ */
+public class NexmarkLauncher<OptionT extends NexmarkOptions> {
+  private static final org.slf4j.Logger LOG = LoggerFactory.getLogger(NexmarkLauncher.class);
+  /**
+   * Minimum number of samples needed for 'stead-state' rate calculation.
+   */
+  private static final int MIN_SAMPLES = 9;
+  /**
+   * Minimum length of time over which to consider samples for 'steady-state' rate calculation.
+   */
+  private static final Duration MIN_WINDOW = Duration.standardMinutes(2);
+  /**
+   * Delay between perf samples.
+   */
+  private static final Duration PERF_DELAY = Duration.standardSeconds(15);
+  /**
+   * How long to let streaming pipeline run after all events have been generated and we've
+   * seen no activity.
+   */
+  private static final Duration DONE_DELAY = Duration.standardMinutes(1);
+  /**
+   * How long to allow no activity without warning.
+   */
+  private static final Duration STUCK_WARNING_DELAY = Duration.standardMinutes(10);
+  /**
+   * How long to let streaming pipeline run after we've
+   * seen no activity, even if all events have not been generated.
+   */
+  private static final Duration STUCK_TERMINATE_DELAY = Duration.standardDays(3);
+  /**
+   * NexmarkOptions shared by all runs.
+   */
+  private final OptionT options;
+
+  /**
+   * Which configuration we are running.
+   */
+  @Nullable
+  private NexmarkConfiguration configuration;
+
+  /**
+   * If in --pubsubMode=COMBINED, the event monitor for the publisher pipeline. Otherwise null.
+   */
+  @Nullable
+  private Monitor<Event> publisherMonitor;
+
+  /**
+   * If in --pubsubMode=COMBINED, the pipeline result for the publisher pipeline. Otherwise null.
+   */
+  @Nullable
+  private PipelineResult publisherResult;
+
+  /**
+   * Result for the main pipeline.
+   */
+  @Nullable
+  private PipelineResult mainResult;
+
+  /**
+   * Query name we are running.
+   */
+  @Nullable
+  private String queryName;
+
+  public NexmarkLauncher(OptionT options) {
+    this.options = options;
+  }
+
+
+  /**
+   * Is this query running in streaming mode?
+   */
+  private boolean isStreaming() {
+    return options.isStreaming();
+  }
+
+  /**
+   * Return maximum number of workers.
+   */
+  private int maxNumWorkers() {
+    return 5;
+  }
+
+  /**
+   * Return the current value for a long counter, or a default value if can't be retrieved.
+   * Note this uses only attempted metrics because some runners don't support committed metrics.
+   */
+  private long getCounterMetric(PipelineResult result, String namespace, String name,
+    long defaultValue) {
+    MetricQueryResults metrics = result.metrics().queryMetrics(
+        MetricsFilter.builder().addNameFilter(MetricNameFilter.named(namespace, name)).build());
+    Iterable<MetricResult<Long>> counters = metrics.counters();
+    try {
+      MetricResult<Long> metricResult = counters.iterator().next();
+      return metricResult.attempted();
+    } catch (NoSuchElementException e) {
+      LOG.error("Failed to get metric {}, from namespace {}", name, namespace);
+    }
+    return defaultValue;
+  }
+
+  /**
+   * Return the current value for a long counter, or a default value if can't be retrieved.
+   * Note this uses only attempted metrics because some runners don't support committed metrics.
+   */
+  private long getDistributionMetric(PipelineResult result, String namespace, String name,
+      DistributionType distType, long defaultValue) {
+    MetricQueryResults metrics = result.metrics().queryMetrics(
+        MetricsFilter.builder().addNameFilter(MetricNameFilter.named(namespace, name)).build());
+    Iterable<MetricResult<DistributionResult>> distributions = metrics.distributions();
+    try {
+      MetricResult<DistributionResult> distributionResult = distributions.iterator().next();
+      switch (distType) {
+        case MIN:
+          return distributionResult.attempted().min();
+        case MAX:
+          return distributionResult.attempted().max();
+        default:
+          return defaultValue;
+      }
+    } catch (NoSuchElementException e) {
+      LOG.error(
+          "Failed to get distribution metric {} for namespace {}",
+          name,
+          namespace);
+    }
+    return defaultValue;
+  }
+
+  private enum DistributionType {MIN, MAX}
+
+  /**
+   * Return the current value for a time counter, or -1 if can't be retrieved.
+   */
+  private long getTimestampMetric(long now, long value) {
+    // timestamp metrics are used to monitor time of execution of transforms.
+    // If result timestamp metric is too far from now, consider that metric is erroneous
+
+    if (Math.abs(value - now) > Duration.standardDays(10000).getMillis()) {
+      return -1;
+    }
+    return value;
+  }
+
+  /**
+   * Find a 'steady state' events/sec from {@code snapshots} and
+   * store it in {@code perf} if found.
+   */
+  private void captureSteadyState(NexmarkPerf perf, List<NexmarkPerf.ProgressSnapshot> snapshots) {
+    if (!options.isStreaming()) {
+      return;
+    }
+
+    // Find the first sample with actual event and result counts.
+    int dataStart = 0;
+    for (; dataStart < snapshots.size(); dataStart++) {
+      if (snapshots.get(dataStart).numEvents >= 0 && snapshots.get(dataStart).numResults >= 0) {
+        break;
+      }
+    }
+
+    // Find the last sample which demonstrated progress.
+    int dataEnd = snapshots.size() - 1;
+    for (; dataEnd > dataStart; dataEnd--) {
+      if (snapshots.get(dataEnd).anyActivity(snapshots.get(dataEnd - 1))) {
+        break;
+      }
+    }
+
+    int numSamples = dataEnd - dataStart + 1;
+    if (numSamples < MIN_SAMPLES) {
+      // Not enough samples.
+      NexmarkUtils.console("%d samples not enough to calculate steady-state event rate",
+          numSamples);
+      return;
+    }
+
+    // We'll look at only the middle third samples.
+    int sampleStart = dataStart + numSamples / 3;
+    int sampleEnd = dataEnd - numSamples / 3;
+
+    double sampleSec =
+        snapshots.get(sampleEnd).secSinceStart - snapshots.get(sampleStart).secSinceStart;
+    if (sampleSec < MIN_WINDOW.getStandardSeconds()) {
+      // Not sampled over enough time.
+      NexmarkUtils.console(
+          "sample of %.1f sec not long enough to calculate steady-state event rate",
+          sampleSec);
+      return;
+    }
+
+    // Find rate with least squares error.
+    double sumxx = 0.0;
+    double sumxy = 0.0;
+    long prevNumEvents = -1;
+    for (int i = sampleStart; i <= sampleEnd; i++) {
+      if (prevNumEvents == snapshots.get(i).numEvents) {
+        // Skip samples with no change in number of events since they contribute no data.
+        continue;
+      }
+      // Use the effective runtime instead of wallclock time so we can
+      // insulate ourselves from delays and stutters in the query manager.
+      double x = snapshots.get(i).runtimeSec;
+      prevNumEvents = snapshots.get(i).numEvents;
+      double y = prevNumEvents;
+      sumxx += x * x;
+      sumxy += x * y;
+    }
+    double eventsPerSec = sumxy / sumxx;
+    NexmarkUtils.console("revising events/sec from %.1f to %.1f", perf.eventsPerSec, eventsPerSec);
+    perf.eventsPerSec = eventsPerSec;
+  }
+
+  /**
+   * Return the current performance given {@code eventMonitor} and {@code resultMonitor}.
+   */
+  private NexmarkPerf currentPerf(
+      long startMsSinceEpoch, long now, PipelineResult result,
+      List<NexmarkPerf.ProgressSnapshot> snapshots, Monitor<?> eventMonitor,
+      Monitor<?> resultMonitor) {
+    NexmarkPerf perf = new NexmarkPerf();
+
+    long numEvents =
+      getCounterMetric(result, eventMonitor.name, eventMonitor.prefix + ".elements", -1);
+    long numEventBytes =
+      getCounterMetric(result, eventMonitor.name, eventMonitor.prefix + ".bytes", -1);
+    long eventStart =
+      getTimestampMetric(now,
+        getDistributionMetric(result, eventMonitor.name, eventMonitor.prefix + ".startTime",
+          DistributionType.MIN, -1));
+    long eventEnd =
+      getTimestampMetric(now,
+        getDistributionMetric(result, eventMonitor.name, eventMonitor.prefix + ".endTime",
+          DistributionType.MAX, -1));
+
+    long numResults =
+      getCounterMetric(result, resultMonitor.name, resultMonitor.prefix + ".elements", -1);
+    long numResultBytes =
+      getCounterMetric(result, resultMonitor.name, resultMonitor.prefix + ".bytes", -1);
+    long resultStart =
+      getTimestampMetric(now,
+        getDistributionMetric(result, resultMonitor.name, resultMonitor.prefix + ".startTime",
+          DistributionType.MIN, -1));
+    long resultEnd =
+      getTimestampMetric(now,
+        getDistributionMetric(result, resultMonitor.name, resultMonitor.prefix + ".endTime",
+          DistributionType.MAX, -1));
+    long timestampStart =
+      getTimestampMetric(now,
+        getDistributionMetric(result,
+          resultMonitor.name, resultMonitor.prefix + ".startTimestamp",
+          DistributionType.MIN, -1));
+    long timestampEnd =
+      getTimestampMetric(now,
+        getDistributionMetric(result,
+          resultMonitor.name, resultMonitor.prefix + ".endTimestamp",
+          DistributionType.MAX, -1));
+
+    long effectiveEnd = -1;
+    if (eventEnd >= 0 && resultEnd >= 0) {
+      // It is possible for events to be generated after the last result was emitted.
+      // (Eg Query 2, which only yields results for a small prefix of the event stream.)
+      // So use the max of last event and last result times.
+      effectiveEnd = Math.max(eventEnd, resultEnd);
+    } else if (resultEnd >= 0) {
+      effectiveEnd = resultEnd;
+    } else if (eventEnd >= 0) {
+      // During startup we may have no result yet, but we would still like to track how
+      // long the pipeline has been running.
+      effectiveEnd = eventEnd;
+    }
+
+    if (effectiveEnd >= 0 && eventStart >= 0 && effectiveEnd >= eventStart) {
+      perf.runtimeSec = (effectiveEnd - eventStart) / 1000.0;
+    }
+
+    if (numEvents >= 0) {
+      perf.numEvents = numEvents;
+    }
+
+    if (numEvents >= 0 && perf.runtimeSec > 0.0) {
+      // For streaming we may later replace this with a 'steady-state' value calculated
+      // from the progress snapshots.
+      perf.eventsPerSec = numEvents / perf.runtimeSec;
+    }
+
+    if (numEventBytes >= 0 && perf.runtimeSec > 0.0) {
+      perf.eventBytesPerSec = numEventBytes / perf.runtimeSec;
+    }
+
+    if (numResults >= 0) {
+      perf.numResults = numResults;
+    }
+
+    if (numResults >= 0 && perf.runtimeSec > 0.0) {
+      perf.resultsPerSec = numResults / perf.runtimeSec;
+    }
+
+    if (numResultBytes >= 0 && perf.runtimeSec > 0.0) {
+      perf.resultBytesPerSec = numResultBytes / perf.runtimeSec;
+    }
+
+    if (eventStart >= 0) {
+      perf.startupDelaySec = (eventStart - startMsSinceEpoch) / 1000.0;
+    }
+
+    if (resultStart >= 0 && eventStart >= 0 && resultStart >= eventStart) {
+      perf.processingDelaySec = (resultStart - eventStart) / 1000.0;
+    }
+
+    if (timestampStart >= 0 && timestampEnd >= 0 && perf.runtimeSec > 0.0) {
+      double eventRuntimeSec = (timestampEnd - timestampStart) / 1000.0;
+      perf.timeDilation = eventRuntimeSec / perf.runtimeSec;
+    }
+
+    if (resultEnd >= 0) {
+      // Fill in the shutdown delay assuming the job has now finished.
+      perf.shutdownDelaySec = (now - resultEnd) / 1000.0;
+    }
+
+    // As soon as available, try to capture cumulative cost at this point too.
+
+    NexmarkPerf.ProgressSnapshot snapshot = new NexmarkPerf.ProgressSnapshot();
+    snapshot.secSinceStart = (now - startMsSinceEpoch) / 1000.0;
+    snapshot.runtimeSec = perf.runtimeSec;
+    snapshot.numEvents = numEvents;
+    snapshot.numResults = numResults;
+    snapshots.add(snapshot);
+
+    captureSteadyState(perf, snapshots);
+
+    return perf;
+  }
+
+  /**
+   * Build and run a pipeline using specified options.
+   */
+  interface PipelineBuilder<OptionT extends NexmarkOptions> {
+    void build(OptionT publishOnlyOptions);
+  }
+
+  /**
+   * Invoke the builder with options suitable for running a publish-only child pipeline.
+   */
+  private void invokeBuilderForPublishOnlyPipeline(PipelineBuilder<NexmarkOptions> builder) {
+    builder.build(options);
+  }
+
+  /**
+   * Monitor the performance and progress of a running job. Return final performance if
+   * it was measured.
+   */
+  @Nullable
+  private NexmarkPerf monitor(NexmarkQuery query) {
+    if (!options.getMonitorJobs()) {
+      return null;
+    }
+
+    if (configuration.debug) {
+      NexmarkUtils.console("Waiting for main pipeline to 'finish'");
+    } else {
+      NexmarkUtils.console("--debug=false, so job will not self-cancel");
+    }
+
+    PipelineResult job = mainResult;
+    PipelineResult publisherJob = publisherResult;
+    List<NexmarkPerf.ProgressSnapshot> snapshots = new ArrayList<>();
+    long startMsSinceEpoch = System.currentTimeMillis();
+    long endMsSinceEpoch = -1;
+    if (options.getRunningTimeMinutes() != null) {
+      endMsSinceEpoch = startMsSinceEpoch
+                        + Duration.standardMinutes(options.getRunningTimeMinutes()).getMillis()
+                        - Duration.standardSeconds(configuration.preloadSeconds).getMillis();
+    }
+    long lastActivityMsSinceEpoch = -1;
+    NexmarkPerf perf = null;
+    boolean waitingForShutdown = false;
+    boolean publisherCancelled = false;
+    List<String> errors = new ArrayList<>();
+
+    while (true) {
+      long now = System.currentTimeMillis();
+      if (endMsSinceEpoch >= 0 && now > endMsSinceEpoch && !waitingForShutdown) {
+        NexmarkUtils.console("Reached end of test, cancelling job");
+        try {
+          job.cancel();
+        } catch (IOException e) {
+          throw new RuntimeException("Unable to cancel main job: ", e);
+        }
+        if (publisherResult != null) {
+          try {
+            publisherJob.cancel();
+          } catch (IOException e) {
+            throw new RuntimeException("Unable to cancel publisher job: ", e);
+          }
+          publisherCancelled = true;
+        }
+        waitingForShutdown = true;
+      }
+
+      PipelineResult.State state = job.getState();
+      NexmarkUtils.console("%s %s%s", state, queryName,
+          waitingForShutdown ? " (waiting for shutdown)" : "");
+
+      NexmarkPerf currPerf;
+      if (configuration.debug) {
+        currPerf = currentPerf(startMsSinceEpoch, now, job, snapshots,
+                               query.eventMonitor, query.resultMonitor);
+      } else {
+        currPerf = null;
+      }
+
+      if (perf == null || perf.anyActivity(currPerf)) {
+        lastActivityMsSinceEpoch = now;
+      }
+
+      if (options.isStreaming() && !waitingForShutdown) {
+        Duration quietFor = new Duration(lastActivityMsSinceEpoch, now);
+        long fatalCount = getCounterMetric(job, query.getName(), "fatal", 0);
+        if (fatalCount > 0) {
+          NexmarkUtils.console("job has fatal errors, cancelling.");
+          errors.add(String.format("Pipeline reported %s fatal errors", fatalCount));
+          waitingForShutdown = true;
+        } else if (configuration.debug && configuration.numEvents > 0
+                   && currPerf.numEvents == configuration.numEvents
+                   && currPerf.numResults >= 0 && quietFor.isLongerThan(DONE_DELAY)) {
+          NexmarkUtils.console("streaming query appears to have finished, cancelling job.");
+          waitingForShutdown = true;
+        } else if (quietFor.isLongerThan(STUCK_TERMINATE_DELAY)) {
+          NexmarkUtils.console("streaming query appears to have gotten stuck, cancelling job.");
+          errors.add("Streaming job was cancelled since appeared stuck");
+          waitingForShutdown = true;
+        } else if (quietFor.isLongerThan(STUCK_WARNING_DELAY)) {
+          NexmarkUtils.console("WARNING: streaming query appears to have been stuck for %d min.",
+              quietFor.getStandardMinutes());
+          errors.add(
+              String.format("Streaming query was stuck for %d min", quietFor.getStandardMinutes()));
+        }
+
+        if (waitingForShutdown) {
+          try {
+            job.cancel();
+          } catch (IOException e) {
+            throw new RuntimeException("Unable to cancel main job: ", e);
+          }
+        }
+      }
+
+      perf = currPerf;
+
+      boolean running = true;
+      switch (state) {
+        case UNKNOWN:
+        case STOPPED:
+        case RUNNING:
+          // Keep going.
+          break;
+        case DONE:
+          // All done.
+          running = false;
+          break;
+        case CANCELLED:
+          running = false;
+          if (!waitingForShutdown) {
+            errors.add("Job was unexpectedly cancelled");
+          }
+          break;
+        case FAILED:
+        case UPDATED:
+          // Abnormal termination.
+          running = false;
+          errors.add("Job was unexpectedly updated");
+          break;
+      }
+
+      if (!running) {
+        break;
+      }
+
+      if (lastActivityMsSinceEpoch == now) {
+        NexmarkUtils.console("new perf %s", perf);
+      } else {
+        NexmarkUtils.console("no activity");
+      }
+
+      try {
+        Thread.sleep(PERF_DELAY.getMillis());
+      } catch (InterruptedException e) {
+        Thread.interrupted();
+        NexmarkUtils.console("Interrupted: pipeline is still running");
+      }
+    }
+
+    perf.errors = errors;
+    perf.snapshots = snapshots;
+
+    if (publisherResult != null) {
+      NexmarkUtils.console("Shutting down publisher pipeline.");
+      try {
+        if (!publisherCancelled) {
+          publisherJob.cancel();
+        }
+        publisherJob.waitUntilFinish(Duration.standardMinutes(5));
+      } catch (IOException e) {
+        throw new RuntimeException("Unable to cancel publisher job: ", e);
+      }
+    }
+
+    return perf;
+  }
+
+  // ================================================================================
+  // Basic sources and sinks
+  // ================================================================================
+
+  /**
+   * Return a topic name.
+   */
+  private String shortTopic(long now) {
+    String baseTopic = options.getPubsubTopic();
+    if (Strings.isNullOrEmpty(baseTopic)) {
+      throw new RuntimeException("Missing --pubsubTopic");
+    }
+    switch (options.getResourceNameMode()) {
+      case VERBATIM:
+        return baseTopic;
+      case QUERY:
+        return String.format("%s_%s_source", baseTopic, queryName);
+      case QUERY_AND_SALT:
+        return String.format("%s_%s_%d_source", baseTopic, queryName, now);
+    }
+    throw new RuntimeException("Unrecognized enum " + options.getResourceNameMode());
+  }
+
+  /**
+   * Return a subscription name.
+   */
+  private String shortSubscription(long now) {
+    String baseSubscription = options.getPubsubSubscription();
+    if (Strings.isNullOrEmpty(baseSubscription)) {
+      throw new RuntimeException("Missing --pubsubSubscription");
+    }
+    switch (options.getResourceNameMode()) {
+      case VERBATIM:
+        return baseSubscription;
+      case QUERY:
+        return String.format("%s_%s_source", baseSubscription, queryName);
+      case QUERY_AND_SALT:
+        return String.format("%s_%s_%d_source", baseSubscription, queryName, now);
+    }
+    throw new RuntimeException("Unrecognized enum " + options.getResourceNameMode());
+  }
+
+  /**
+   * Return a file name for plain text.
+   */
+  private String textFilename(long now) {
+    String baseFilename = options.getOutputPath();
+    if (Strings.isNullOrEmpty(baseFilename)) {
+      throw new RuntimeException("Missing --outputPath");
+    }
+    switch (options.getResourceNameMode()) {
+      case VERBATIM:
+        return baseFilename;
+      case QUERY:
+        return String.format("%s/nexmark_%s.txt", baseFilename, queryName);
+      case QUERY_AND_SALT:
+        return String.format("%s/nexmark_%s_%d.txt", baseFilename, queryName, now);
+    }
+    throw new RuntimeException("Unrecognized enum " + options.getResourceNameMode());
+  }
+
+  /**
+   * Return a BigQuery table spec.
+   */
+  private String tableSpec(long now, String version) {
+    String baseTableName = options.getBigQueryTable();
+    if (Strings.isNullOrEmpty(baseTableName)) {
+      throw new RuntimeException("Missing --bigQueryTable");
+    }
+    switch (options.getResourceNameMode()) {
+      case VERBATIM:
+        return String.format("%s:nexmark.%s_%s",
+                             options.getProject(), baseTableName, version);
+      case QUERY:
+        return String.format("%s:nexmark.%s_%s_%s",
+                             options.getProject(), baseTableName, queryName, version);
+      case QUERY_AND_SALT:
+        return String.format("%s:nexmark.%s_%s_%s_%d",
+                             options.getProject(), baseTableName, queryName, version, now);
+    }
+    throw new RuntimeException("Unrecognized enum " + options.getResourceNameMode());
+  }
+
+  /**
+   * Return a directory for logs.
+   */
+  private String logsDir(long now) {
+    String baseFilename = options.getOutputPath();
+    if (Strings.isNullOrEmpty(baseFilename)) {
+      throw new RuntimeException("Missing --outputPath");
+    }
+    switch (options.getResourceNameMode()) {
+      case VERBATIM:
+        return baseFilename;
+      case QUERY:
+        return String.format("%s/logs_%s", baseFilename, queryName);
+      case QUERY_AND_SALT:
+        return String.format("%s/logs_%s_%d", baseFilename, queryName, now);
+    }
+    throw new RuntimeException("Unrecognized enum " + options.getResourceNameMode());
+  }
+
+  /**
+   * Return a source of synthetic events.
+   */
+  private PCollection<Event> sourceEventsFromSynthetic(Pipeline p) {
+    if (isStreaming()) {
+      NexmarkUtils.console("Generating %d events in streaming mode", configuration.numEvents);
+      return p.apply(queryName + ".ReadUnbounded", NexmarkUtils.streamEventsSource(configuration));
+    } else {
+      NexmarkUtils.console("Generating %d events in batch mode", configuration.numEvents);
+      return p.apply(queryName + ".ReadBounded", NexmarkUtils.batchEventsSource(configuration));
+    }
+  }
+
+  /**
+   * Return source of events from Pubsub.
+   */
+  private PCollection<Event> sourceEventsFromPubsub(Pipeline p, long now) {
+    String shortSubscription = shortSubscription(now);
+    NexmarkUtils.console("Reading events from Pubsub %s", shortSubscription);
+
+    PubsubIO.Read<PubsubMessage> io =
+        PubsubIO.readMessagesWithAttributes().fromSubscription(shortSubscription)
+            .withIdAttribute(NexmarkUtils.PUBSUB_ID);
+    if (!configuration.usePubsubPublishTime) {
+      io = io.withTimestampAttribute(NexmarkUtils.PUBSUB_TIMESTAMP);
+    }
+
+    return p
+      .apply(queryName + ".ReadPubsubEvents", io)
+      .apply(queryName + ".PubsubMessageToEvent", ParDo.of(new DoFn<PubsubMessage, Event>() {
+        @ProcessElement
+        public void processElement(ProcessContext c) {
+          byte[] payload = c.element().getPayload();
+          try {
+            Event event = CoderUtils.decodeFromByteArray(Event.CODER, payload);
+            c.output(event);
+          } catch (CoderException e) {
+            LOG.error("Error while decoding Event from pusbSub message: serialization error");
+          }
+        }
+      }));
+  }
+
+  /**
+   * Return Avro source of events from {@code options.getInputFilePrefix}.
+   */
+  private PCollection<Event> sourceEventsFromAvro(Pipeline p) {
+    String filename = options.getInputPath();
+    if (Strings.isNullOrEmpty(filename)) {
+      throw new RuntimeException("Missing --inputPath");
+    }
+    NexmarkUtils.console("Reading events from Avro files at %s", filename);
+    return p
+        .apply(queryName + ".ReadAvroEvents", AvroIO.read(Event.class)
+                          .from(filename + "*.avro"))
+        .apply("OutputWithTimestamp", NexmarkQuery.EVENT_TIMESTAMP_FROM_DATA);
+  }
+
+  /**
+   * Send {@code events} to Pubsub.
+   */
+  private void sinkEventsToPubsub(PCollection<Event> events, long now) {
+    String shortTopic = shortTopic(now);
+    NexmarkUtils.console("Writing events to Pubsub %s", shortTopic);
+
+    PubsubIO.Write<PubsubMessage> io =
+        PubsubIO.writeMessages().to(shortTopic)
+            .withIdAttribute(NexmarkUtils.PUBSUB_ID);
+    if (!configuration.usePubsubPublishTime) {
+      io = io.withTimestampAttribute(NexmarkUtils.PUBSUB_TIMESTAMP);
+    }
+
+    events.apply(queryName + ".EventToPubsubMessage",
+            ParDo.of(new DoFn<Event, PubsubMessage>() {
+              @ProcessElement
+              public void processElement(ProcessContext c) {
+                try {
+                  byte[] payload = CoderUtils.encodeToByteArray(Event.CODER, c.element());
+                  c.output(new PubsubMessage(payload, new HashMap<String, String>()));
+                } catch (CoderException e1) {
+                  LOG.error("Error while sending Event {} to pusbSub: serialization error",
+                      c.element().toString());
+                }
+              }
+            })
+        )
+        .apply(queryName + ".WritePubsubEvents", io);
+  }
+
+  /**
+   * Send {@code formattedResults} to Pubsub.
+   */
+  private void sinkResultsToPubsub(PCollection<String> formattedResults, long now) {
+    String shortTopic = shortTopic(now);
+    NexmarkUtils.console("Writing results to Pubsub %s", shortTopic);
+    PubsubIO.Write<String> io =
+        PubsubIO.writeStrings().to(shortTopic)
+            .withIdAttribute(NexmarkUtils.PUBSUB_ID);
+    if (!configuration.usePubsubPublishTime) {
+      io = io.withTimestampAttribute(NexmarkUtils.PUBSUB_TIMESTAMP);
+    }
+    formattedResults.apply(queryName + ".WritePubsubResults", io);
+  }
+
+  /**
+   * Sink all raw Events in {@code source} to {@code options.getOutputPath}.
+   * This will configure the job to write the following files:
+   * <ul>
+   * <li>{@code $outputPath/event*.avro} All Event entities.
+   * <li>{@code $outputPath/auction*.avro} Auction entities.
+   * <li>{@code $outputPath/bid*.avro} Bid entities.
+   * <li>{@code $outputPath/person*.avro} Person entities.
+   * </ul>
+   *
+   * @param source A PCollection of events.
+   */
+  private void sinkEventsToAvro(PCollection<Event> source) {
+    String filename = options.getOutputPath();
+    if (Strings.isNullOrEmpty(filename)) {
+      throw new RuntimeException("Missing --outputPath");
+    }
+    NexmarkUtils.console("Writing events to Avro files at %s", filename);
+    source.apply(queryName + ".WriteAvroEvents",
+            AvroIO.write(Event.class).to(filename + "/event").withSuffix(".avro"));
+    source.apply(NexmarkQuery.JUST_BIDS)
+          .apply(queryName + ".WriteAvroBids",
+            AvroIO.write(Bid.class).to(filename + "/bid").withSuffix(".avro"));
+    source.apply(NexmarkQuery.JUST_NEW_AUCTIONS)
+          .apply(queryName + ".WriteAvroAuctions",
+            AvroIO.write(Auction.class).to(filename + "/auction").withSuffix(".avro"));
+    source.apply(NexmarkQuery.JUST_NEW_PERSONS)
+          .apply(queryName + ".WriteAvroPeople",
+            AvroIO.write(Person.class).to(filename + "/person").withSuffix(".avro"));
+  }
+
+  /**
+   * Send {@code formattedResults} to text files.
+   */
+  private void sinkResultsToText(PCollection<String> formattedResults, long now) {
+    String filename = textFilename(now);
+    NexmarkUtils.console("Writing results to text files at %s", filename);
+    formattedResults.apply(queryName + ".WriteTextResults",
+        TextIO.write().to(filename));
+  }
+
+  private static class StringToTableRow extends DoFn<String, TableRow> {
+    @ProcessElement
+    public void processElement(ProcessContext c) {
+      int n = ThreadLocalRandom.current().nextInt(10);
+      List<TableRow> records = new ArrayList<>(n);
+      for (int i = 0; i < n; i++) {
+        records.add(new TableRow().set("index", i).set("value", Integer.toString(i)));
+      }
+      c.output(new TableRow().set("result", c.element()).set("records", records));
+    }
+  }
+
+  /**
+   * Send {@code formattedResults} to BigQuery.
+   */
+  private void sinkResultsToBigQuery(
+      PCollection<String> formattedResults, long now,
+      String version) {
+    String tableSpec = tableSpec(now, version);
+    TableSchema tableSchema =
+        new TableSchema().setFields(ImmutableList.of(
+            new TableFieldSchema().setName("result").setType("STRING"),
+            new TableFieldSchema().setName("records").setMode("REPEATED").setType("RECORD")
+                                  .setFields(ImmutableList.of(
+                                      new TableFieldSchema().setName("index").setType("INTEGER"),
+                                      new TableFieldSchema().setName("value").setType("STRING")))));
+    NexmarkUtils.console("Writing results to BigQuery table %s", tableSpec);
+    BigQueryIO.Write io =
+        BigQueryIO.write().to(tableSpec)
+                        .withSchema(tableSchema)
+                        .withCreateDisposition(BigQueryIO.Write.CreateDisposition.CREATE_IF_NEEDED)
+                        .withWriteDisposition(BigQueryIO.Write.WriteDisposition.WRITE_APPEND);
+    formattedResults
+        .apply(queryName + ".StringToTableRow", ParDo.of(new StringToTableRow()))
+        .apply(queryName + ".WriteBigQueryResults", io);
+  }
+
+  // ================================================================================
+  // Construct overall pipeline
+  // ================================================================================
+
+  /**
+   * Return source of events for this run, or null if we are simply publishing events
+   * to Pubsub.
+   */
+  private PCollection<Event> createSource(Pipeline p, final long now) {
+    PCollection<Event> source = null;
+    switch (configuration.sourceType) {
+      case DIRECT:
+        source = sourceEventsFromSynthetic(p);
+        break;
+      case AVRO:
+        source = sourceEventsFromAvro(p);
+        break;
+      case PUBSUB:
+        // Setup the sink for the publisher.
+        switch (configuration.pubSubMode) {
+          case SUBSCRIBE_ONLY:
+            // Nothing to publish.
+            break;
+          case PUBLISH_ONLY:
+            // Send synthesized events to Pubsub in this job.
+            sinkEventsToPubsub(sourceEventsFromSynthetic(p).apply(queryName + ".Snoop",
+                    NexmarkUtils.snoop(queryName)), now);
+            break;
+          case COMBINED:
+            // Send synthesized events to Pubsub in separate publisher job.
+            // We won't start the main pipeline until the publisher has sent the pre-load events.
+            // We'll shutdown the publisher job when we notice the main job has finished.
+            invokeBuilderForPublishOnlyPipeline(new PipelineBuilder<NexmarkOptions>() {
+              @Override
+              public void build(NexmarkOptions publishOnlyOptions) {
+                Pipeline sp = Pipeline.create(options);
+                NexmarkUtils.setupPipeline(configuration.coderStrategy, sp);
+                publisherMonitor = new Monitor<>(queryName, "publisher");
+                sinkEventsToPubsub(
+                    sourceEventsFromSynthetic(sp)
+                            .apply(queryName + ".Monitor", publisherMonitor.getTransform()),
+                    now);
+                publisherResult = sp.run();
+              }
+            });
+            break;
+        }
+
+        // Setup the source for the consumer.
+        switch (configuration.pubSubMode) {
+          case PUBLISH_ONLY:
+            // Nothing to consume. Leave source null.
+            break;
+          case SUBSCRIBE_ONLY:
+          case COMBINED:
+            // Read events from pubsub.
+            source = sourceEventsFromPubsub(p, now);
+            break;
+        }
+        break;
+    }
+    return source;
+  }
+
+  private static final TupleTag<String> MAIN = new TupleTag<String>(){};
+  private static final TupleTag<String> SIDE = new TupleTag<String>(){};
+
+  private static class PartitionDoFn extends DoFn<String, String> {
+    @ProcessElement
+    public void processElement(ProcessContext c) {
+      if (c.element().hashCode() % 2 == 0) {
+        c.output(c.element());
+      } else {
+        c.output(SIDE, c.element());
+      }
+    }
+  }
+
+  /**
+   * Consume {@code results}.
+   */
+  private void sink(PCollection<TimestampedValue<KnownSize>> results, long now) {
+    if (configuration.sinkType == NexmarkUtils.SinkType.COUNT_ONLY) {
+      // Avoid the cost of formatting the results.
+      results.apply(queryName + ".DevNull", NexmarkUtils.devNull(queryName));
+      return;
+    }
+
+    PCollection<String> formattedResults =
+      results.apply(queryName + ".Format", NexmarkUtils.format(queryName));
+    if (options.getLogResults()) {
+      formattedResults = formattedResults.apply(queryName + ".Results.Log",
+              NexmarkUtils.<String>log(queryName + ".Results"));
+    }
+
+    switch (configuration.sinkType) {
+      case DEVNULL:
+        // Discard all results
+        formattedResults.apply(queryName + ".DevNull", NexmarkUtils.devNull(queryName));
+        break;
+      case PUBSUB:
+        sinkResultsToPubsub(formattedResults, now);
+        break;
+      case TEXT:
+        sinkResultsToText(formattedResults, now);
+        break;
+      case AVRO:
+        NexmarkUtils.console(
+            "WARNING: with --sinkType=AVRO, actual query results will be discarded.");
+        break;
+      case BIGQUERY:
+        // Multiple BigQuery backends to mimic what most customers do.
+        PCollectionTuple res = formattedResults.apply(queryName + ".Partition",
+            ParDo.of(new PartitionDoFn()).withOutputTags(MAIN, TupleTagList.of(SIDE)));
+        sinkResultsToBigQuery(res.get(MAIN), now, "main");
+        sinkResultsToBigQuery(res.get(SIDE), now, "side");
+        sinkResultsToBigQuery(formattedResults, now, "copy");
+        break;
+      case COUNT_ONLY:
+        // Short-circuited above.
+        throw new RuntimeException();
+    }
+  }
+
+  // ================================================================================
+  // Entry point
+  // ================================================================================
+
+  /**
+   * Calculate the distribution of the expected rate of results per minute (in event time, not
+   * wallclock time).
+   */
+  private void modelResultRates(NexmarkQueryModel model) {
+    List<Long> counts = Lists.newArrayList(model.simulator().resultsPerWindow());
+    Collections.sort(counts);
+    int n = counts.size();
+    if (n < 5) {
+      NexmarkUtils.console("Query%d: only %d samples", model.configuration.query, n);
+    } else {
+      NexmarkUtils.console("Query%d: N:%d; min:%d; 1st%%:%d; mean:%d; 3rd%%:%d; max:%d",
+                           model.configuration.query, n, counts.get(0), counts.get(n / 4),
+                           counts.get(n / 2),
+                           counts.get(n - 1 - n / 4), counts.get(n - 1));
+    }
+  }
+
+  /**
+   * Run {@code configuration} and return its performance if possible.
+   */
+  @Nullable
+  public NexmarkPerf run(NexmarkConfiguration runConfiguration) {
+    if (options.getManageResources() && !options.getMonitorJobs()) {
+      throw new RuntimeException("If using --manageResources then must also use --monitorJobs.");
+    }
+
+    //
+    // Setup per-run state.
+    //
+    checkState(configuration == null);
+    checkState(queryName == null);
+    configuration = runConfiguration;
+
+    try {
+      NexmarkUtils.console("Running %s", configuration.toShortString());
+
+      if (configuration.numEvents < 0) {
+        NexmarkUtils.console("skipping since configuration is disabled");
+        return null;
+      }
+
+      List<NexmarkQuery> queries = Arrays.asList(new Query0(configuration),
+                                                 new Query1(configuration),
+                                                 new Query2(configuration),
+                                                 new Query3(configuration),
+                                                 new Query4(configuration),
+                                                 new Query5(configuration),
+                                                 new Query6(configuration),
+                                                 new Query7(configuration),
+                                                 new Query8(configuration),
+                                                 new Query9(configuration),
+                                                 new Query10(configuration),
+                                                 new Query11(configuration),
+                                                 new Query12(configuration));
+      NexmarkQuery query = queries.get(configuration.query);
+      queryName = query.getName();
+
+      List<NexmarkQueryModel> models = Arrays.asList(
+          new Query0Model(configuration),
+          new Query1Model(configuration),
+          new Query2Model(configuration),
+          new Query3Model(configuration),
+          new Query4Model(configuration),
+          new Query5Model(configuration),
+          new Query6Model(configuration),
+          new Query7Model(configuration),
+          new Query8Model(configuration),
+          new Query9Model(configuration),
+          null,
+          null,
+          null);
+      NexmarkQueryModel model = models.get(configuration.query);
+
+      if (options.getJustModelResultRate()) {
+        if (model == null) {
+          throw new RuntimeException(String.format("No model for %s", queryName));
+        }
+        modelResultRates(model);
+        return null;
+      }
+
+      long now = System.currentTimeMillis();
+      Pipeline p = Pipeline.create(options);
+      NexmarkUtils.setupPipeline(configuration.coderStrategy, p);
+
+      // Generate events.
+      PCollection<Event> source = createSource(p, now);
+
+      if (options.getLogEvents()) {
+        source = source.apply(queryName + ".Events.Log",
+                NexmarkUtils.<Event>log(queryName + ".Events"));
+      }
+
+      // Source will be null if source type is PUBSUB and mode is PUBLISH_ONLY.
+      // In that case there's nothing more to add to pipeline.
+      if (source != null) {
+        // Optionally sink events in Avro format.
+        // (Query results are ignored).
+        if (configuration.sinkType == NexmarkUtils.SinkType.AVRO) {
+          sinkEventsToAvro(source);
+        }
+
+        // Query 10 logs all events to Google Cloud storage files. It could generate a lot of logs,
+        // so, set parallelism. Also set the output path where to write log files.
+        if (configuration.query == 10) {
+          String path = null;
+          if (options.getOutputPath() != null && !options.getOutputPath().isEmpty()) {
+            path = logsDir(now);
+          }
+          ((Query10) query).setOutputPath(path);
+          ((Query10) query).setMaxNumWorkers(maxNumWorkers());
+        }
+
+        // Apply query.
+        PCollection<TimestampedValue<KnownSize>> results = source.apply(query);
+
+        if (options.getAssertCorrectness()) {
+          if (model == null) {
+            throw new RuntimeException(String.format("No model for %s", queryName));
+          }
+          // We know all our streams have a finite number of elements.
+          results.setIsBoundedInternal(PCollection.IsBounded.BOUNDED);
+          // If we have a finite number of events then assert our pipeline's
+          // results match those of a model using the same sequence of events.
+          PAssert.that(results).satisfies(model.assertionFor());
+        }
+
+        // Output results.
+        sink(results, now);
+      }
+
+      mainResult = p.run();
+      mainResult.waitUntilFinish(Duration.standardSeconds(configuration.streamTimeout));
+      return monitor(query);
+    } finally {
+      configuration = null;
+      queryName = null;
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/f4333df7/sdks/java/nexmark/src/main/java/org/apache/beam/sdk/nexmark/NexmarkOptions.java
----------------------------------------------------------------------
diff --git a/sdks/java/nexmark/src/main/java/org/apache/beam/sdk/nexmark/NexmarkOptions.java b/sdks/java/nexmark/src/main/java/org/apache/beam/sdk/nexmark/NexmarkOptions.java
new file mode 100644
index 0000000..2a2a5a7
--- /dev/null
+++ b/sdks/java/nexmark/src/main/java/org/apache/beam/sdk/nexmark/NexmarkOptions.java
@@ -0,0 +1,403 @@
+/*
+ * 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.beam.sdk.nexmark;
+
+import javax.annotation.Nullable;
+import org.apache.beam.sdk.extensions.gcp.options.GcpOptions;
+import org.apache.beam.sdk.options.ApplicationNameOptions;
+import org.apache.beam.sdk.options.Default;
+import org.apache.beam.sdk.options.Description;
+import org.apache.beam.sdk.options.PipelineOptions;
+import org.apache.beam.sdk.options.StreamingOptions;
+
+/**
+ * Command line flags.
+ */
+public interface NexmarkOptions
+    extends ApplicationNameOptions, GcpOptions, PipelineOptions, StreamingOptions {
+  @Description("Which suite to run. Default is to use command line arguments for one job.")
+  @Default.Enum("DEFAULT")
+  NexmarkSuite getSuite();
+
+  void setSuite(NexmarkSuite suite);
+
+  @Description("If true, monitor the jobs as they run.")
+  @Default.Boolean(false)
+  boolean getMonitorJobs();
+
+  void setMonitorJobs(boolean monitorJobs);
+
+  @Description("Where the events come from.")
+  @Nullable
+  NexmarkUtils.SourceType getSourceType();
+
+  void setSourceType(NexmarkUtils.SourceType sourceType);
+
+  @Description("Prefix for input files if using avro input")
+  @Nullable
+  String getInputPath();
+
+  void setInputPath(String inputPath);
+
+  @Description("Where results go.")
+  @Nullable
+  NexmarkUtils.SinkType getSinkType();
+
+  void setSinkType(NexmarkUtils.SinkType sinkType);
+
+  @Description("Which mode to run in when source is PUBSUB.")
+  @Nullable
+  NexmarkUtils.PubSubMode getPubSubMode();
+
+  void setPubSubMode(NexmarkUtils.PubSubMode pubSubMode);
+
+  @Description("Which query to run.")
+  @Nullable
+  Integer getQuery();
+
+  void setQuery(Integer query);
+
+  @Description("Prefix for output files if using text output for results or running Query 10.")
+  @Nullable
+  String getOutputPath();
+
+  void setOutputPath(String outputPath);
+
+  @Description("Base name of pubsub topic to publish to in streaming mode.")
+  @Nullable
+  @Default.String("nexmark")
+  String getPubsubTopic();
+
+  void setPubsubTopic(String pubsubTopic);
+
+  @Description("Base name of pubsub subscription to read from in streaming mode.")
+  @Nullable
+  @Default.String("nexmark")
+  String getPubsubSubscription();
+
+  void setPubsubSubscription(String pubsubSubscription);
+
+  @Description("Base name of BigQuery table name if using BigQuery output.")
+  @Nullable
+  @Default.String("nexmark")
+  String getBigQueryTable();
+
+  void setBigQueryTable(String bigQueryTable);
+
+  @Description("Approximate number of events to generate. "
+               + "Zero for effectively unlimited in streaming mode.")
+  @Nullable
+  Long getNumEvents();
+
+  void setNumEvents(Long numEvents);
+
+  @Description("Time in seconds to preload the subscription with data, at the initial input rate "
+               + "of the pipeline.")
+  @Nullable
+  Integer getPreloadSeconds();
+
+  void setPreloadSeconds(Integer preloadSeconds);
+
+  @Description(
+      "Time in seconds to wait in pipelineResult.waitUntilFinish(), useful in streaming mode")
+  @Nullable
+  Integer getStreamTimeout();
+
+  void setStreamTimeout(Integer streamTimeout);
+
+  @Description("Number of unbounded sources to create events.")
+  @Nullable
+  Integer getNumEventGenerators();
+
+  void setNumEventGenerators(Integer numEventGenerators);
+
+  @Description("Shape of event rate curve.")
+  @Nullable
+  NexmarkUtils.RateShape getRateShape();
+
+  void setRateShape(NexmarkUtils.RateShape rateShape);
+
+  @Description("Initial overall event rate (in --rateUnit).")
+  @Nullable
+  Integer getFirstEventRate();
+
+  void setFirstEventRate(Integer firstEventRate);
+
+  @Description("Next overall event rate (in --rateUnit).")
+  @Nullable
+  Integer getNextEventRate();
+
+  void setNextEventRate(Integer nextEventRate);
+
+  @Description("Unit for rates.")
+  @Nullable
+  NexmarkUtils.RateUnit getRateUnit();
+
+  void setRateUnit(NexmarkUtils.RateUnit rateUnit);
+
+  @Description("Overall period of rate shape, in seconds.")
+  @Nullable
+  Integer getRatePeriodSec();
+
+  void setRatePeriodSec(Integer ratePeriodSec);
+
+  @Description("If true, relay events in real time in streaming mode.")
+  @Nullable
+  Boolean getIsRateLimited();
+
+  void setIsRateLimited(Boolean isRateLimited);
+
+  @Description("If true, use wallclock time as event time. Otherwise, use a deterministic"
+               + " time in the past so that multiple runs will see exactly the same event streams"
+               + " and should thus have exactly the same results.")
+  @Nullable
+  Boolean getUseWallclockEventTime();
+
+  void setUseWallclockEventTime(Boolean useWallclockEventTime);
+
+  @Description("Assert pipeline results match model results.")
+  @Nullable
+  boolean getAssertCorrectness();
+
+  void setAssertCorrectness(boolean assertCorrectness);
+
+  @Description("Log all input events.")
+  @Nullable
+  boolean getLogEvents();
+
+  void setLogEvents(boolean logEvents);
+
+  @Description("Log all query results.")
+  @Nullable
+  boolean getLogResults();
+
+  void setLogResults(boolean logResults);
+
+  @Description("Average size in bytes for a person record.")
+  @Nullable
+  Integer getAvgPersonByteSize();
+
+  void setAvgPersonByteSize(Integer avgPersonByteSize);
+
+  @Description("Average size in bytes for an auction record.")
+  @Nullable
+  Integer getAvgAuctionByteSize();
+
+  void setAvgAuctionByteSize(Integer avgAuctionByteSize);
+
+  @Description("Average size in bytes for a bid record.")
+  @Nullable
+  Integer getAvgBidByteSize();
+
+  void setAvgBidByteSize(Integer avgBidByteSize);
+
+  @Description("Ratio of bids for 'hot' auctions above the background.")
+  @Nullable
+  Integer getHotAuctionRatio();
+
+  void setHotAuctionRatio(Integer hotAuctionRatio);
+
+  @Description("Ratio of auctions for 'hot' sellers above the background.")
+  @Nullable
+  Integer getHotSellersRatio();
+
+  void setHotSellersRatio(Integer hotSellersRatio);
+
+  @Description("Ratio of auctions for 'hot' bidders above the background.")
+  @Nullable
+  Integer getHotBiddersRatio();
+
+  void setHotBiddersRatio(Integer hotBiddersRatio);
+
+  @Description("Window size in seconds.")
+  @Nullable
+  Long getWindowSizeSec();
+
+  void setWindowSizeSec(Long windowSizeSec);
+
+  @Description("Window period in seconds.")
+  @Nullable
+  Long getWindowPeriodSec();
+
+  void setWindowPeriodSec(Long windowPeriodSec);
+
+  @Description("If in streaming mode, the holdback for watermark in seconds.")
+  @Nullable
+  Long getWatermarkHoldbackSec();
+
+  void setWatermarkHoldbackSec(Long watermarkHoldbackSec);
+
+  @Description("Roughly how many auctions should be in flight for each generator.")
+  @Nullable
+  Integer getNumInFlightAuctions();
+
+  void setNumInFlightAuctions(Integer numInFlightAuctions);
+
+
+  @Description("Maximum number of people to consider as active for placing auctions or bids.")
+  @Nullable
+  Integer getNumActivePeople();
+
+  void setNumActivePeople(Integer numActivePeople);
+
+  @Description("Filename of perf data to append to.")
+  @Nullable
+  String getPerfFilename();
+
+  void setPerfFilename(String perfFilename);
+
+  @Description("Filename of baseline perf data to read from.")
+  @Nullable
+  String getBaselineFilename();
+
+  void setBaselineFilename(String baselineFilename);
+
+  @Description("Filename of summary perf data to append to.")
+  @Nullable
+  String getSummaryFilename();
+
+  void setSummaryFilename(String summaryFilename);
+
+  @Description("Filename for javascript capturing all perf data and any baselines.")
+  @Nullable
+  String getJavascriptFilename();
+
+  void setJavascriptFilename(String javascriptFilename);
+
+  @Description("If true, don't run the actual query. Instead, calculate the distribution "
+               + "of number of query results per (event time) minute according to the query model.")
+  @Nullable
+  boolean getJustModelResultRate();
+
+  void setJustModelResultRate(boolean justModelResultRate);
+
+  @Description("Coder strategy to use.")
+  @Nullable
+  NexmarkUtils.CoderStrategy getCoderStrategy();
+
+  void setCoderStrategy(NexmarkUtils.CoderStrategy coderStrategy);
+
+  @Description("Delay, in milliseconds, for each event. We will peg one core for this "
+               + "number of milliseconds to simulate CPU-bound computation.")
+  @Nullable
+  Long getCpuDelayMs();
+
+  void setCpuDelayMs(Long cpuDelayMs);
+
+  @Description("Extra data, in bytes, to save to persistent state for each event. "
+               + "This will force I/O all the way to durable storage to simulate an "
+               + "I/O-bound computation.")
+  @Nullable
+  Long getDiskBusyBytes();
+
+  void setDiskBusyBytes(Long diskBusyBytes);
+
+  @Description("Skip factor for query 2. We select bids for every {@code auctionSkip}'th auction")
+  @Nullable
+  Integer getAuctionSkip();
+
+  void setAuctionSkip(Integer auctionSkip);
+
+  @Description("Fanout for queries 4 (groups by category id) and 7 (finds a global maximum).")
+  @Nullable
+  Integer getFanout();
+
+  void setFanout(Integer fanout);
+
+  @Description("Maximum waiting time to clean personState in query3 "
+      + "(ie maximum waiting of the auctions related to person in state in seconds in event time).")
+  @Nullable
+  Integer getMaxAuctionsWaitingTime();
+
+  void setMaxAuctionsWaitingTime(Integer fanout);
+
+  @Description("Length of occasional delay to impose on events (in seconds).")
+  @Nullable
+  Long getOccasionalDelaySec();
+
+  void setOccasionalDelaySec(Long occasionalDelaySec);
+
+  @Description("Probability that an event will be delayed by delayS.")
+  @Nullable
+  Double getProbDelayedEvent();
+
+  void setProbDelayedEvent(Double probDelayedEvent);
+
+  @Description("Maximum size of each log file (in events). For Query10 only.")
+  @Nullable
+  Integer getMaxLogEvents();
+
+  void setMaxLogEvents(Integer maxLogEvents);
+
+  @Description("How to derive names of resources.")
+  @Default.Enum("QUERY_AND_SALT")
+  NexmarkUtils.ResourceNameMode getResourceNameMode();
+
+  void setResourceNameMode(NexmarkUtils.ResourceNameMode mode);
+
+  @Description("If true, manage the creation and cleanup of topics, subscriptions and gcs files.")
+  @Default.Boolean(true)
+  boolean getManageResources();
+
+  void setManageResources(boolean manageResources);
+
+  @Description("If true, use pub/sub publish time instead of event time.")
+  @Nullable
+  Boolean getUsePubsubPublishTime();
+
+  void setUsePubsubPublishTime(Boolean usePubsubPublishTime);
+
+  @Description("Number of events in out-of-order groups. 1 implies no out-of-order events. "
+               + "1000 implies every 1000 events per generator are emitted in pseudo-random order.")
+  @Nullable
+  Long getOutOfOrderGroupSize();
+
+  void setOutOfOrderGroupSize(Long outOfOrderGroupSize);
+
+  @Description("If false, do not add the Monitor and Snoop transforms.")
+  @Nullable
+  Boolean getDebug();
+
+  void setDebug(Boolean value);
+
+  @Description("If set, cancel running pipelines after this long")
+  @Nullable
+  Long getRunningTimeMinutes();
+
+  void setRunningTimeMinutes(Long value);
+
+  @Description("If set and --monitorJobs is true, check that the system watermark is never more "
+               + "than this far behind real time")
+  @Nullable
+  Long getMaxSystemLagSeconds();
+
+  void setMaxSystemLagSeconds(Long value);
+
+  @Description("If set and --monitorJobs is true, check that the data watermark is never more "
+               + "than this far behind real time")
+  @Nullable
+  Long getMaxDataLagSeconds();
+
+  void setMaxDataLagSeconds(Long value);
+
+  @Description("Only start validating watermarks after this many seconds")
+  @Nullable
+  Long getWatermarkValidationDelaySeconds();
+
+  void setWatermarkValidationDelaySeconds(Long value);
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/f4333df7/sdks/java/nexmark/src/main/java/org/apache/beam/sdk/nexmark/NexmarkPerf.java
----------------------------------------------------------------------
diff --git a/sdks/java/nexmark/src/main/java/org/apache/beam/sdk/nexmark/NexmarkPerf.java b/sdks/java/nexmark/src/main/java/org/apache/beam/sdk/nexmark/NexmarkPerf.java
new file mode 100644
index 0000000..2edf4e8
--- /dev/null
+++ b/sdks/java/nexmark/src/main/java/org/apache/beam/sdk/nexmark/NexmarkPerf.java
@@ -0,0 +1,207 @@
+/*
+ * 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.beam.sdk.nexmark;
+
+import com.fasterxml.jackson.annotation.JsonProperty;
+import com.fasterxml.jackson.core.JsonProcessingException;
+import java.io.IOException;
+import java.util.List;
+import javax.annotation.Nullable;
+
+/**
+ * Summary of performance for a particular run of a configuration.
+ */
+public class NexmarkPerf {
+  /**
+   * A sample of the number of events and number of results (if known) generated at
+   * a particular time.
+   */
+  public static class ProgressSnapshot {
+    /** Seconds since job was started (in wallclock time). */
+    @JsonProperty
+    double secSinceStart;
+
+    /** Job runtime in seconds (time from first event to last generated event or output result). */
+    @JsonProperty
+    double runtimeSec;
+
+    /** Cumulative number of events generated. -1 if not known. */
+    @JsonProperty
+    long numEvents;
+
+    /** Cumulative number of results emitted. -1 if not known. */
+    @JsonProperty
+    long numResults;
+
+    /**
+     * Return true if there looks to be activity between {@code this} and {@code that}
+     * snapshots.
+     */
+    public boolean anyActivity(ProgressSnapshot that) {
+      if (runtimeSec != that.runtimeSec) {
+        // An event or result end timestamp looks to have changed.
+        return true;
+      }
+      if (numEvents != that.numEvents) {
+        // Some more events were generated.
+        return true;
+      }
+      if (numResults != that.numResults) {
+        // Some more results were emitted.
+        return true;
+      }
+      return false;
+    }
+  }
+
+  /**
+   * Progess snapshots. Null if not yet calculated.
+   */
+  @JsonProperty
+  @Nullable
+  public List<ProgressSnapshot> snapshots = null;
+
+  /**
+   * Effective runtime, in seconds. Measured from timestamp of first generated event to latest of
+   * timestamp of last generated event and last emitted result. -1 if not known.
+   */
+  @JsonProperty
+  public double runtimeSec = -1.0;
+
+  /**
+   * Number of events generated. -1 if not known.
+   */
+  @JsonProperty
+  public long numEvents = -1;
+
+  /**
+   * Number of events generated per second of runtime. For batch this is number of events
+   * over the above runtime. For streaming this is the 'steady-state' event generation rate sampled
+   * over the lifetime of the job. -1 if not known.
+   */
+  @JsonProperty
+  public double eventsPerSec = -1.0;
+
+  /**
+   * Number of event bytes generated per second of runtime. -1 if not known.
+   */
+  @JsonProperty
+  public double eventBytesPerSec = -1.0;
+
+  /**
+   * Number of results emitted. -1 if not known.
+   */
+  @JsonProperty
+  public long numResults = -1;
+
+  /**
+   * Number of results generated per second of runtime. -1 if not known.
+   */
+  @JsonProperty
+  public double resultsPerSec = -1.0;
+
+  /**
+   * Number of result bytes generated per second of runtime. -1 if not known.
+   */
+  @JsonProperty
+  public double resultBytesPerSec = -1.0;
+
+  /**
+   * Delay between start of job and first event in second. -1 if not known.
+   */
+  @JsonProperty
+  public double startupDelaySec = -1.0;
+
+  /**
+   * Delay between first event and first result in seconds. -1 if not known.
+   */
+  @JsonProperty
+  public double processingDelaySec = -1.0;
+
+  /**
+   * Delay between last result and job completion in seconds. -1 if not known.
+   */
+  @JsonProperty
+  public double shutdownDelaySec = -1.0;
+
+  /**
+   * Time-dilation factor.  Calculate as event time advancement rate relative to real time.
+   * Greater than one implies we processed events faster than they would have been generated
+   * in real time. Less than one implies we could not keep up with events in real time.
+   * -1 if not known.
+   */
+  @JsonProperty
+  double timeDilation = -1.0;
+
+  /**
+   * List of errors encountered during job execution.
+   */
+  @JsonProperty
+  @Nullable
+  public List<String> errors = null;
+
+  /**
+   * The job id this perf was drawn from. Null if not known.
+   */
+  @JsonProperty
+  @Nullable
+  public String jobId = null;
+
+  /**
+   * Return a JSON representation of performance.
+   */
+  @Override
+  public String toString() {
+    try {
+      return NexmarkUtils.MAPPER.writeValueAsString(this);
+    } catch (JsonProcessingException e) {
+      throw new RuntimeException(e);
+    }
+  }
+
+  /**
+   * Parse a {@link NexmarkPerf} object from JSON {@code string}.
+   */
+  public static NexmarkPerf fromString(String string) {
+    try {
+      return NexmarkUtils.MAPPER.readValue(string, NexmarkPerf.class);
+    } catch (IOException e) {
+      throw new RuntimeException("Unable to parse nexmark perf: ", e);
+    }
+  }
+
+  /**
+   * Return true if there looks to be activity between {@code this} and {@code that}
+   * perf values.
+   */
+  public boolean anyActivity(NexmarkPerf that) {
+    if (runtimeSec != that.runtimeSec) {
+      // An event or result end timestamp looks to have changed.
+      return true;
+    }
+    if (numEvents != that.numEvents) {
+      // Some more events were generated.
+      return true;
+    }
+    if (numResults != that.numResults) {
+      // Some more results were emitted.
+      return true;
+    }
+    return false;
+  }
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/f4333df7/sdks/java/nexmark/src/main/java/org/apache/beam/sdk/nexmark/NexmarkSuite.java
----------------------------------------------------------------------
diff --git a/sdks/java/nexmark/src/main/java/org/apache/beam/sdk/nexmark/NexmarkSuite.java b/sdks/java/nexmark/src/main/java/org/apache/beam/sdk/nexmark/NexmarkSuite.java
new file mode 100644
index 0000000..d38cb7b
--- /dev/null
+++ b/sdks/java/nexmark/src/main/java/org/apache/beam/sdk/nexmark/NexmarkSuite.java
@@ -0,0 +1,112 @@
+/*
+ * 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.beam.sdk.nexmark;
+
+import java.util.ArrayList;
+import java.util.LinkedHashSet;
+import java.util.List;
+import java.util.Set;
+
+/**
+ * A set of {@link NexmarkConfiguration}s.
+ */
+public enum NexmarkSuite {
+  /**
+   * The default.
+   */
+  DEFAULT(defaultConf()),
+
+  /**
+   * Sweep through all queries using the default configuration.
+   * 100k/10k events (depending on query).
+   */
+  SMOKE(smoke()),
+
+  /**
+   * As for SMOKE, but with 10m/1m events.
+   */
+  STRESS(stress()),
+
+  /**
+   * As for SMOKE, but with 1b/100m events.
+   */
+  FULL_THROTTLE(fullThrottle());
+
+  private static List<NexmarkConfiguration> defaultConf() {
+    List<NexmarkConfiguration> configurations = new ArrayList<>();
+    NexmarkConfiguration configuration = new NexmarkConfiguration();
+    configurations.add(configuration);
+    return configurations;
+  }
+
+  private static List<NexmarkConfiguration> smoke() {
+    List<NexmarkConfiguration> configurations = new ArrayList<>();
+    for (int query = 0; query <= 12; query++) {
+      NexmarkConfiguration configuration = NexmarkConfiguration.DEFAULT.copy();
+      configuration.query = query;
+      configuration.numEvents = 100_000;
+      if (query == 4 || query == 6 || query == 9) {
+        // Scale back so overall runtimes are reasonably close across all queries.
+        configuration.numEvents /= 10;
+      }
+      configurations.add(configuration);
+    }
+    return configurations;
+  }
+
+  private static List<NexmarkConfiguration> stress() {
+    List<NexmarkConfiguration> configurations = smoke();
+    for (NexmarkConfiguration configuration : configurations) {
+      if (configuration.numEvents >= 0) {
+        configuration.numEvents *= 1000;
+      }
+    }
+    return configurations;
+  }
+
+  private static List<NexmarkConfiguration> fullThrottle() {
+    List<NexmarkConfiguration> configurations = smoke();
+    for (NexmarkConfiguration configuration : configurations) {
+      if (configuration.numEvents >= 0) {
+        configuration.numEvents *= 1000;
+      }
+    }
+    return configurations;
+  }
+
+  private final List<NexmarkConfiguration> configurations;
+
+  NexmarkSuite(List<NexmarkConfiguration> configurations) {
+    this.configurations = configurations;
+  }
+
+  /**
+   * Return the configurations corresponding to this suite. We'll override each configuration
+   * with any set command line flags, except for --isStreaming which is only respected for
+   * the {@link #DEFAULT} suite.
+   */
+  public Iterable<NexmarkConfiguration> getConfigurations(NexmarkOptions options) {
+    Set<NexmarkConfiguration> results = new LinkedHashSet<>();
+    for (NexmarkConfiguration configuration : configurations) {
+      NexmarkConfiguration result = configuration.copy();
+      result.overrideFromOptions(options);
+      results.add(result);
+    }
+    return results;
+  }
+}


[03/55] [abbrv] beam git commit: NexMark

Posted by ie...@apache.org.
http://git-wip-us.apache.org/repos/asf/beam/blob/1f08970a/integration/java/src/main/java/org/apache/beam/integration/nexmark/NexmarkUtils.java
----------------------------------------------------------------------
diff --git a/integration/java/src/main/java/org/apache/beam/integration/nexmark/NexmarkUtils.java b/integration/java/src/main/java/org/apache/beam/integration/nexmark/NexmarkUtils.java
new file mode 100644
index 0000000..13ed580
--- /dev/null
+++ b/integration/java/src/main/java/org/apache/beam/integration/nexmark/NexmarkUtils.java
@@ -0,0 +1,681 @@
+/*
+ * 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.beam.integration.nexmark;
+
+import org.apache.beam.sdk.Pipeline;
+import org.apache.beam.sdk.coders.AvroCoder;
+import org.apache.beam.sdk.coders.ByteArrayCoder;
+import org.apache.beam.sdk.coders.Coder;
+import org.apache.beam.sdk.coders.CoderException;
+import org.apache.beam.sdk.coders.CoderRegistry;
+import org.apache.beam.sdk.coders.CustomCoder;
+import org.apache.beam.sdk.coders.SerializableCoder;
+import org.apache.beam.sdk.io.Read;
+import org.apache.beam.sdk.runners.DirectPipelineRunner;
+import org.apache.beam.sdk.runners.PipelineRunner;
+import org.apache.beam.sdk.transforms.Aggregator;
+import org.apache.beam.sdk.transforms.Combine;
+import org.apache.beam.sdk.transforms.DoFn;
+import org.apache.beam.sdk.transforms.PTransform;
+import org.apache.beam.sdk.transforms.ParDo;
+import org.apache.beam.sdk.transforms.Sum.SumLongFn;
+import org.apache.beam.sdk.transforms.windowing.AfterPane;
+import org.apache.beam.sdk.transforms.windowing.BoundedWindow;
+import org.apache.beam.sdk.transforms.windowing.GlobalWindows;
+import org.apache.beam.sdk.transforms.windowing.Window;
+import org.apache.beam.sdk.util.state.StateNamespaces;
+import org.apache.beam.sdk.util.state.StateTag;
+import org.apache.beam.sdk.util.state.StateTags;
+import org.apache.beam.sdk.util.state.ValueState;
+import org.apache.beam.sdk.values.PCollection;
+import org.apache.beam.sdk.values.PInput;
+import org.apache.beam.sdk.values.TimestampedValue;
+import com.google.common.collect.ImmutableList;
+import com.google.common.hash.Hashing;
+
+import com.fasterxml.jackson.databind.ObjectMapper;
+
+import org.joda.time.Duration;
+import org.joda.time.Instant;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
+import java.nio.charset.StandardCharsets;
+import java.util.Iterator;
+import java.util.List;
+
+/**
+ * Odd's 'n Ends used throughout queries and driver.
+ */
+public class NexmarkUtils {
+  private static final Logger LOG = LoggerFactory.getLogger(NexmarkGoogleDriver.class.getName());
+
+  /**
+   * Mapper for (de)serializing JSON.
+   */
+  static final ObjectMapper MAPPER = new ObjectMapper();
+
+  /**
+   * Possible sources for events.
+   */
+  public enum SourceType {
+    /**
+     * Produce events directly.
+     */
+    DIRECT,
+    /**
+     * Read events from an Avro file.
+     */
+    AVRO,
+    /**
+     * Read from a PubSub topic. It will be fed the same synthetic events by this pipeline.
+     */
+    PUBSUB
+  }
+
+  /**
+   * Possible sinks for query results.
+   */
+  public enum SinkType {
+    /**
+     * Discard all results.
+     */
+    COUNT_ONLY,
+    /**
+     * Discard all results after converting them to strings.
+     */
+    DEVNULL,
+    /**
+     * Write to a PubSub topic. It will be drained by this pipeline.
+     */
+    PUBSUB,
+    /**
+     * Write to a text file. Only works in batch mode.
+     */
+    TEXT,
+    /**
+     * Write raw Events to Avro. Only works in batch mode.
+     */
+    AVRO,
+    /**
+     * Write raw Events to BigQuery.
+     */
+    BIGQUERY,
+  }
+
+  /**
+   * Pub/sub mode to run in.
+   */
+  public enum PubSubMode {
+    /**
+     * Publish events to pub/sub, but don't run the query.
+     */
+    PUBLISH_ONLY,
+    /**
+     * Consume events from pub/sub and run the query, but don't publish.
+     */
+    SUBSCRIBE_ONLY,
+    /**
+     * Both publish and consume, but as separate jobs.
+     */
+    COMBINED
+  }
+
+  /**
+   * Coder strategies.
+   */
+  public enum CoderStrategy {
+    /**
+     * Hand-written.
+     */
+    HAND,
+    /**
+     * Avro.
+     */
+    AVRO,
+    /**
+     * Java serialization.
+     */
+    JAVA
+  }
+
+  /**
+   * How to determine resource names.
+   */
+  public enum ResourceNameMode {
+    /** Names are used as provided. */
+    VERBATIM,
+    /** Names are suffixed with the query being run. */
+    QUERY,
+    /** Names are suffixed with the query being run and a random number. */
+    QUERY_AND_SALT;
+  }
+
+  /**
+   * Units for rates.
+   */
+  public enum RateUnit {
+    PER_SECOND(1_000_000L),
+    PER_MINUTE(60_000_000L);
+
+    RateUnit(long usPerUnit) {
+      this.usPerUnit = usPerUnit;
+    }
+
+    /**
+     * Number of microseconds per unit.
+     */
+    private final long usPerUnit;
+
+    /**
+     * Number of microseconds between events at given rate.
+     */
+    public long rateToPeriodUs(long rate) {
+      return (usPerUnit + rate / 2) / rate;
+    }
+  }
+
+  /**
+   * Shape of event rate.
+   */
+  public static enum RateShape {
+    SQUARE,
+    SINE;
+
+    /**
+     * Number of steps used to approximate sine wave.
+     */
+    private static final int N = 10;
+
+    /**
+     * Return inter-event delay, in microseconds, for each generator
+     * to follow in order to achieve {@code rate} at {@code unit} using {@code numGenerators}.
+     */
+    public long interEventDelayUs(int rate, RateUnit unit, int numGenerators) {
+      return unit.rateToPeriodUs(rate) * numGenerators;
+    }
+
+    /**
+     * Return array of successive inter-event delays, in microseconds, for each generator
+     * to follow in order to achieve this shape with {@code firstRate/nextRate} at
+     * {@code unit} using {@code numGenerators}.
+     */
+    public long[] interEventDelayUs(
+        int firstRate, int nextRate, RateUnit unit, int numGenerators) {
+      if (firstRate == nextRate) {
+        long[] interEventDelayUs = new long[1];
+        interEventDelayUs[0] = unit.rateToPeriodUs(firstRate) * numGenerators;
+        return interEventDelayUs;
+      }
+
+      switch (this) {
+        case SQUARE: {
+          long[] interEventDelayUs = new long[2];
+          interEventDelayUs[0] = unit.rateToPeriodUs(firstRate) * numGenerators;
+          interEventDelayUs[1] = unit.rateToPeriodUs(nextRate) * numGenerators;
+          return interEventDelayUs;
+        }
+        case SINE: {
+          double mid = (firstRate + nextRate) / 2.0;
+          double amp = (firstRate - nextRate) / 2.0; // may be -ve
+          long[] interEventDelayUs = new long[N];
+          for (int i = 0; i < N; i++) {
+            double r = (2.0 * Math.PI * i) / N;
+            double rate = mid + amp * Math.cos(r);
+            interEventDelayUs[i] = unit.rateToPeriodUs(Math.round(rate)) * numGenerators;
+          }
+          return interEventDelayUs;
+        }
+      }
+      throw new RuntimeException(); // switch should be exhaustive
+    }
+
+    /**
+     * Return delay between steps, in seconds, for result of {@link #interEventDelayUs}, so
+     * as to cycle through the entire sequence every {@code ratePeriodSec}.
+     */
+    public int stepLengthSec(int ratePeriodSec) {
+      int n = 0;
+      switch (this) {
+        case SQUARE:
+          n = 2;
+          break;
+        case SINE:
+          n = N;
+          break;
+      }
+      return (ratePeriodSec + n - 1) / n;
+    }
+  }
+
+  /**
+   * Set to true to capture all info messages. The logging level flags don't currently work.
+   */
+  private static final boolean LOG_INFO = false;
+
+  /**
+   * Set to true to capture all error messages. The logging level flags don't currently work.
+   */
+  private static final boolean LOG_ERROR = true;
+
+  /**
+   * Set to true to log directly to stdout on VM. You can watch the results in real-time with:
+   * tail -f /var/log/dataflow/streaming-harness/harness-stdout.log
+   */
+  private static final boolean LOG_TO_CONSOLE = false;
+
+  /**
+   * Log info message.
+   */
+  public static void info(String format, Object... args) {
+    if (LOG_INFO) {
+      LOG.info(String.format(format, args));
+      if (LOG_TO_CONSOLE) {
+        System.out.println(String.format(format, args));
+      }
+    }
+  }
+
+  /**
+   * Log error message.
+   */
+  public static void error(String format, Object... args) {
+    if (LOG_ERROR) {
+      LOG.error(String.format(format, args));
+      if (LOG_TO_CONSOLE) {
+        System.out.println(String.format(format, args));
+      }
+    }
+  }
+
+  /**
+   * Log message to console. For client side only.
+   */
+  public static void console(String format, Object... args) {
+    System.out.printf("%s %s\n", Instant.now(), String.format(format, args));
+  }
+
+  /**
+   * Label to use for timestamps on pub/sub messages.
+   */
+  public static final String PUBSUB_TIMESTAMP = "timestamp";
+
+  /**
+   * Label to use for windmill ids on pub/sub messages.
+   */
+  public static final String PUBSUB_ID = "id";
+
+  /**
+   * All events will be given a timestamp relative to this time (ms since epoch).
+   */
+  public static final long BASE_TIME = Instant.parse("2015-07-15T00:00:00.000Z").getMillis();
+
+  /**
+   * Instants guaranteed to be strictly before and after all event timestamps, and which won't
+   * be subject to underflow/overflow.
+   */
+  public static final Instant BEGINNING_OF_TIME = new Instant(0).plus(Duration.standardDays(365));
+  public static final Instant END_OF_TIME =
+      BoundedWindow.TIMESTAMP_MAX_VALUE.minus(Duration.standardDays(365));
+
+  /**
+   * Setup pipeline with codes and some other options.
+   */
+  public static void setupPipeline(CoderStrategy coderStrategy, Pipeline p) {
+    PipelineRunner<?> runner = p.getRunner();
+    if (runner instanceof DirectPipelineRunner) {
+      // Disable randomization of output since we want to check batch and streaming match the
+      // model both locally and on the cloud.
+      ((DirectPipelineRunner) runner).withUnorderednessTesting(false);
+    }
+
+    CoderRegistry registry = p.getCoderRegistry();
+    switch (coderStrategy) {
+      case HAND:
+        registry.registerCoder(Auction.class, Auction.CODER);
+        registry.registerCoder(AuctionBid.class, AuctionBid.CODER);
+        registry.registerCoder(AuctionCount.class, AuctionCount.CODER);
+        registry.registerCoder(AuctionPrice.class, AuctionPrice.CODER);
+        registry.registerCoder(Bid.class, Bid.CODER);
+        registry.registerCoder(CategoryPrice.class, CategoryPrice.CODER);
+        registry.registerCoder(Event.class, Event.CODER);
+        registry.registerCoder(IdNameReserve.class, IdNameReserve.CODER);
+        registry.registerCoder(NameCityStateId.class, NameCityStateId.CODER);
+        registry.registerCoder(Person.class, Person.CODER);
+        registry.registerCoder(SellerPrice.class, SellerPrice.CODER);
+        registry.registerCoder(Done.class, Done.CODER);
+        registry.registerCoder(BidsPerSession.class, BidsPerSession.CODER);
+        break;
+      case AVRO:
+        registry.setFallbackCoderProvider(AvroCoder.PROVIDER);
+        break;
+      case JAVA:
+        registry.setFallbackCoderProvider(SerializableCoder.PROVIDER);
+        break;
+    }
+  }
+
+  /**
+   * Return a generator config to match the given {@code options}.
+   */
+  public static GeneratorConfig standardGeneratorConfig(NexmarkConfiguration configuration) {
+    return new GeneratorConfig(configuration,
+                               configuration.useWallclockEventTime ? System.currentTimeMillis()
+                                                                   : BASE_TIME, 0,
+                               configuration.numEvents, 0);
+  }
+
+  /**
+   * Return an iterator of events using the 'standard' generator config.
+   */
+  public static Iterator<TimestampedValue<Event>> standardEventIterator(
+      NexmarkConfiguration configuration) {
+    return new Generator(standardGeneratorConfig(configuration));
+  }
+
+  /**
+   * Return a transform which yields a finite number of synthesized events generated
+   * as a batch.
+   */
+  public static PTransform<PInput, PCollection<Event>> batchEventsSource(
+      String name, NexmarkConfiguration configuration) {
+    return Read
+        .from(new BoundedEventSource(
+            NexmarkUtils.standardGeneratorConfig(configuration), configuration.numEventGenerators))
+        .named(name + ".ReadBounded");
+  }
+
+  /**
+   * Return a transform which yields a finite number of synthesized events generated
+   * on-the-fly in real time.
+   */
+  public static PTransform<PInput, PCollection<Event>> streamEventsSource(
+      String name, NexmarkConfiguration configuration) {
+    return Read.from(new UnboundedEventSource(NexmarkUtils.standardGeneratorConfig(configuration),
+                                              configuration.numEventGenerators,
+                                              configuration.watermarkHoldbackSec,
+                                              configuration.isRateLimited))
+               .named(name + ".ReadUnbounded");
+  }
+
+  /**
+   * Return a transform to pass-through events, but count them as they go by.
+   */
+  public static ParDo.Bound<Event, Event> snoop(final String name) {
+    return ParDo.named(name + ".Snoop")
+                .of(new DoFn<Event, Event>() {
+                  final Aggregator<Long, Long> eventCounter =
+                      createAggregator("events", new SumLongFn());
+                  final Aggregator<Long, Long> newPersonCounter =
+                      createAggregator("newPersons", new SumLongFn());
+                  final Aggregator<Long, Long> newAuctionCounter =
+                      createAggregator("newAuctions", new SumLongFn());
+                  final Aggregator<Long, Long> bidCounter =
+                      createAggregator("bids", new SumLongFn());
+                  final Aggregator<Long, Long> endOfStreamCounter =
+                      createAggregator("endOfStream", new SumLongFn());
+
+                  @Override
+                  public void processElement(ProcessContext c) {
+                    eventCounter.addValue(1L);
+                    if (c.element().newPerson != null) {
+                      newPersonCounter.addValue(1L);
+                    } else if (c.element().newAuction != null) {
+                      newAuctionCounter.addValue(1L);
+                    } else if (c.element().bid != null) {
+                      bidCounter.addValue(1L);
+                    } else {
+                      endOfStreamCounter.addValue(1L);
+                    }
+                    info("%s snooping element %s", name, c.element());
+                    c.output(c.element());
+                  }
+                });
+  }
+
+  /**
+   * Return a transform to count and discard each element.
+   */
+  public static <T> ParDo.Bound<T, Void> devNull(String name) {
+    return ParDo.named(name + ".DevNull")
+                .of(new DoFn<T, Void>() {
+                  final Aggregator<Long, Long> discardCounter =
+                      createAggregator("discarded", new SumLongFn());
+
+                  @Override
+                  public void processElement(ProcessContext c) {
+                    discardCounter.addValue(1L);
+                  }
+                });
+  }
+
+  /**
+   * Return a transform to log each element, passing it through unchanged.
+   */
+  public static <T> ParDo.Bound<T, T> log(final String name) {
+    return ParDo.named(name + ".Log")
+                .of(new DoFn<T, T>() {
+                  @Override
+                  public void processElement(ProcessContext c) {
+                    error("%s: %s", name, c.element());
+                    c.output(c.element());
+                  }
+                });
+  }
+
+  /**
+   * Return a transform to format each element as a string.
+   */
+  public static <T> ParDo.Bound<T, String> format(String name) {
+    return ParDo.named(name + ".Format")
+                .of(new DoFn<T, String>() {
+                  final Aggregator<Long, Long> recordCounter =
+                      createAggregator("records", new SumLongFn());
+
+                  @Override
+                  public void processElement(ProcessContext c) {
+                    recordCounter.addValue(1L);
+                    c.output(c.element().toString());
+                  }
+                });
+  }
+
+  /**
+   * Return a transform to make explicit the timestamp of each element.
+   */
+  public static <T> ParDo.Bound<T, TimestampedValue<T>> stamp(String name) {
+    return ParDo.named(name + ".Stamp")
+                .of(new DoFn<T, TimestampedValue<T>>() {
+                  @Override
+                  public void processElement(ProcessContext c) {
+                    c.output(TimestampedValue.of(c.element(), c.timestamp()));
+                  }
+                });
+  }
+
+  /**
+   * Return a transform to reduce a stream to a single, order-invariant long hash.
+   */
+  public static <T> PTransform<PCollection<T>, PCollection<Long>> hash(
+      final long numEvents, String name) {
+    return new PTransform<PCollection<T>, PCollection<Long>>(name) {
+      @Override
+      public PCollection<Long> apply(PCollection<T> input) {
+        return input.apply(Window.<T>into(new GlobalWindows())
+                               .triggering(AfterPane.elementCountAtLeast((int) numEvents))
+                               .withAllowedLateness(Duration.standardDays(1))
+                               .discardingFiredPanes())
+
+                    .apply(ParDo.named(name + ".Hash").of(new DoFn<T, Long>() {
+                      @Override
+                      public void processElement(ProcessContext c) {
+                        long hash =
+                            Hashing.murmur3_128()
+                                   .newHasher()
+                                   .putLong(c.timestamp().getMillis())
+                                   .putString(c.element().toString(), StandardCharsets.UTF_8)
+                                   .hash()
+                                   .asLong();
+                        c.output(hash);
+                      }
+                    }))
+
+                    .apply(Combine.globally(new Combine.BinaryCombineFn<Long>() {
+                      @Override
+                      public Long apply(Long left, Long right) {
+                        return left ^ right;
+                      }
+                    }));
+      }
+    };
+  }
+
+  private static final long MASK = (1L << 16) - 1L;
+  private static final long HASH = 0x243F6A8885A308D3L;
+  private static final long INIT_PLAINTEXT = 50000L;
+
+  /**
+   * Return a transform to keep the CPU busy for given milliseconds on every record.
+   */
+  public static <T> ParDo.Bound<T, T> cpuDelay(String name, final long delayMs) {
+    return ParDo.named(name + ".CpuDelay")
+                .of(new DoFn<T, T>() {
+                  @Override
+                  public void processElement(ProcessContext c) {
+                    long now = System.currentTimeMillis();
+                    long end = now + delayMs;
+                    while (now < end) {
+                      // Find plaintext which hashes to HASH in lowest MASK bits.
+                      // Values chosen to roughly take 1ms on typical workstation.
+                      long p = INIT_PLAINTEXT;
+                      while (true) {
+                        long t = Hashing.murmur3_128().hashLong(p).asLong();
+                        if ((t & MASK) == (HASH & MASK)) {
+                          break;
+                        }
+                        p++;
+                      }
+                      long next = System.currentTimeMillis();
+                      now = next;
+                    }
+                    c.output(c.element());
+                  }
+                });
+  }
+
+  private static final StateTag<Object, ValueState<byte[]>> DUMMY_TAG =
+      StateTags.value("dummy", ByteArrayCoder.of());
+  private static final int MAX_BUFFER_SIZE = 1 << 24;
+
+  /**
+   * Return a transform to write given number of bytes to durable store on every record.
+   */
+  public static <T> ParDo.Bound<T, T> diskBusy(String name, final long bytes) {
+    return ParDo.named(name + ".DiskBusy")
+                .of(new DoFn<T, T>() {
+                  @Override
+                  public void processElement(ProcessContext c) {
+                    long remain = bytes;
+                    long start = System.currentTimeMillis();
+                    long now = start;
+                    while (remain > 0) {
+                      long thisBytes = Math.min(remain, MAX_BUFFER_SIZE);
+                      remain -= thisBytes;
+                      byte[] arr = new byte[(int) thisBytes];
+                      for (int i = 0; i < thisBytes; i++) {
+                        arr[i] = (byte) now;
+                      }
+                      ValueState<byte[]> state = c.windowingInternals().stateInternals().state(
+                          StateNamespaces.global(), DUMMY_TAG);
+                      state.write(arr);
+                      now = System.currentTimeMillis();
+                    }
+                    c.output(c.element());
+                  }
+                });
+  }
+
+  /**
+   * Return a transform to cast each element to {@link KnownSize}.
+   */
+  private static <T extends KnownSize> ParDo.Bound<T, KnownSize> castToKnownSize(
+      final String name) {
+    return ParDo.named(name + ".Forget")
+                .of(new DoFn<T, KnownSize>() {
+                  @Override
+                  public void processElement(ProcessContext c) {
+                    c.output(c.element());
+                  }
+                });
+  }
+
+  /**
+   * A coder for instances of {@code T} cast up to {@link KnownSize}.
+   *
+   * @param <T> True type of object.
+   */
+  private static class CastingCoder<T extends KnownSize> extends CustomCoder<KnownSize> {
+    private final Coder<T> trueCoder;
+
+    public CastingCoder(Coder<T> trueCoder) {
+      this.trueCoder = trueCoder;
+    }
+
+    @Override
+    public void encode(KnownSize value, OutputStream outStream, Context context)
+        throws CoderException, IOException {
+      @SuppressWarnings("unchecked")
+      T typedValue = (T) value;
+      trueCoder.encode(typedValue, outStream, context);
+    }
+
+    @Override
+    public KnownSize decode(InputStream inStream, Context context)
+        throws CoderException, IOException {
+      return trueCoder.decode(inStream, context);
+    }
+
+    @Override
+    public List<? extends Coder<?>> getComponents() {
+      return ImmutableList.of(trueCoder);
+    }
+  }
+
+  /**
+   * Return a coder for {@code KnownSize} that are known to be exactly of type {@code T}.
+   */
+  private static <T extends KnownSize> Coder<KnownSize> makeCastingCoder(Coder<T> trueCoder) {
+    return new CastingCoder<>(trueCoder);
+  }
+
+  /**
+   * Return {@code elements} as {@code KnownSize}s.
+   */
+  public static <T extends KnownSize> PCollection<KnownSize> castToKnownSize(
+      final String name, PCollection<T> elements) {
+    return elements.apply(castToKnownSize(name)).setCoder(makeCastingCoder(elements.getCoder()));
+  }
+
+  // Do not instantiate.
+  private NexmarkUtils() {
+  }
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/1f08970a/integration/java/src/main/java/org/apache/beam/integration/nexmark/Options.java
----------------------------------------------------------------------
diff --git a/integration/java/src/main/java/org/apache/beam/integration/nexmark/Options.java b/integration/java/src/main/java/org/apache/beam/integration/nexmark/Options.java
new file mode 100644
index 0000000..4f5304d
--- /dev/null
+++ b/integration/java/src/main/java/org/apache/beam/integration/nexmark/Options.java
@@ -0,0 +1,360 @@
+/*
+ * 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.beam.integration.nexmark;
+
+import org.apache.beam.sdk.options.Default;
+import org.apache.beam.sdk.options.Description;
+import org.apache.beam.sdk.options.PubsubOptions;
+import javax.annotation.Nullable;
+
+/**
+ * Command line flags.
+ */
+public interface Options extends PubsubOptions {
+  @Description("Which suite to run. Default is to use command line arguments for one job.")
+  @Default.Enum("DEFAULT")
+  NexmarkSuite getSuite();
+
+  void setSuite(NexmarkSuite suite);
+
+  @Description("If true, and using the DataflowPipelineRunner, monitor the jobs as they run.")
+  @Default.Boolean(false)
+  boolean getMonitorJobs();
+
+  void setMonitorJobs(boolean monitorJobs);
+
+  @Description("Where the events come from.")
+  @Nullable
+  NexmarkUtils.SourceType getSourceType();
+
+  void setSourceType(NexmarkUtils.SourceType sourceType);
+
+  @Description("Prefix for input files if using avro input")
+  @Nullable
+  String getInputPath();
+
+  void setInputPath(String inputPath);
+
+  @Description("Where results go.")
+  @Nullable
+  NexmarkUtils.SinkType getSinkType();
+
+  void setSinkType(NexmarkUtils.SinkType sinkType);
+
+  @Description("Which mode to run in when source is PUBSUB.")
+  @Nullable
+  NexmarkUtils.PubSubMode getPubSubMode();
+
+  void setPubSubMode(NexmarkUtils.PubSubMode pubSubMode);
+
+  @Description("Which query to run.")
+  @Nullable
+  Integer getQuery();
+
+  void setQuery(Integer query);
+
+  @Description("Prefix for output files if using text output for results or running Query 10.")
+  @Nullable
+  String getOutputPath();
+
+  void setOutputPath(String outputPath);
+
+  @Description("Base name of pubsub topic to publish to in streaming mode.")
+  @Nullable
+  @Default.String("nexmark")
+  String getPubsubTopic();
+
+  void setPubsubTopic(String pubsubTopic);
+
+  @Description("Base name of pubsub subscription to read from in streaming mode.")
+  @Nullable
+  @Default.String("nexmark")
+  String getPubsubSubscription();
+
+  void setPubsubSubscription(String pubsubSubscription);
+
+  @Description("Base name of BigQuery table name if using BigQuery output.")
+  @Nullable
+  @Default.String("nexmark")
+  String getBigQueryTable();
+
+  void setBigQueryTable(String bigQueryTable);
+
+  @Description("Approximate number of events to generate. "
+               + "Zero for effectively unlimited in streaming mode.")
+  @Nullable
+  Long getNumEvents();
+
+  void setNumEvents(Long numEvents);
+
+  @Description("Time in seconds to preload the subscription with data, at the initial input rate "
+               + "of the pipeline.")
+  @Nullable
+  Integer getPreloadSeconds();
+
+  void setPreloadSeconds(Integer preloadSeconds);
+
+  @Description("Number of unbounded sources to create events.")
+  @Nullable
+  Integer getNumEventGenerators();
+
+  void setNumEventGenerators(Integer numEventGenerators);
+
+  @Description("Shape of event rate curve.")
+  @Nullable
+  NexmarkUtils.RateShape getRateShape();
+
+  void setRateShape(NexmarkUtils.RateShape rateShape);
+
+  @Description("Initial overall event rate (in --rateUnit).")
+  @Nullable
+  Integer getFirstEventRate();
+
+  void setFirstEventRate(Integer firstEventRate);
+
+  @Description("Next overall event rate (in --rateUnit).")
+  @Nullable
+  Integer getNextEventRate();
+
+  void setNextEventRate(Integer nextEventRate);
+
+  @Description("Unit for rates.")
+  @Nullable
+  NexmarkUtils.RateUnit getRateUnit();
+
+  void setRateUnit(NexmarkUtils.RateUnit rateUnit);
+
+  @Description("Overall period of rate shape, in seconds.")
+  @Nullable
+  Integer getRatePeriodSec();
+
+  void setRatePeriodSec(Integer ratePeriodSec);
+
+  @Description("If true, relay events in real time in streaming mode.")
+  @Nullable
+  Boolean getIsRateLimited();
+
+  void setIsRateLimited(Boolean isRateLimited);
+
+  @Description("If true, use wallclock time as event time. Otherwise, use a deterministic"
+               + " time in the past so that multiple runs will see exactly the same event streams"
+               + " and should thus have exactly the same results.")
+  @Nullable
+  Boolean getUseWallclockEventTime();
+
+  void setUseWallclockEventTime(Boolean useWallclockEventTime);
+
+  @Description("Assert pipeline results match model results.")
+  @Nullable
+  boolean getAssertCorrectness();
+
+  void setAssertCorrectness(boolean assertCorrectness);
+
+  @Description("Log all input events.")
+  @Nullable
+  boolean getLogEvents();
+
+  void setLogEvents(boolean logEvents);
+
+  @Description("Log all query results.")
+  @Nullable
+  boolean getLogResults();
+
+  void setLogResults(boolean logResults);
+
+  @Description("Average size in bytes for a person record.")
+  @Nullable
+  Integer getAvgPersonByteSize();
+
+  void setAvgPersonByteSize(Integer avgPersonByteSize);
+
+  @Description("Average size in bytes for an auction record.")
+  @Nullable
+  Integer getAvgAuctionByteSize();
+
+  void setAvgAuctionByteSize(Integer avgAuctionByteSize);
+
+  @Description("Average size in bytes for a bid record.")
+  @Nullable
+  Integer getAvgBidByteSize();
+
+  void setAvgBidByteSize(Integer avgBidByteSize);
+
+  @Description("Ratio of bids for 'hot' auctions above the background.")
+  @Nullable
+  Integer getHotAuctionRatio();
+
+  void setHotAuctionRatio(Integer hotAuctionRatio);
+
+  @Description("Ratio of auctions for 'hot' sellers above the background.")
+  @Nullable
+  Integer getHotSellersRatio();
+
+  void setHotSellersRatio(Integer hotSellersRatio);
+
+  @Description("Ratio of auctions for 'hot' bidders above the background.")
+  @Nullable
+  Integer getHotBiddersRatio();
+
+  void setHotBiddersRatio(Integer hotBiddersRatio);
+
+  @Description("Window size in seconds.")
+  @Nullable
+  Long getWindowSizeSec();
+
+  void setWindowSizeSec(Long windowSizeSec);
+
+  @Description("Window period in seconds.")
+  @Nullable
+  Long getWindowPeriodSec();
+
+  void setWindowPeriodSec(Long windowPeriodSec);
+
+  @Description("If in streaming mode, the holdback for watermark in seconds.")
+  @Nullable
+  Long getWatermarkHoldbackSec();
+
+  void setWatermarkHoldbackSec(Long watermarkHoldbackSec);
+
+  @Description("Roughly how many auctions should be in flight for each generator.")
+  @Nullable
+  Integer getNumInFlightAuctions();
+
+  void setNumInFlightAuctions(Integer numInFlightAuctions);
+
+
+  @Description("Maximum number of people to consider as active for placing auctions or bids.")
+  @Nullable
+  Integer getNumActivePeople();
+
+  void setNumActivePeople(Integer numActivePeople);
+
+  @Description("Filename of perf data to append to.")
+  @Nullable
+  String getPerfFilename();
+
+  void setPerfFilename(String perfFilename);
+
+  @Description("Filename of baseline perf data to read from.")
+  @Nullable
+  String getBaselineFilename();
+
+  void setBaselineFilename(String baselineFilename);
+
+  @Description("Filename of summary perf data to append to.")
+  @Nullable
+  String getSummaryFilename();
+
+  void setSummaryFilename(String summaryFilename);
+
+  @Description("Filename for javascript capturing all perf data and any baselines.")
+  @Nullable
+  String getJavascriptFilename();
+
+  void setJavascriptFilename(String javascriptFilename);
+
+  @Description("If true, don't run the actual query. Instead, calculate the distribution "
+               + "of number of query results per (event time) minute according to the query model.")
+  @Nullable
+  boolean getJustModelResultRate();
+
+  void setJustModelResultRate(boolean justModelResultRate);
+
+  @Description("Coder strategy to use.")
+  @Nullable
+  NexmarkUtils.CoderStrategy getCoderStrategy();
+
+  void setCoderStrategy(NexmarkUtils.CoderStrategy coderStrategy);
+
+  @Description("Delay, in milliseconds, for each event. We will peg one core for this "
+               + "number of milliseconds to simulate CPU-bound computation.")
+  @Nullable
+  Long getCpuDelayMs();
+
+  void setCpuDelayMs(Long cpuDelayMs);
+
+  @Description("Extra data, in bytes, to save to persistent state for each event. "
+               + "This will force I/O all the way to durable storage to simulate an "
+               + "I/O-bound computation.")
+  @Nullable
+  Long getDiskBusyBytes();
+
+  void setDiskBusyBytes(Long diskBusyBytes);
+
+  @Description("Skip factor for query 2. We select bids for every {@code auctionSkip}'th auction")
+  @Nullable
+  Integer getAuctionSkip();
+
+  void setAuctionSkip(Integer auctionSkip);
+
+  @Description("Fanout for queries 4 (groups by category id) and 7 (finds a global maximum).")
+  @Nullable
+  Integer getFanout();
+
+  void setFanout(Integer fanout);
+
+  @Description("Length of occasional delay to impose on events (in seconds).")
+  @Nullable
+  Long getOccasionalDelaySec();
+
+  void setOccasionalDelaySec(Long occasionalDelaySec);
+
+  @Description("Probability that an event will be delayed by delayS.")
+  @Nullable
+  Double getProbDelayedEvent();
+
+  void setProbDelayedEvent(Double probDelayedEvent);
+
+  @Description("Maximum size of each log file (in events). For Query10 only.")
+  @Nullable
+  Integer getMaxLogEvents();
+
+  void setMaxLogEvents(Integer maxLogEvents);
+
+  @Description("How to derive names of resources.")
+  @Default.Enum("QUERY_AND_SALT")
+  NexmarkUtils.ResourceNameMode getResourceNameMode();
+
+  void setResourceNameMode(NexmarkUtils.ResourceNameMode mode);
+
+  @Description("If true, manage the creation and cleanup of topics, subscriptions and gcs files.")
+  @Default.Boolean(true)
+  boolean getManageResources();
+
+  void setManageResources(boolean manageResources);
+
+  @Description("If true, use pub/sub publish time instead of event time.")
+  @Nullable
+  Boolean getUsePubsubPublishTime();
+
+  void setUsePubsubPublishTime(Boolean usePubsubPublishTime);
+
+  @Description("Number of events in out-of-order groups. 1 implies no out-of-order events. "
+               + "1000 implies every 1000 events per generator are emitted in pseudo-random order.")
+  @Nullable
+  Long getOutOfOrderGroupSize();
+
+  void setOutOfOrderGroupSize(Long outOfOrderGroupSize);
+
+  @Description("If false, do not add the Monitor and Snoop transforms.")
+  @Nullable
+  Boolean getDebug();
+
+  void setDebug(Boolean value);
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/1f08970a/integration/java/src/main/java/org/apache/beam/integration/nexmark/Person.java
----------------------------------------------------------------------
diff --git a/integration/java/src/main/java/org/apache/beam/integration/nexmark/Person.java b/integration/java/src/main/java/org/apache/beam/integration/nexmark/Person.java
new file mode 100644
index 0000000..6fcf388
--- /dev/null
+++ b/integration/java/src/main/java/org/apache/beam/integration/nexmark/Person.java
@@ -0,0 +1,166 @@
+/*
+ * 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.beam.integration.nexmark;
+
+import org.apache.beam.sdk.coders.AtomicCoder;
+import org.apache.beam.sdk.coders.Coder;
+import org.apache.beam.sdk.coders.CoderException;
+import org.apache.beam.sdk.coders.StringUtf8Coder;
+import org.apache.beam.sdk.coders.VarLongCoder;
+
+import com.fasterxml.jackson.annotation.JsonProperty;
+import com.fasterxml.jackson.core.JsonProcessingException;
+
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
+import java.io.Serializable;
+
+/**
+ * A person either creating an auction or making a bid.
+ */
+public class Person implements KnownSize, Serializable {
+  private static final Coder<Long> LONG_CODER = VarLongCoder.of();
+  private static final Coder<String> STRING_CODER = StringUtf8Coder.of();
+  public static final Coder<Person> CODER = new AtomicCoder<Person>() {
+    @Override
+    public void encode(Person value, OutputStream outStream,
+        Coder.Context context)
+        throws CoderException, IOException {
+      LONG_CODER.encode(value.id, outStream, Context.NESTED);
+      STRING_CODER.encode(value.name, outStream, Context.NESTED);
+      STRING_CODER.encode(value.emailAddress, outStream, Context.NESTED);
+      STRING_CODER.encode(value.creditCard, outStream, Context.NESTED);
+      STRING_CODER.encode(value.city, outStream, Context.NESTED);
+      STRING_CODER.encode(value.state, outStream, Context.NESTED);
+      LONG_CODER.encode(value.dateTime, outStream, Context.NESTED);
+      STRING_CODER.encode(value.extra, outStream, Context.NESTED);
+    }
+
+    @Override
+    public Person decode(
+        InputStream inStream, Coder.Context context)
+        throws CoderException, IOException {
+      long id = LONG_CODER.decode(inStream, Context.NESTED);
+      String name = STRING_CODER.decode(inStream, Context.NESTED);
+      String emailAddress = STRING_CODER.decode(inStream, Context.NESTED);
+      String creditCard = STRING_CODER.decode(inStream, Context.NESTED);
+      String city = STRING_CODER.decode(inStream, Context.NESTED);
+      String state = STRING_CODER.decode(inStream, Context.NESTED);
+      long dateTime = LONG_CODER.decode(inStream, Context.NESTED);
+      String extra = STRING_CODER.decode(inStream, Context.NESTED);
+      return new Person(id, name, emailAddress, creditCard, city, state, dateTime, extra);
+    }
+  };
+
+  /** Id of person. */
+  @JsonProperty
+  public final long id; // primary key
+
+  /** Extra person properties. */
+  @JsonProperty
+  public final String name;
+
+  @JsonProperty
+  public final String emailAddress;
+
+  @JsonProperty
+  public final String creditCard;
+
+  @JsonProperty
+  public final String city;
+
+  @JsonProperty
+  public final String state;
+
+  @JsonProperty
+  public final long dateTime;
+
+  /** Additional arbitrary payload for performance testing. */
+  @JsonProperty
+  public final String extra;
+
+  // For Avro only.
+  @SuppressWarnings("unused")
+  private Person() {
+    id = 0;
+    name = null;
+    emailAddress = null;
+    creditCard = null;
+    city = null;
+    state = null;
+    dateTime = 0;
+    extra = null;
+  }
+
+  public Person(long id, String name, String emailAddress, String creditCard, String city,
+      String state, long dateTime, String extra) {
+    this.id = id;
+    this.name = name;
+    this.emailAddress = emailAddress;
+    this.creditCard = creditCard;
+    this.city = city;
+    this.state = state;
+    this.dateTime = dateTime;
+    this.extra = extra;
+  }
+
+  /**
+   * Return a copy of person which capture the given annotation.
+   * (Used for debugging).
+   */
+  public Person withAnnotation(String annotation) {
+    return new Person(id, name, emailAddress, creditCard, city, state, dateTime,
+        annotation + ": " + extra);
+  }
+
+  /**
+   * Does person have {@code annotation}? (Used for debugging.)
+   */
+  public boolean hasAnnotation(String annotation) {
+    return extra.startsWith(annotation + ": ");
+  }
+
+  /**
+   * Remove {@code annotation} from person. (Used for debugging.)
+   */
+  public Person withoutAnnotation(String annotation) {
+    if (hasAnnotation(annotation)) {
+      return new Person(id, name, emailAddress, creditCard, city, state, dateTime,
+          extra.substring(annotation.length() + 2));
+    } else {
+      return this;
+    }
+  }
+
+  @Override
+  public long sizeInBytes() {
+    return 8 + name.length() + 1 + emailAddress.length() + 1 + creditCard.length() + 1
+        + city.length() + 1 + state.length() + 8 + 1 + extra.length() + 1;
+  }
+
+  @Override
+  public String toString() {
+    try {
+      return NexmarkUtils.MAPPER.writeValueAsString(this);
+    } catch (JsonProcessingException e) {
+      throw new RuntimeException(e);
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/1f08970a/integration/java/src/main/java/org/apache/beam/integration/nexmark/PubsubHelper.java
----------------------------------------------------------------------
diff --git a/integration/java/src/main/java/org/apache/beam/integration/nexmark/PubsubHelper.java b/integration/java/src/main/java/org/apache/beam/integration/nexmark/PubsubHelper.java
new file mode 100644
index 0000000..1255154
--- /dev/null
+++ b/integration/java/src/main/java/org/apache/beam/integration/nexmark/PubsubHelper.java
@@ -0,0 +1,217 @@
+/*
+ * 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.beam.integration.nexmark;
+
+import org.apache.beam.sdk.options.PubsubOptions;
+import org.apache.beam.sdk.util.PubsubClient;
+import org.apache.beam.sdk.util.PubsubJsonClient;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.List;
+
+/**
+ * Helper for working with pubsub.
+ */
+public class PubsubHelper implements AutoCloseable {
+  /**
+   * Underlying pub/sub client.
+   */
+  private final PubsubClient pubsubClient;
+
+  /**
+   * Project id.
+   */
+  private final String projectId;
+
+  /**
+   * Topics we should delete on close.
+   */
+  private final List<PubsubClient.TopicPath> createdTopics;
+
+  /**
+   * Subscriptions we should delete on close.
+   */
+  private final List<PubsubClient.SubscriptionPath> createdSubscriptions;
+
+  private PubsubHelper(PubsubClient pubsubClient, String projectId) {
+    this.pubsubClient = pubsubClient;
+    this.projectId = projectId;
+    createdTopics = new ArrayList<>();
+    createdSubscriptions = new ArrayList<>();
+  }
+
+  /**
+   * Create a helper.
+   */
+  public static PubsubHelper create(PubsubOptions options) {
+    try {
+      return new PubsubHelper(
+          PubsubJsonClient.FACTORY.newClient(null, null, options),
+          options.getProject());
+    } catch (IOException e) {
+      throw new RuntimeException("Unable to create Pubsub client: ", e);
+    }
+  }
+
+  /**
+   * Create a topic from short name. Delete it if it already exists. Ensure the topic will be
+   * deleted on cleanup. Return full topic name.
+   */
+  public PubsubClient.TopicPath createTopic(String shortTopic) {
+    PubsubClient.TopicPath topic = PubsubClient.topicPathFromName(projectId, shortTopic);
+    try {
+      if (topicExists(shortTopic)) {
+        NexmarkUtils.console("attempting to cleanup topic %s", topic);
+        pubsubClient.deleteTopic(topic);
+      }
+      NexmarkUtils.console("create topic %s", topic);
+      pubsubClient.createTopic(topic);
+      createdTopics.add(topic);
+      return topic;
+    } catch (IOException e) {
+      throw new RuntimeException("Unable to create Pubsub topic " + topic + ": ", e);
+    }
+  }
+
+  /**
+   * Create a topic from short name if it does not already exist. The topic will not be
+   * deleted on cleanup. Return full topic name.
+   */
+  public PubsubClient.TopicPath createOrReuseTopic(String shortTopic) {
+    PubsubClient.TopicPath topic = PubsubClient.topicPathFromName(projectId, shortTopic);
+    try {
+      if (topicExists(shortTopic)) {
+        NexmarkUtils.console("topic %s already exists", topic);
+        return topic;
+      }
+      NexmarkUtils.console("create topic %s", topic);
+      pubsubClient.createTopic(topic);
+      return topic;
+    } catch (IOException e) {
+      throw new RuntimeException("Unable to create or reuse Pubsub topic " + topic + ": ", e);
+    }
+  }
+
+  /**
+   * Check a topic corresponding to short name exists, and throw exception if not. The
+   * topic will not be deleted on cleanup. Return full topic name.
+   */
+  public PubsubClient.TopicPath reuseTopic(String shortTopic) {
+    PubsubClient.TopicPath topic = PubsubClient.topicPathFromName(projectId, shortTopic);
+    if (topicExists(shortTopic)) {
+      NexmarkUtils.console("reusing existing topic %s", topic);
+      return topic;
+    }
+    throw new RuntimeException("topic '" + topic + "' does not already exist");
+  }
+
+  /**
+   * Does topic corresponding to short name exist?
+   */
+  public boolean topicExists(String shortTopic) {
+    PubsubClient.ProjectPath project = PubsubClient.projectPathFromId(projectId);
+    PubsubClient.TopicPath topic = PubsubClient.topicPathFromName(projectId, shortTopic);
+    try {
+      Collection<PubsubClient.TopicPath> existingTopics = pubsubClient.listTopics(project);
+      return existingTopics.contains(topic);
+    } catch (IOException e) {
+      throw new RuntimeException("Unable to check Pubsub topic " + topic + ": ", e);
+    }
+  }
+
+  /**
+   * Create subscription from short name. Delete subscription if it already exists. Ensure the
+   * subscription will be deleted on cleanup. Return full subscription name.
+   */
+  public PubsubClient.SubscriptionPath createSubscription(
+      String shortTopic, String shortSubscription) {
+    PubsubClient.TopicPath topic = PubsubClient.topicPathFromName(projectId, shortTopic);
+    PubsubClient.SubscriptionPath subscription =
+        PubsubClient.subscriptionPathFromName(projectId, shortSubscription);
+    try {
+      if (subscriptionExists(shortTopic, shortSubscription)) {
+        NexmarkUtils.console("attempting to cleanup subscription %s", subscription);
+        pubsubClient.deleteSubscription(subscription);
+      }
+      NexmarkUtils.console("create subscription %s", subscription);
+      pubsubClient.createSubscription(topic, subscription, 60);
+      createdSubscriptions.add(subscription);
+    } catch (IOException e) {
+      throw new RuntimeException("Unable to create Pubsub subscription " + subscription + ": ", e);
+    }
+    return subscription;
+  }
+
+  /**
+   * Check a subscription corresponding to short name exists, and throw exception if not. The
+   * subscription will not be deleted on cleanup. Return full topic name.
+   */
+  public PubsubClient.SubscriptionPath reuseSubscription(
+      String shortTopic, String shortSubscription) {
+    PubsubClient.SubscriptionPath subscription =
+        PubsubClient.subscriptionPathFromName(projectId, shortSubscription);
+    if (subscriptionExists(shortTopic, shortSubscription)) {
+      NexmarkUtils.console("reusing existing subscription %s", subscription);
+      return subscription;
+    }
+    throw new RuntimeException("subscription'" + subscription + "' does not already exist");
+  }
+
+  /**
+   * Does subscription corresponding to short name exist?
+   */
+  public boolean subscriptionExists(String shortTopic, String shortSubscription) {
+    PubsubClient.ProjectPath project = PubsubClient.projectPathFromId(projectId);
+    PubsubClient.TopicPath topic = PubsubClient.topicPathFromName(projectId, shortTopic);
+    PubsubClient.SubscriptionPath subscription =
+        PubsubClient.subscriptionPathFromName(projectId, shortSubscription);
+    try {
+      Collection<PubsubClient.SubscriptionPath> existingSubscriptions =
+          pubsubClient.listSubscriptions(project, topic);
+      return existingSubscriptions.contains(subscription);
+    } catch (IOException e) {
+      throw new RuntimeException("Unable to check Pubsub subscription" + subscription + ": ", e);
+    }
+  }
+
+  /**
+   * Delete all the subscriptions and topics we created.
+   */
+  @Override
+  public void close() {
+    for (PubsubClient.SubscriptionPath subscription : createdSubscriptions) {
+      try {
+        NexmarkUtils.console("delete subscription %s", subscription);
+        pubsubClient.deleteSubscription(subscription);
+      } catch (IOException ex) {
+        NexmarkUtils.console("could not delete subscription %s", subscription);
+      }
+    }
+    for (PubsubClient.TopicPath topic : createdTopics) {
+      try {
+        NexmarkUtils.console("delete topic %s", topic);
+        pubsubClient.deleteTopic(topic);
+      } catch (IOException ex) {
+        NexmarkUtils.console("could not delete topic %s", topic);
+      }
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/1f08970a/integration/java/src/main/java/org/apache/beam/integration/nexmark/Query0.java
----------------------------------------------------------------------
diff --git a/integration/java/src/main/java/org/apache/beam/integration/nexmark/Query0.java b/integration/java/src/main/java/org/apache/beam/integration/nexmark/Query0.java
new file mode 100644
index 0000000..ea0d7ca
--- /dev/null
+++ b/integration/java/src/main/java/org/apache/beam/integration/nexmark/Query0.java
@@ -0,0 +1,72 @@
+/*
+ * 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.beam.integration.nexmark;
+
+import org.apache.beam.sdk.coders.Coder;
+import org.apache.beam.sdk.coders.CoderException;
+import org.apache.beam.sdk.transforms.Aggregator;
+import org.apache.beam.sdk.transforms.DoFn;
+import org.apache.beam.sdk.transforms.ParDo;
+import org.apache.beam.sdk.transforms.Sum.SumLongFn;
+import org.apache.beam.sdk.values.PCollection;
+
+
+import java.io.ByteArrayInputStream;
+import java.io.ByteArrayOutputStream;
+import java.io.IOException;
+
+/**
+ * Query 0: Pass events through unchanged. However, force them to do a round trip through
+ * serialization so that we measure the impact of the choice of coders.
+ */
+public class Query0 extends NexmarkQuery {
+  public Query0(NexmarkConfiguration configuration) {
+    super(configuration, "Query0");
+  }
+
+  private PCollection<Event> applyTyped(PCollection<Event> events) {
+    final Coder<Event> coder = events.getCoder();
+
+    return events
+
+        // Force round trip through coder.
+        .apply(
+            ParDo.named(name + ".Serialize")
+                .of(new DoFn<Event, Event>() {
+                  private final Aggregator<Long, Long> bytes =
+                      createAggregator("bytes", new SumLongFn());
+
+                  @Override
+                  public void processElement(ProcessContext c) throws CoderException, IOException {
+                    ByteArrayOutputStream outStream = new ByteArrayOutputStream();
+                    coder.encode(c.element(), outStream, Coder.Context.OUTER);
+                    byte[] byteArray = outStream.toByteArray();
+                    bytes.addValue((long) byteArray.length);
+                    ByteArrayInputStream inStream = new ByteArrayInputStream(byteArray);
+                    Event event = coder.decode(inStream, Coder.Context.OUTER);
+                    c.output(event);
+                  }
+                }));
+  }
+
+  @Override
+  protected PCollection<KnownSize> applyPrim(PCollection<Event> events) {
+    return NexmarkUtils.castToKnownSize(name, applyTyped(events));
+  }
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/1f08970a/integration/java/src/main/java/org/apache/beam/integration/nexmark/Query0Model.java
----------------------------------------------------------------------
diff --git a/integration/java/src/main/java/org/apache/beam/integration/nexmark/Query0Model.java b/integration/java/src/main/java/org/apache/beam/integration/nexmark/Query0Model.java
new file mode 100644
index 0000000..f3ceca2
--- /dev/null
+++ b/integration/java/src/main/java/org/apache/beam/integration/nexmark/Query0Model.java
@@ -0,0 +1,62 @@
+/*
+ * 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.beam.integration.nexmark;
+
+import org.apache.beam.sdk.values.TimestampedValue;
+
+import java.util.Collection;
+import java.util.Iterator;
+
+/**
+ * A direct implementation of {@link Query0}.
+ */
+public class Query0Model extends NexmarkQueryModel {
+  /**
+   * Simulator for query 0.
+   */
+  private class Simulator extends AbstractSimulator<Event, Event> {
+    public Simulator(NexmarkConfiguration configuration) {
+      super(NexmarkUtils.standardEventIterator(configuration));
+    }
+
+    @Override
+    protected void run() {
+      TimestampedValue<Event> timestampedEvent = nextInput();
+      if (timestampedEvent == null) {
+        allDone();
+        return;
+      }
+      addResult(timestampedEvent);
+    }
+  }
+
+  public Query0Model(NexmarkConfiguration configuration) {
+    super(configuration);
+  }
+
+  @Override
+  protected AbstractSimulator<?, ?> simulator() {
+    return new Simulator(configuration);
+  }
+
+  @Override
+  protected <T> Collection<String> toCollection(Iterator<TimestampedValue<T>> itr) {
+    return toValueTimestampOrder(itr);
+  }
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/1f08970a/integration/java/src/main/java/org/apache/beam/integration/nexmark/Query1.java
----------------------------------------------------------------------
diff --git a/integration/java/src/main/java/org/apache/beam/integration/nexmark/Query1.java b/integration/java/src/main/java/org/apache/beam/integration/nexmark/Query1.java
new file mode 100644
index 0000000..7e60b9c
--- /dev/null
+++ b/integration/java/src/main/java/org/apache/beam/integration/nexmark/Query1.java
@@ -0,0 +1,64 @@
+/*
+ * 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.beam.integration.nexmark;
+
+import org.apache.beam.sdk.transforms.DoFn;
+import org.apache.beam.sdk.transforms.ParDo;
+import org.apache.beam.sdk.values.PCollection;
+
+/**
+ * Query 1, 'Currency Conversion'. Convert each bid value from dollars to euros.
+ * In CQL syntax:
+ *
+ * <pre>
+ * SELECT Istream(auction, DOLTOEUR(price), bidder, datetime)
+ * FROM bid [ROWS UNBOUNDED];
+ * </pre>
+ *
+ * <p>To make things more interesting, allow the 'currency conversion' to be arbitrarily
+ * slowed down.
+ */
+class Query1 extends NexmarkQuery {
+  public Query1(NexmarkConfiguration configuration) {
+    super(configuration, "Query1");
+  }
+
+  private PCollection<Bid> applyTyped(PCollection<Event> events) {
+    return events
+        // Only want the bid events.
+        .apply(JUST_BIDS)
+
+        // Map the conversion function over all bids.
+        .apply(
+            ParDo.named(name + ".ToEuros")
+                .of(new DoFn<Bid, Bid>() {
+                  @Override
+                  public void processElement(ProcessContext c) {
+                    Bid bid = c.element();
+                    c.output(new Bid(
+                        bid.auction, bid.bidder, (bid.price * 89) / 100, bid.dateTime, bid.extra));
+                  }
+                }));
+  }
+
+  @Override
+  protected PCollection<KnownSize> applyPrim(PCollection<Event> events) {
+    return NexmarkUtils.castToKnownSize(name, applyTyped(events));
+  }
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/1f08970a/integration/java/src/main/java/org/apache/beam/integration/nexmark/Query10.java
----------------------------------------------------------------------
diff --git a/integration/java/src/main/java/org/apache/beam/integration/nexmark/Query10.java b/integration/java/src/main/java/org/apache/beam/integration/nexmark/Query10.java
new file mode 100644
index 0000000..74fb28c
--- /dev/null
+++ b/integration/java/src/main/java/org/apache/beam/integration/nexmark/Query10.java
@@ -0,0 +1,378 @@
+/*
+ * 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.beam.integration.nexmark;
+
+import org.apache.beam.sdk.coders.Coder;
+import org.apache.beam.sdk.options.GcsOptions;
+import org.apache.beam.sdk.transforms.Aggregator;
+import org.apache.beam.sdk.transforms.DoFn;
+import org.apache.beam.sdk.transforms.DoFnWithContext;
+import org.apache.beam.sdk.transforms.GroupByKey;
+import org.apache.beam.sdk.transforms.ParDo;
+import org.apache.beam.sdk.transforms.Sum.SumLongFn;
+import org.apache.beam.sdk.transforms.windowing.AfterEach;
+import org.apache.beam.sdk.transforms.windowing.AfterFirst;
+import org.apache.beam.sdk.transforms.windowing.AfterPane;
+import org.apache.beam.sdk.transforms.windowing.AfterProcessingTime;
+import org.apache.beam.sdk.transforms.windowing.AfterWatermark;
+import org.apache.beam.sdk.transforms.windowing.BoundedWindow;
+import org.apache.beam.sdk.transforms.windowing.FixedWindows;
+import org.apache.beam.sdk.transforms.windowing.PaneInfo;
+import org.apache.beam.sdk.transforms.windowing.PaneInfo.Timing;
+import org.apache.beam.sdk.transforms.windowing.Repeatedly;
+import org.apache.beam.sdk.transforms.windowing.Window;
+import org.apache.beam.sdk.util.GcsIOChannelFactory;
+import org.apache.beam.sdk.values.KV;
+import org.apache.beam.sdk.values.PCollection;
+
+import com.google.cloud.hadoop.gcsio.GoogleCloudStorageWriteChannel;
+import com.google.common.base.Preconditions;
+
+import org.joda.time.Duration;
+import org.joda.time.Instant;
+
+import java.io.IOException;
+import java.io.OutputStream;
+import java.io.Serializable;
+import java.nio.channels.Channels;
+import java.nio.channels.WritableByteChannel;
+import java.util.concurrent.ThreadLocalRandom;
+import javax.annotation.Nullable;
+
+/**
+ * Query "10", 'Log to sharded files' (Not in original suite.)
+ *
+ * <p>Every windowSizeSec, save all events from the last period into 2*maxWorkers log files.
+ */
+class Query10 extends NexmarkQuery {
+  private static final int CHANNEL_BUFFER = 8 << 20; // 8MB
+  private static final int NUM_SHARDS_PER_WORKER = 5;
+  private static final Duration LATE_BATCHING_PERIOD = Duration.standardSeconds(10);
+
+  /**
+   * Capture everything we need to know about the records in a single output file.
+   */
+  private static class OutputFile implements Serializable {
+    /** Maximum possible timestamp of records in file. */
+    private final Instant maxTimestamp;
+    /** Shard within window. */
+    private final String shard;
+    /** Index of file in all files in shard. */
+    private final long index;
+    /** Timing of records in this file. */
+    private final PaneInfo.Timing timing;
+    /** Path to file containing records, or {@literal null} if no output required. */
+    @Nullable
+    private final String filename;
+
+    public OutputFile(
+        Instant maxTimestamp,
+        String shard,
+        long index,
+        PaneInfo.Timing timing,
+        @Nullable String filename) {
+      this.maxTimestamp = maxTimestamp;
+      this.shard = shard;
+      this.index = index;
+      this.timing = timing;
+      this.filename = filename;
+    }
+
+    @Override
+    public String toString() {
+      return String.format("%s %s %d %s %s\n", maxTimestamp, shard, index, timing, filename);
+    }
+  }
+
+  /**
+   * GCS uri prefix for all log and 'finished' files. If null they won't be written.
+   */
+  @Nullable
+  private String outputPath;
+
+  /**
+   * Maximum number of workers, used to determine log sharding factor.
+   */
+  private int maxNumWorkers;
+
+  public Query10(NexmarkConfiguration configuration) {
+    super(configuration, "Query10");
+  }
+
+  public void setOutputPath(@Nullable String outputPath) {
+    this.outputPath = outputPath;
+  }
+
+  public void setMaxNumWorkers(int maxNumWorkers) {
+    this.maxNumWorkers = maxNumWorkers;
+  }
+
+  /**
+   * Return channel for writing bytes to GCS.
+   *
+   * @throws IOException
+   */
+  private WritableByteChannel openWritableGcsFile(GcsOptions options, String filename)
+      throws IOException {
+    WritableByteChannel channel = new GcsIOChannelFactory(options).create(filename, "text/plain");
+    Preconditions.checkState(channel instanceof GoogleCloudStorageWriteChannel);
+    ((GoogleCloudStorageWriteChannel) channel).setUploadBufferSize(CHANNEL_BUFFER);
+    return channel;
+  }
+
+  /** Return a short string to describe {@code timing}. */
+  private String timingToString(PaneInfo.Timing timing) {
+    switch (timing) {
+      case EARLY:
+        return "E";
+      case ON_TIME:
+        return "O";
+      case LATE:
+        return "L";
+    }
+    throw new RuntimeException(); // cases are exhaustive
+  }
+
+  /** Construct an {@link OutputFile} for {@code pane} in {@code window} for {@code shard}. */
+  private OutputFile outputFileFor(BoundedWindow window, String shard, PaneInfo pane) {
+    @Nullable String filename =
+        outputPath == null
+        ? null
+        : String.format("%s/LOG-%s-%s-%03d-%s-%x",
+            outputPath, window.maxTimestamp(), shard, pane.getIndex(),
+            timingToString(pane.getTiming()),
+            ThreadLocalRandom.current().nextLong());
+    return new OutputFile(window.maxTimestamp(), shard, pane.getIndex(),
+        pane.getTiming(), filename);
+  }
+
+  /**
+   * Return path to which we should write the index for {@code window}, or {@literal null}
+   * if no output required.
+   */
+  @Nullable
+  private String indexPathFor(BoundedWindow window) {
+    if (outputPath == null) {
+      return null;
+    }
+    return String.format("%s/INDEX-%s", outputPath, window.maxTimestamp());
+  }
+
+  private PCollection<Done> applyTyped(PCollection<Event> events) {
+    final int numLogShards = maxNumWorkers * NUM_SHARDS_PER_WORKER;
+
+    return events
+        .apply(ParDo.named(name + ".ShardEvents")
+                    .of(new DoFn<Event, KV<String, Event>>() {
+                      final Aggregator<Long, Long> lateCounter =
+                          createAggregator("actuallyLateEvent", new SumLongFn());
+                      final Aggregator<Long, Long> onTimeCounter =
+                          createAggregator("actuallyOnTimeEvent", new SumLongFn());
+
+                      @Override
+                      public void processElement(ProcessContext c) {
+                        if (c.element().hasAnnotation("LATE")) {
+                          lateCounter.addValue(1L);
+                          NexmarkUtils.error("Observed late: %s", c.element());
+                        } else {
+                          onTimeCounter.addValue(1L);
+                        }
+                        int shardNum = (int) Math.abs((long) c.element().hashCode() % numLogShards);
+                        String shard = String.format("shard-%05d-of-%05d", shardNum, numLogShards);
+                        c.output(KV.of(shard, c.element()));
+                      }
+                    }))
+        .apply(Window.<KV<String, Event>>into(
+            FixedWindows.of(Duration.standardSeconds(configuration.windowSizeSec)))
+            .named(name + ".WindowEvents")
+            .triggering(AfterEach.inOrder(
+                Repeatedly
+                    .forever(AfterPane.elementCountAtLeast(configuration.maxLogEvents))
+                    .orFinally(AfterWatermark.pastEndOfWindow()),
+                Repeatedly.forever(
+                    AfterFirst.of(AfterPane.elementCountAtLeast(configuration.maxLogEvents),
+                        AfterProcessingTime.pastFirstElementInPane()
+                                           .plusDelayOf(LATE_BATCHING_PERIOD)))))
+            .discardingFiredPanes()
+            // Use a 1 day allowed lateness so that any forgotten hold will stall the
+            // pipeline for that period and be very noticeable.
+            .withAllowedLateness(Duration.standardDays(1)))
+        .apply(GroupByKey.<String, Event>create())
+        .apply(
+            ParDo.named(name + ".CheckForLateEvents")
+                 .of(new DoFnWithContext<KV<String, Iterable<Event>>,
+                     KV<String, Iterable<Event>>>() {
+                   final Aggregator<Long, Long> earlyCounter =
+                       createAggregator("earlyShard", new SumLongFn());
+                   final Aggregator<Long, Long> onTimeCounter =
+                       createAggregator("onTimeShard", new SumLongFn());
+                   final Aggregator<Long, Long> lateCounter =
+                       createAggregator("lateShard", new SumLongFn());
+                   final Aggregator<Long, Long> unexpectedLatePaneCounter =
+                       createAggregator("ERROR_unexpectedLatePane", new SumLongFn());
+                   final Aggregator<Long, Long> unexpectedOnTimeElementCounter =
+                       createAggregator("ERROR_unexpectedOnTimeElement", new SumLongFn());
+
+                   @ProcessElement
+                   public void processElement(ProcessContext c, BoundedWindow window) {
+                     int numLate = 0;
+                     int numOnTime = 0;
+                     for (Event event : c.element().getValue()) {
+                       if (event.hasAnnotation("LATE")) {
+                         numLate++;
+                       } else {
+                         numOnTime++;
+                       }
+                     }
+                     String shard = c.element().getKey();
+                     NexmarkUtils.error(
+                         "%s with timestamp %s has %d actually late and %d on-time "
+                             + "elements in pane %s for window %s",
+                         shard, c.timestamp(), numLate, numOnTime, c.pane(),
+                         window.maxTimestamp());
+                     if (c.pane().getTiming() == PaneInfo.Timing.LATE) {
+                       if (numLate == 0) {
+                         NexmarkUtils.error(
+                             "ERROR! No late events in late pane for %s", shard);
+                         unexpectedLatePaneCounter.addValue(1L);
+                       }
+                       if (numOnTime > 0) {
+                         NexmarkUtils.error(
+                             "ERROR! Have %d on-time events in late pane for %s",
+                             numOnTime, shard);
+                         unexpectedOnTimeElementCounter.addValue(1L);
+                       }
+                       lateCounter.addValue(1L);
+                     } else if (c.pane().getTiming() == PaneInfo.Timing.EARLY) {
+                       if (numOnTime + numLate < configuration.maxLogEvents) {
+                         NexmarkUtils.error(
+                             "ERROR! Only have %d events in early pane for %s",
+                             numOnTime + numLate, shard);
+                       }
+                       earlyCounter.addValue(1L);
+                     } else {
+                       onTimeCounter.addValue(1L);
+                     }
+                     c.output(c.element());
+                   }
+                 }))
+        .apply(
+            ParDo.named(name + ".UploadEvents")
+                 .of(new DoFnWithContext<KV<String, Iterable<Event>>,
+                     KV<Void, OutputFile>>() {
+                   final Aggregator<Long, Long> savedFileCounter =
+                       createAggregator("savedFile", new SumLongFn());
+                   final Aggregator<Long, Long> writtenRecordsCounter =
+                       createAggregator("writtenRecords", new SumLongFn());
+
+                   @ProcessElement
+                   public void process(ProcessContext c, BoundedWindow window) throws IOException {
+                     String shard = c.element().getKey();
+                     GcsOptions options = c.getPipelineOptions().as(GcsOptions.class);
+                     OutputFile outputFile = outputFileFor(window, shard, c.pane());
+                     NexmarkUtils.error(
+                         "Writing %s with record timestamp %s, window timestamp %s, pane %s",
+                         shard, c.timestamp(), window.maxTimestamp(), c.pane());
+                     if (outputFile.filename != null) {
+                       NexmarkUtils.error("Beginning write to '%s'", outputFile.filename);
+                       int n = 0;
+                       try (OutputStream output =
+                                Channels.newOutputStream(openWritableGcsFile(options, outputFile
+                                    .filename))) {
+                         for (Event event : c.element().getValue()) {
+                           Event.CODER.encode(event, output, Coder.Context.OUTER);
+                           writtenRecordsCounter.addValue(1L);
+                           if (++n % 10000 == 0) {
+                             NexmarkUtils.error("So far written %d records to '%s'", n,
+                                 outputFile.filename);
+                           }
+                         }
+                       }
+                       NexmarkUtils.error("Written all %d records to '%s'", n, outputFile.filename);
+                     }
+                     savedFileCounter.addValue(1L);
+                     c.output(KV.<Void, OutputFile>of(null, outputFile));
+                   }
+                 }))
+        // Clear fancy triggering from above.
+        .apply(Window.<KV<Void, OutputFile>>into(
+            FixedWindows.of(Duration.standardSeconds(configuration.windowSizeSec)))
+            .named(name + ".WindowLogFiles")
+            .triggering(AfterWatermark.pastEndOfWindow())
+            // We expect no late data here, but we'll assume the worst so we can detect any.
+            .withAllowedLateness(Duration.standardDays(1))
+            .discardingFiredPanes())
+        .apply(GroupByKey.<Void, OutputFile>create())
+        .apply(
+            ParDo.named(name + ".Index")
+                 .of(new DoFnWithContext<KV<Void, Iterable<OutputFile>>, Done>() {
+                   final Aggregator<Long, Long> unexpectedLateCounter =
+                       createAggregator("ERROR_unexpectedLate", new SumLongFn());
+                   final Aggregator<Long, Long> unexpectedEarlyCounter =
+                       createAggregator("ERROR_unexpectedEarly", new SumLongFn());
+                   final Aggregator<Long, Long> unexpectedIndexCounter =
+                       createAggregator("ERROR_unexpectedIndex", new SumLongFn());
+                   final Aggregator<Long, Long> finalizedCounter =
+                       createAggregator("indexed", new SumLongFn());
+
+                   @ProcessElement
+                   public void process(ProcessContext c, BoundedWindow window) throws IOException {
+                     if (c.pane().getTiming() == Timing.LATE) {
+                       unexpectedLateCounter.addValue(1L);
+                       NexmarkUtils.error("ERROR! Unexpected LATE pane: %s", c.pane());
+                     } else if (c.pane().getTiming() == Timing.EARLY) {
+                       unexpectedEarlyCounter.addValue(1L);
+                       NexmarkUtils.error("ERROR! Unexpected EARLY pane: %s", c.pane());
+                     } else if (c.pane().getTiming() == Timing.ON_TIME
+                         && c.pane().getIndex() != 0) {
+                       unexpectedIndexCounter.addValue(1L);
+                       NexmarkUtils.error("ERROR! Unexpected ON_TIME pane index: %s", c.pane());
+                     } else {
+                       GcsOptions options = c.getPipelineOptions().as(GcsOptions.class);
+                       NexmarkUtils.error(
+                           "Index with record timestamp %s, window timestamp %s, pane %s",
+                           c.timestamp(), window.maxTimestamp(), c.pane());
+
+                       @Nullable String filename = indexPathFor(window);
+                       if (filename != null) {
+                         NexmarkUtils.error("Beginning write to '%s'", filename);
+                         int n = 0;
+                         try (OutputStream output =
+                                  Channels.newOutputStream(
+                                      openWritableGcsFile(options, filename))) {
+                           for (OutputFile outputFile : c.element().getValue()) {
+                             output.write(outputFile.toString().getBytes());
+                             n++;
+                           }
+                         }
+                         NexmarkUtils.error("Written all %d lines to '%s'", n, filename);
+                       }
+                       c.output(
+                           new Done("written for timestamp " + window.maxTimestamp()));
+                       finalizedCounter.addValue(1L);
+                     }
+                   }
+                 }));
+  }
+
+  @Override
+  protected PCollection<KnownSize> applyPrim(PCollection<Event> events) {
+    return NexmarkUtils.castToKnownSize(name, applyTyped(events));
+  }
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/1f08970a/integration/java/src/main/java/org/apache/beam/integration/nexmark/Query11.java
----------------------------------------------------------------------
diff --git a/integration/java/src/main/java/org/apache/beam/integration/nexmark/Query11.java b/integration/java/src/main/java/org/apache/beam/integration/nexmark/Query11.java
new file mode 100644
index 0000000..9841421
--- /dev/null
+++ b/integration/java/src/main/java/org/apache/beam/integration/nexmark/Query11.java
@@ -0,0 +1,76 @@
+/*
+ * 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.beam.integration.nexmark;
+
+import org.apache.beam.sdk.transforms.Count;
+import org.apache.beam.sdk.transforms.DoFn;
+import org.apache.beam.sdk.transforms.ParDo;
+import org.apache.beam.sdk.transforms.windowing.AfterPane;
+import org.apache.beam.sdk.transforms.windowing.Repeatedly;
+import org.apache.beam.sdk.transforms.windowing.Sessions;
+import org.apache.beam.sdk.transforms.windowing.Window;
+import org.apache.beam.sdk.values.KV;
+import org.apache.beam.sdk.values.PCollection;
+
+import org.joda.time.Duration;
+
+/**
+ * Query "11", 'User sessions' (Not in original suite.)
+ *
+ * <p>Group bids by the same user into sessions with {@code windowSizeSec} max gap.
+ * However limit the session to at most {@code maxLogEvents}. Emit the number of
+ * bids per session.
+ */
+class Query11 extends NexmarkQuery {
+  public Query11(NexmarkConfiguration configuration) {
+    super(configuration, "Query11");
+  }
+
+  private PCollection<BidsPerSession> applyTyped(PCollection<Event> events) {
+    return events.apply(JUST_BIDS)
+        .apply(
+            ParDo.named(name + ".Rekey")
+                .of(new DoFn<Bid, KV<Long, Void>>() {
+                  @Override
+                  public void processElement(ProcessContext c) {
+                    Bid bid = c.element();
+                    c.output(KV.of(bid.bidder, (Void) null));
+                  }
+                }))
+        .apply(Window.<KV<Long, Void>>into(
+            Sessions.withGapDuration(Duration.standardSeconds(configuration.windowSizeSec)))
+        .triggering(Repeatedly.forever(AfterPane.elementCountAtLeast(configuration.maxLogEvents)))
+        .discardingFiredPanes()
+        .withAllowedLateness(Duration.standardSeconds(configuration.occasionalDelaySec / 2)))
+        .apply(Count.<Long, Void>perKey())
+        .apply(
+            ParDo.named(name + ".ToResult")
+                .of(new DoFn<KV<Long, Long>, BidsPerSession>() {
+                  @Override
+                  public void processElement(ProcessContext c) {
+                    c.output(new BidsPerSession(c.element().getKey(), c.element().getValue()));
+                  }
+                }));
+  }
+
+  @Override
+  protected PCollection<KnownSize> applyPrim(PCollection<Event> events) {
+    return NexmarkUtils.castToKnownSize(name, applyTyped(events));
+  }
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/1f08970a/integration/java/src/main/java/org/apache/beam/integration/nexmark/Query12.java
----------------------------------------------------------------------
diff --git a/integration/java/src/main/java/org/apache/beam/integration/nexmark/Query12.java b/integration/java/src/main/java/org/apache/beam/integration/nexmark/Query12.java
new file mode 100644
index 0000000..dd39971
--- /dev/null
+++ b/integration/java/src/main/java/org/apache/beam/integration/nexmark/Query12.java
@@ -0,0 +1,79 @@
+/*
+ * 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.beam.integration.nexmark;
+
+import org.apache.beam.sdk.transforms.Count;
+import org.apache.beam.sdk.transforms.DoFn;
+import org.apache.beam.sdk.transforms.ParDo;
+import org.apache.beam.sdk.transforms.windowing.AfterProcessingTime;
+import org.apache.beam.sdk.transforms.windowing.GlobalWindows;
+import org.apache.beam.sdk.transforms.windowing.Repeatedly;
+import org.apache.beam.sdk.transforms.windowing.Window;
+import org.apache.beam.sdk.values.KV;
+import org.apache.beam.sdk.values.PCollection;
+import org.joda.time.Duration;
+
+/**
+ * Query "12", 'Processing time windows' (Not in original suite.)
+ * <p>
+ * <p>Group bids by the same user into processing time windows of windowSize. Emit the count
+ * of bids per window.
+ */
+class Query12 extends NexmarkQuery {
+  public Query12(NexmarkConfiguration configuration) {
+    super(configuration, "Query12");
+  }
+
+  private PCollection<BidsPerSession> applyTyped(PCollection<Event> events) {
+    return events
+        .apply(JUST_BIDS)
+        .apply(
+            ParDo.named(name + ".Rekey")
+                 .of(new DoFn<Bid, KV<Long, Void>>() {
+                   @Override
+                   public void processElement(ProcessContext c) {
+                     Bid bid = c.element();
+                     c.output(KV.of(bid.bidder, (Void) null));
+                   }
+                 }))
+        .apply(Window.<KV<Long, Void>>into(new GlobalWindows())
+            .triggering(
+                Repeatedly.forever(
+                    AfterProcessingTime.pastFirstElementInPane()
+                                       .plusDelayOf(
+                                           Duration.standardSeconds(configuration.windowSizeSec))))
+            .discardingFiredPanes()
+            .withAllowedLateness(Duration.ZERO))
+        .apply(Count.<Long, Void>perKey())
+        .apply(
+            ParDo.named(name + ".ToResult")
+                 .of(new DoFn<KV<Long, Long>, BidsPerSession>() {
+                   @Override
+                   public void processElement(ProcessContext c) {
+                     c.output(
+                         new BidsPerSession(c.element().getKey(), c.element().getValue()));
+                   }
+                 }));
+  }
+
+  @Override
+  protected PCollection<KnownSize> applyPrim(PCollection<Event> events) {
+    return NexmarkUtils.castToKnownSize(name, applyTyped(events));
+  }
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/1f08970a/integration/java/src/main/java/org/apache/beam/integration/nexmark/Query1Model.java
----------------------------------------------------------------------
diff --git a/integration/java/src/main/java/org/apache/beam/integration/nexmark/Query1Model.java b/integration/java/src/main/java/org/apache/beam/integration/nexmark/Query1Model.java
new file mode 100644
index 0000000..462d426
--- /dev/null
+++ b/integration/java/src/main/java/org/apache/beam/integration/nexmark/Query1Model.java
@@ -0,0 +1,73 @@
+/*
+ * 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.beam.integration.nexmark;
+
+import org.apache.beam.sdk.values.TimestampedValue;
+
+import java.io.Serializable;
+import java.util.Collection;
+import java.util.Iterator;
+
+/**
+ * A direct implementation of {@link Query1}.
+ */
+public class Query1Model extends NexmarkQueryModel implements Serializable {
+  /**
+   * Simulator for query 1.
+   */
+  private class Simulator extends AbstractSimulator<Event, Bid> {
+    public Simulator(NexmarkConfiguration configuration) {
+      super(NexmarkUtils.standardEventIterator(configuration));
+    }
+
+    @Override
+    protected void run() {
+      TimestampedValue<Event> timestampedEvent = nextInput();
+      if (timestampedEvent == null) {
+        allDone();
+        return;
+      }
+      Event event = timestampedEvent.getValue();
+      if (event.bid == null) {
+        // Ignore non-bid events.
+        return;
+      }
+      Bid bid = event.bid;
+      Bid resultBid =
+          new Bid(bid.auction, bid.bidder, bid.price * 89 / 100, bid.dateTime, bid.extra);
+      TimestampedValue<Bid> result =
+          TimestampedValue.of(resultBid, timestampedEvent.getTimestamp());
+      addResult(result);
+    }
+  }
+
+  public Query1Model(NexmarkConfiguration configuration) {
+    super(configuration);
+  }
+
+  @Override
+  public AbstractSimulator<?, ?> simulator() {
+    return new Simulator(configuration);
+  }
+
+  @Override
+  protected <T> Collection<String> toCollection(Iterator<TimestampedValue<T>> itr) {
+    return toValueTimestampOrder(itr);
+  }
+}


[41/55] [abbrv] beam git commit: Move module beam-integration-java-nexmark to beam-sdks-java-nexmark

Posted by ie...@apache.org.
http://git-wip-us.apache.org/repos/asf/beam/blob/f4333df7/sdks/java/nexmark/src/main/java/org/apache/beam/sdk/nexmark/queries/package-info.java
----------------------------------------------------------------------
diff --git a/sdks/java/nexmark/src/main/java/org/apache/beam/sdk/nexmark/queries/package-info.java b/sdks/java/nexmark/src/main/java/org/apache/beam/sdk/nexmark/queries/package-info.java
new file mode 100644
index 0000000..2ca5a1c
--- /dev/null
+++ b/sdks/java/nexmark/src/main/java/org/apache/beam/sdk/nexmark/queries/package-info.java
@@ -0,0 +1,22 @@
+/*
+ * 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.
+ */
+
+/**
+ * Nexmark Queries.
+ */
+package org.apache.beam.sdk.nexmark.queries;

http://git-wip-us.apache.org/repos/asf/beam/blob/f4333df7/sdks/java/nexmark/src/main/java/org/apache/beam/sdk/nexmark/sources/BoundedEventSource.java
----------------------------------------------------------------------
diff --git a/sdks/java/nexmark/src/main/java/org/apache/beam/sdk/nexmark/sources/BoundedEventSource.java b/sdks/java/nexmark/src/main/java/org/apache/beam/sdk/nexmark/sources/BoundedEventSource.java
new file mode 100644
index 0000000..60124bb
--- /dev/null
+++ b/sdks/java/nexmark/src/main/java/org/apache/beam/sdk/nexmark/sources/BoundedEventSource.java
@@ -0,0 +1,190 @@
+/*
+ * 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.beam.sdk.nexmark.sources;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.NoSuchElementException;
+import javax.annotation.Nullable;
+import org.apache.beam.sdk.coders.Coder;
+import org.apache.beam.sdk.io.BoundedSource;
+import org.apache.beam.sdk.nexmark.NexmarkUtils;
+import org.apache.beam.sdk.nexmark.model.Event;
+import org.apache.beam.sdk.options.PipelineOptions;
+import org.apache.beam.sdk.values.TimestampedValue;
+import org.joda.time.Instant;
+
+/**
+ * A custom, bounded source of event records.
+ */
+public class BoundedEventSource extends BoundedSource<Event> {
+  /** Configuration we generate events against. */
+  private final GeneratorConfig config;
+
+  /** How many bounded sources to create. */
+  private final int numEventGenerators;
+
+  public BoundedEventSource(GeneratorConfig config, int numEventGenerators) {
+    this.config = config;
+    this.numEventGenerators = numEventGenerators;
+  }
+
+  /** A reader to pull events from the generator. */
+  private static class EventReader extends BoundedReader<Event> {
+    /**
+     * Event source we purporting to be reading from.
+     * (We can't use Java's capture-outer-class pointer since we must update
+     * this field on calls to splitAtFraction.)
+     */
+    private BoundedEventSource source;
+
+    /** Generator we are reading from. */
+    private final Generator generator;
+
+    private boolean reportedStop;
+
+    @Nullable
+    private TimestampedValue<Event> currentEvent;
+
+    public EventReader(BoundedEventSource source, GeneratorConfig config) {
+      this.source = source;
+      generator = new Generator(config);
+      reportedStop = false;
+    }
+
+    @Override
+    public synchronized boolean start() {
+      NexmarkUtils.info("starting bounded generator %s", generator);
+      return advance();
+    }
+
+    @Override
+    public synchronized boolean advance() {
+      if (!generator.hasNext()) {
+        // No more events.
+        if (!reportedStop) {
+          reportedStop = true;
+          NexmarkUtils.info("stopped bounded generator %s", generator);
+        }
+        return false;
+      }
+      currentEvent = generator.next();
+      return true;
+    }
+
+    @Override
+    public synchronized Event getCurrent() throws NoSuchElementException {
+      if (currentEvent == null) {
+        throw new NoSuchElementException();
+      }
+      return currentEvent.getValue();
+    }
+
+    @Override
+    public synchronized Instant getCurrentTimestamp() throws NoSuchElementException {
+      if (currentEvent == null) {
+        throw new NoSuchElementException();
+      }
+      return currentEvent.getTimestamp();
+    }
+
+    @Override
+    public void close() throws IOException {
+      // Nothing to close.
+    }
+
+    @Override
+    public synchronized Double getFractionConsumed() {
+      return generator.getFractionConsumed();
+    }
+
+    @Override
+    public synchronized BoundedSource<Event> getCurrentSource() {
+      return source;
+    }
+
+    @Override
+    @Nullable
+    public synchronized BoundedEventSource splitAtFraction(double fraction) {
+      long startId = generator.getCurrentConfig().getStartEventId();
+      long stopId = generator.getCurrentConfig().getStopEventId();
+      long size = stopId - startId;
+      long splitEventId = startId + Math.min((int) (size * fraction), size);
+      if (splitEventId <= generator.getNextEventId() || splitEventId == stopId) {
+        // Already passed this position or split results in left or right being empty.
+        NexmarkUtils.info("split failed for bounded generator %s at %f", generator, fraction);
+        return null;
+      }
+
+      NexmarkUtils.info("about to split bounded generator %s at %d", generator, splitEventId);
+
+      // Scale back the event space of the current generator, and return a generator config
+      // representing the event space we just 'stole' from the current generator.
+      GeneratorConfig remainingConfig = generator.splitAtEventId(splitEventId);
+
+      NexmarkUtils.info("split bounded generator into %s and %s", generator, remainingConfig);
+
+      // At this point
+      //   generator.events() ++ new Generator(remainingConfig).events()
+      //   == originalGenerator.events()
+
+      // We need a new source to represent the now smaller key space for this reader, so
+      // that we can maintain the invariant that
+      //   this.getCurrentSource().createReader(...)
+      // will yield the same output as this.
+      source = new BoundedEventSource(generator.getCurrentConfig(), source.numEventGenerators);
+
+      // Return a source from which we may read the 'stolen' event space.
+      return new BoundedEventSource(remainingConfig, source.numEventGenerators);
+    }
+  }
+
+  @Override
+  public List<BoundedEventSource> split(
+      long desiredBundleSizeBytes, PipelineOptions options) {
+    NexmarkUtils.info("slitting bounded source %s into %d sub-sources", config, numEventGenerators);
+    List<BoundedEventSource> results = new ArrayList<>();
+    // Ignore desiredBundleSizeBytes and use numEventGenerators instead.
+    for (GeneratorConfig subConfig : config.split(numEventGenerators)) {
+      results.add(new BoundedEventSource(subConfig, 1));
+    }
+    return results;
+  }
+
+  @Override
+  public long getEstimatedSizeBytes(PipelineOptions options) {
+    return config.getEstimatedSizeBytes();
+  }
+
+  @Override
+  public EventReader createReader(PipelineOptions options) {
+    NexmarkUtils.info("creating initial bounded reader for %s", config);
+    return new EventReader(this, config);
+  }
+
+  @Override
+  public void validate() {
+    // Nothing to validate.
+  }
+
+  @Override
+  public Coder<Event> getDefaultOutputCoder() {
+    return Event.CODER;
+  }
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/f4333df7/sdks/java/nexmark/src/main/java/org/apache/beam/sdk/nexmark/sources/Generator.java
----------------------------------------------------------------------
diff --git a/sdks/java/nexmark/src/main/java/org/apache/beam/sdk/nexmark/sources/Generator.java b/sdks/java/nexmark/src/main/java/org/apache/beam/sdk/nexmark/sources/Generator.java
new file mode 100644
index 0000000..c368d72
--- /dev/null
+++ b/sdks/java/nexmark/src/main/java/org/apache/beam/sdk/nexmark/sources/Generator.java
@@ -0,0 +1,609 @@
+/*
+ * 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.beam.sdk.nexmark.sources;
+
+import static com.google.common.base.Preconditions.checkNotNull;
+
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
+import java.io.Serializable;
+import java.util.Arrays;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Objects;
+import java.util.Random;
+import org.apache.beam.sdk.coders.Coder;
+import org.apache.beam.sdk.coders.CoderException;
+import org.apache.beam.sdk.coders.CustomCoder;
+import org.apache.beam.sdk.coders.VarLongCoder;
+import org.apache.beam.sdk.io.UnboundedSource;
+import org.apache.beam.sdk.nexmark.model.Auction;
+import org.apache.beam.sdk.nexmark.model.Bid;
+import org.apache.beam.sdk.nexmark.model.Event;
+import org.apache.beam.sdk.nexmark.model.Person;
+import org.apache.beam.sdk.values.TimestampedValue;
+import org.joda.time.Instant;
+
+/**
+ * A generator for synthetic events. We try to make the data vaguely reasonable. We also ensure
+ * most primary key/foreign key relations are correct. Eg: a {@link Bid} event will usually have
+ * valid auction and bidder ids which can be joined to already-generated Auction and Person events.
+ *
+ * <p>To help with testing, we generate timestamps relative to a given {@code baseTime}. Each new
+ * event is given a timestamp advanced from the previous timestamp by {@code interEventDelayUs}
+ * (in microseconds). The event stream is thus fully deterministic and does not depend on
+ * wallclock time.
+ *
+ * <p>This class implements {@link org.apache.beam.sdk.io.UnboundedSource.CheckpointMark}
+ * so that we can resume generating events from a saved snapshot.
+ */
+public class Generator implements Iterator<TimestampedValue<Event>>, Serializable {
+  /**
+   * Keep the number of categories small so the example queries will find results even with
+   * a small batch of events.
+   */
+  private static final int NUM_CATEGORIES = 5;
+
+  /** Smallest random string size. */
+  private static final int MIN_STRING_LENGTH = 3;
+
+  /**
+   * Keep the number of states small so that the example queries will find results even with
+   * a small batch of events.
+   */
+  private static final List<String> US_STATES = Arrays.asList(("AZ,CA,ID,OR,WA,WY").split(","));
+
+  private static final List<String> US_CITIES =
+      Arrays.asList(
+          ("Phoenix,Los Angeles,San Francisco,Boise,Portland,Bend,Redmond,Seattle,Kent,Cheyenne")
+              .split(","));
+
+  private static final List<String> FIRST_NAMES =
+      Arrays.asList(("Peter,Paul,Luke,John,Saul,Vicky,Kate,Julie,Sarah,Deiter,Walter").split(","));
+
+  private static final List<String> LAST_NAMES =
+      Arrays.asList(("Shultz,Abrams,Spencer,White,Bartels,Walton,Smith,Jones,Noris").split(","));
+
+  /**
+   * Number of yet-to-be-created people and auction ids allowed.
+   */
+  private static final int PERSON_ID_LEAD = 10;
+  private static final int AUCTION_ID_LEAD = 10;
+
+  /**
+   * Fraction of people/auctions which may be 'hot' sellers/bidders/auctions are 1
+   * over these values.
+   */
+  private static final int HOT_AUCTION_RATIO = 100;
+  private static final int HOT_SELLER_RATIO = 100;
+  private static final int HOT_BIDDER_RATIO = 100;
+
+  /**
+   * Just enough state to be able to restore a generator back to where it was checkpointed.
+   */
+  public static class Checkpoint implements UnboundedSource.CheckpointMark {
+    private static final Coder<Long> LONG_CODER = VarLongCoder.of();
+
+    /** Coder for this class. */
+    public static final Coder<Checkpoint> CODER_INSTANCE =
+        new CustomCoder<Checkpoint>() {
+          @Override public void encode(Checkpoint value, OutputStream outStream)
+          throws CoderException, IOException {
+            LONG_CODER.encode(value.numEvents, outStream);
+            LONG_CODER.encode(value.wallclockBaseTime, outStream);
+          }
+
+          @Override
+          public Checkpoint decode(InputStream inStream)
+              throws CoderException, IOException {
+            long numEvents = LONG_CODER.decode(inStream);
+            long wallclockBaseTime = LONG_CODER.decode(inStream);
+            return new Checkpoint(numEvents, wallclockBaseTime);
+          }
+          @Override public void verifyDeterministic() throws NonDeterministicException {}
+        };
+
+    private final long numEvents;
+    private final long wallclockBaseTime;
+
+    private Checkpoint(long numEvents, long wallclockBaseTime) {
+      this.numEvents = numEvents;
+      this.wallclockBaseTime = wallclockBaseTime;
+    }
+
+    public Generator toGenerator(GeneratorConfig config) {
+      return new Generator(config, numEvents, wallclockBaseTime);
+    }
+
+    @Override
+    public void finalizeCheckpoint() throws IOException {
+      // Nothing to finalize.
+    }
+
+    @Override
+    public String toString() {
+      return String.format("Generator.Checkpoint{numEvents:%d;wallclockBaseTime:%d}",
+          numEvents, wallclockBaseTime);
+    }
+  }
+
+  /**
+   * The next event and its various timestamps. Ordered by increasing wallclock timestamp, then
+   * (arbitrary but stable) event hash order.
+   */
+  public static class NextEvent implements Comparable<NextEvent> {
+    /** When, in wallclock time, should this event be emitted? */
+    public final long wallclockTimestamp;
+
+    /** When, in event time, should this event be considered to have occured? */
+    public final long eventTimestamp;
+
+    /** The event itself. */
+    public final Event event;
+
+    /** The minimum of this and all future event timestamps. */
+    public final long watermark;
+
+    public NextEvent(long wallclockTimestamp, long eventTimestamp, Event event, long watermark) {
+      this.wallclockTimestamp = wallclockTimestamp;
+      this.eventTimestamp = eventTimestamp;
+      this.event = event;
+      this.watermark = watermark;
+    }
+
+    /**
+     * Return a deep copy of next event with delay added to wallclock timestamp and
+     * event annotate as 'LATE'.
+     */
+    public NextEvent withDelay(long delayMs) {
+      return new NextEvent(
+          wallclockTimestamp + delayMs, eventTimestamp, event.withAnnotation("LATE"), watermark);
+    }
+
+    @Override public boolean equals(Object o) {
+      if (this == o) {
+        return true;
+      }
+      if (o == null || getClass() != o.getClass()) {
+        return false;
+      }
+
+      NextEvent nextEvent = (NextEvent) o;
+
+      return (wallclockTimestamp == nextEvent.wallclockTimestamp
+          && eventTimestamp == nextEvent.eventTimestamp
+          && watermark == nextEvent.watermark
+          && event.equals(nextEvent.event));
+    }
+
+    @Override public int hashCode() {
+      return Objects.hash(wallclockTimestamp, eventTimestamp, watermark, event);
+    }
+
+    @Override
+    public int compareTo(NextEvent other) {
+      int i = Long.compare(wallclockTimestamp, other.wallclockTimestamp);
+      if (i != 0) {
+        return i;
+      }
+      return Integer.compare(event.hashCode(), other.event.hashCode());
+    }
+  }
+
+  /**
+   * Configuration to generate events against. Note that it may be replaced by a call to
+   * {@link #splitAtEventId}.
+   */
+  private GeneratorConfig config;
+
+  /** Number of events generated by this generator. */
+  private long numEvents;
+
+  /**
+   * Wallclock time at which we emitted the first event (ms since epoch). Initially -1.
+   */
+  private long wallclockBaseTime;
+
+  private Generator(GeneratorConfig config, long numEvents, long wallclockBaseTime) {
+    checkNotNull(config);
+    this.config = config;
+    this.numEvents = numEvents;
+    this.wallclockBaseTime = wallclockBaseTime;
+  }
+
+  /**
+   * Create a fresh generator according to {@code config}.
+   */
+  public Generator(GeneratorConfig config) {
+    this(config, 0, -1);
+  }
+
+  /**
+   * Return a checkpoint for the current generator.
+   */
+  public Checkpoint toCheckpoint() {
+    return new Checkpoint(numEvents, wallclockBaseTime);
+  }
+
+  /**
+   * Return a deep copy of this generator.
+   */
+  public Generator copy() {
+    checkNotNull(config);
+    Generator result = new Generator(config, numEvents, wallclockBaseTime);
+    return result;
+  }
+
+  /**
+   * Return the current config for this generator. Note that configs may be replaced by {@link
+   * #splitAtEventId}.
+   */
+  public GeneratorConfig getCurrentConfig() {
+    return config;
+  }
+
+  /**
+   * Mutate this generator so that it will only generate events up to but not including
+   * {@code eventId}. Return a config to represent the events this generator will no longer yield.
+   * The generators will run in on a serial timeline.
+   */
+  public GeneratorConfig splitAtEventId(long eventId) {
+    long newMaxEvents = eventId - (config.firstEventId + config.firstEventNumber);
+    GeneratorConfig remainConfig = config.copyWith(config.firstEventId,
+        config.maxEvents - newMaxEvents, config.firstEventNumber + newMaxEvents);
+    config = config.copyWith(config.firstEventId, newMaxEvents, config.firstEventNumber);
+    return remainConfig;
+  }
+
+  /**
+   * Return the next 'event id'. Though events don't have ids we can simulate them to
+   * help with bookkeeping.
+   */
+  public long getNextEventId() {
+    return config.firstEventId + config.nextAdjustedEventNumber(numEvents);
+  }
+
+  /**
+   * Return the last valid person id (ignoring FIRST_PERSON_ID). Will be the current person id if
+   * due to generate a person.
+   */
+  private long lastBase0PersonId() {
+    long eventId = getNextEventId();
+    long epoch = eventId / GeneratorConfig.PROPORTION_DENOMINATOR;
+    long offset = eventId % GeneratorConfig.PROPORTION_DENOMINATOR;
+    if (offset >= GeneratorConfig.PERSON_PROPORTION) {
+      // About to generate an auction or bid.
+      // Go back to the last person generated in this epoch.
+      offset = GeneratorConfig.PERSON_PROPORTION - 1;
+    }
+    // About to generate a person.
+    return epoch * GeneratorConfig.PERSON_PROPORTION + offset;
+  }
+
+  /**
+   * Return the last valid auction id (ignoring FIRST_AUCTION_ID). Will be the current auction id if
+   * due to generate an auction.
+   */
+  private long lastBase0AuctionId() {
+    long eventId = getNextEventId();
+    long epoch = eventId / GeneratorConfig.PROPORTION_DENOMINATOR;
+    long offset = eventId % GeneratorConfig.PROPORTION_DENOMINATOR;
+    if (offset < GeneratorConfig.PERSON_PROPORTION) {
+      // About to generate a person.
+      // Go back to the last auction in the last epoch.
+      epoch--;
+      offset = GeneratorConfig.AUCTION_PROPORTION - 1;
+    } else if (offset >= GeneratorConfig.PERSON_PROPORTION + GeneratorConfig.AUCTION_PROPORTION) {
+      // About to generate a bid.
+      // Go back to the last auction generated in this epoch.
+      offset = GeneratorConfig.AUCTION_PROPORTION - 1;
+    } else {
+      // About to generate an auction.
+      offset -= GeneratorConfig.PERSON_PROPORTION;
+    }
+    return epoch * GeneratorConfig.AUCTION_PROPORTION + offset;
+  }
+
+  /** return a random US state. */
+  private static String nextUSState(Random random) {
+    return US_STATES.get(random.nextInt(US_STATES.size()));
+  }
+
+  /** Return a random US city. */
+  private static String nextUSCity(Random random) {
+    return US_CITIES.get(random.nextInt(US_CITIES.size()));
+  }
+
+  /** Return a random person name. */
+  private static String nextPersonName(Random random) {
+    return FIRST_NAMES.get(random.nextInt(FIRST_NAMES.size())) + " "
+        + LAST_NAMES.get(random.nextInt(LAST_NAMES.size()));
+  }
+
+  /** Return a random string of up to {@code maxLength}. */
+  private static String nextString(Random random, int maxLength) {
+    int len = MIN_STRING_LENGTH + random.nextInt(maxLength - MIN_STRING_LENGTH);
+    StringBuilder sb = new StringBuilder();
+    while (len-- > 0) {
+      if (random.nextInt(13) == 0) {
+        sb.append(' ');
+      } else {
+        sb.append((char) ('a' + random.nextInt(26)));
+      }
+    }
+    return sb.toString().trim();
+  }
+
+  /** Return a random string of exactly {@code length}. */
+  private static String nextExactString(Random random, int length) {
+    StringBuilder sb = new StringBuilder();
+    while (length-- > 0) {
+      sb.append((char) ('a' + random.nextInt(26)));
+    }
+    return sb.toString();
+  }
+
+  /** Return a random email address. */
+  private static String nextEmail(Random random) {
+    return nextString(random, 7) + "@" + nextString(random, 5) + ".com";
+  }
+
+  /** Return a random credit card number. */
+  private static String nextCreditCard(Random random) {
+    StringBuilder sb = new StringBuilder();
+    for (int i = 0; i < 4; i++) {
+      if (i > 0) {
+        sb.append(' ');
+      }
+      sb.append(String.format("%04d", random.nextInt(10000)));
+    }
+    return sb.toString();
+  }
+
+  /** Return a random price. */
+  private static long nextPrice(Random random) {
+    return Math.round(Math.pow(10.0, random.nextDouble() * 6.0) * 100.0);
+  }
+
+  /** Return a random time delay, in milliseconds, for length of auctions. */
+  private long nextAuctionLengthMs(Random random, long timestamp) {
+    // What's our current event number?
+    long currentEventNumber = config.nextAdjustedEventNumber(numEvents);
+    // How many events till we've generated numInFlightAuctions?
+    long numEventsForAuctions =
+        (config.configuration.numInFlightAuctions * GeneratorConfig.PROPORTION_DENOMINATOR)
+        / GeneratorConfig.AUCTION_PROPORTION;
+    // When will the auction numInFlightAuctions beyond now be generated?
+    long futureAuction =
+        config.timestampAndInterEventDelayUsForEvent(currentEventNumber + numEventsForAuctions)
+            .getKey();
+    // System.out.printf("*** auction will be for %dms (%d events ahead) ***\n",
+    //     futureAuction - timestamp, numEventsForAuctions);
+    // Choose a length with average horizonMs.
+    long horizonMs = futureAuction - timestamp;
+    return 1L + nextLong(random, Math.max(horizonMs * 2, 1L));
+  }
+
+  /**
+   * Return a random {@code string} such that {@code currentSize + string.length()} is on average
+   * {@code averageSize}.
+   */
+  private static String nextExtra(Random random, int currentSize, int desiredAverageSize) {
+    if (currentSize > desiredAverageSize) {
+      return "";
+    }
+    desiredAverageSize -= currentSize;
+    int delta = (int) Math.round(desiredAverageSize * 0.2);
+    int minSize = desiredAverageSize - delta;
+    int desiredSize = minSize + (delta == 0 ? 0 : random.nextInt(2 * delta));
+    return nextExactString(random, desiredSize);
+  }
+
+  /** Return a random long from {@code [0, n)}. */
+  private static long nextLong(Random random, long n) {
+    if (n < Integer.MAX_VALUE) {
+      return random.nextInt((int) n);
+    } else {
+      // WARNING: Very skewed distribution! Bad!
+      return Math.abs(random.nextLong() % n);
+    }
+  }
+
+  /**
+   * Generate and return a random person with next available id.
+   */
+  private Person nextPerson(Random random, long timestamp) {
+    long id = lastBase0PersonId() + GeneratorConfig.FIRST_PERSON_ID;
+    String name = nextPersonName(random);
+    String email = nextEmail(random);
+    String creditCard = nextCreditCard(random);
+    String city = nextUSCity(random);
+    String state = nextUSState(random);
+    int currentSize =
+        8 + name.length() + email.length() + creditCard.length() + city.length() + state.length();
+    String extra = nextExtra(random, currentSize, config.configuration.avgPersonByteSize);
+    return new Person(id, name, email, creditCard, city, state, timestamp, extra);
+  }
+
+  /**
+   * Return a random person id (base 0).
+   */
+  private long nextBase0PersonId(Random random) {
+    // Choose a random person from any of the 'active' people, plus a few 'leads'.
+    // By limiting to 'active' we ensure the density of bids or auctions per person
+    // does not decrease over time for long running jobs.
+    // By choosing a person id ahead of the last valid person id we will make
+    // newPerson and newAuction events appear to have been swapped in time.
+    long numPeople = lastBase0PersonId() + 1;
+    long activePeople = Math.min(numPeople, config.configuration.numActivePeople);
+    long n = nextLong(random, activePeople + PERSON_ID_LEAD);
+    return numPeople - activePeople + n;
+  }
+
+  /**
+   * Return a random auction id (base 0).
+   */
+  private long nextBase0AuctionId(Random random) {
+    // Choose a random auction for any of those which are likely to still be in flight,
+    // plus a few 'leads'.
+    // Note that ideally we'd track non-expired auctions exactly, but that state
+    // is difficult to split.
+    long minAuction = Math.max(lastBase0AuctionId() - config.configuration.numInFlightAuctions, 0);
+    long maxAuction = lastBase0AuctionId();
+    return minAuction + nextLong(random, maxAuction - minAuction + 1 + AUCTION_ID_LEAD);
+  }
+
+  /**
+   * Generate and return a random auction with next available id.
+   */
+  private Auction nextAuction(Random random, long timestamp) {
+    long id = lastBase0AuctionId() + GeneratorConfig.FIRST_AUCTION_ID;
+
+    long seller;
+    // Here P(auction will be for a hot seller) = 1 - 1/hotSellersRatio.
+    if (random.nextInt(config.configuration.hotSellersRatio) > 0) {
+      // Choose the first person in the batch of last HOT_SELLER_RATIO people.
+      seller = (lastBase0PersonId() / HOT_SELLER_RATIO) * HOT_SELLER_RATIO;
+    } else {
+      seller = nextBase0PersonId(random);
+    }
+    seller += GeneratorConfig.FIRST_PERSON_ID;
+
+    long category = GeneratorConfig.FIRST_CATEGORY_ID + random.nextInt(NUM_CATEGORIES);
+    long initialBid = nextPrice(random);
+    long expires = timestamp + nextAuctionLengthMs(random, timestamp);
+    String name = nextString(random, 20);
+    String desc = nextString(random, 100);
+    long reserve = initialBid + nextPrice(random);
+    int currentSize = 8 + name.length() + desc.length() + 8 + 8 + 8 + 8 + 8;
+    String extra = nextExtra(random, currentSize, config.configuration.avgAuctionByteSize);
+    return new Auction(id, name, desc, initialBid, reserve, timestamp, expires, seller, category,
+        extra);
+  }
+
+  /**
+   * Generate and return a random bid with next available id.
+   */
+  private Bid nextBid(Random random, long timestamp) {
+    long auction;
+    // Here P(bid will be for a hot auction) = 1 - 1/hotAuctionRatio.
+    if (random.nextInt(config.configuration.hotAuctionRatio) > 0) {
+      // Choose the first auction in the batch of last HOT_AUCTION_RATIO auctions.
+      auction = (lastBase0AuctionId() / HOT_AUCTION_RATIO) * HOT_AUCTION_RATIO;
+    } else {
+      auction = nextBase0AuctionId(random);
+    }
+    auction += GeneratorConfig.FIRST_AUCTION_ID;
+
+    long bidder;
+    // Here P(bid will be by a hot bidder) = 1 - 1/hotBiddersRatio
+    if (random.nextInt(config.configuration.hotBiddersRatio) > 0) {
+      // Choose the second person (so hot bidders and hot sellers don't collide) in the batch of
+      // last HOT_BIDDER_RATIO people.
+      bidder = (lastBase0PersonId() / HOT_BIDDER_RATIO) * HOT_BIDDER_RATIO + 1;
+    } else {
+      bidder = nextBase0PersonId(random);
+    }
+    bidder += GeneratorConfig.FIRST_PERSON_ID;
+
+    long price = nextPrice(random);
+    int currentSize = 8 + 8 + 8 + 8;
+    String extra = nextExtra(random, currentSize, config.configuration.avgBidByteSize);
+    return new Bid(auction, bidder, price, timestamp, extra);
+  }
+
+  @Override
+  public boolean hasNext() {
+    return numEvents < config.maxEvents;
+  }
+
+  /**
+   * Return the next event. The outer timestamp is in wallclock time and corresponds to
+   * when the event should fire. The inner timestamp is in event-time and represents the
+   * time the event is purported to have taken place in the simulation.
+   */
+  public NextEvent nextEvent() {
+    if (wallclockBaseTime < 0) {
+      wallclockBaseTime = System.currentTimeMillis();
+    }
+    // When, in event time, we should generate the event. Monotonic.
+    long eventTimestamp =
+        config.timestampAndInterEventDelayUsForEvent(config.nextEventNumber(numEvents)).getKey();
+    // When, in event time, the event should say it was generated. Depending on outOfOrderGroupSize
+    // may have local jitter.
+    long adjustedEventTimestamp =
+        config.timestampAndInterEventDelayUsForEvent(config.nextAdjustedEventNumber(numEvents))
+            .getKey();
+    // The minimum of this and all future adjusted event timestamps. Accounts for jitter in
+    // the event timestamp.
+    long watermark =
+        config.timestampAndInterEventDelayUsForEvent(config.nextEventNumberForWatermark(numEvents))
+            .getKey();
+    // When, in wallclock time, we should emit the event.
+    long wallclockTimestamp = wallclockBaseTime + (eventTimestamp - getCurrentConfig().baseTime);
+
+    // Seed the random number generator with the next 'event id'.
+    Random random = new Random(getNextEventId());
+    long rem = getNextEventId() % GeneratorConfig.PROPORTION_DENOMINATOR;
+
+    Event event;
+    if (rem < GeneratorConfig.PERSON_PROPORTION) {
+      event = new Event(nextPerson(random, adjustedEventTimestamp));
+    } else if (rem < GeneratorConfig.PERSON_PROPORTION + GeneratorConfig.AUCTION_PROPORTION) {
+      event = new Event(nextAuction(random, adjustedEventTimestamp));
+    } else {
+      event = new Event(nextBid(random, adjustedEventTimestamp));
+    }
+
+    numEvents++;
+    return new NextEvent(wallclockTimestamp, adjustedEventTimestamp, event, watermark);
+  }
+
+  @Override
+  public TimestampedValue<Event> next() {
+    NextEvent next = nextEvent();
+    return TimestampedValue.of(next.event, new Instant(next.eventTimestamp));
+  }
+
+  @Override
+  public void remove() {
+    throw new UnsupportedOperationException();
+  }
+
+  /**
+   * Return how many microseconds till we emit the next event.
+   */
+  public long currentInterEventDelayUs() {
+    return config.timestampAndInterEventDelayUsForEvent(config.nextEventNumber(numEvents))
+        .getValue();
+  }
+
+  /**
+   * Return an estimate of fraction of output consumed.
+   */
+  public double getFractionConsumed() {
+    return (double) numEvents / config.maxEvents;
+  }
+
+  @Override
+  public String toString() {
+    return String.format("Generator{config:%s; numEvents:%d; wallclockBaseTime:%d}", config,
+        numEvents, wallclockBaseTime);
+  }
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/f4333df7/sdks/java/nexmark/src/main/java/org/apache/beam/sdk/nexmark/sources/GeneratorConfig.java
----------------------------------------------------------------------
diff --git a/sdks/java/nexmark/src/main/java/org/apache/beam/sdk/nexmark/sources/GeneratorConfig.java b/sdks/java/nexmark/src/main/java/org/apache/beam/sdk/nexmark/sources/GeneratorConfig.java
new file mode 100644
index 0000000..42183c6
--- /dev/null
+++ b/sdks/java/nexmark/src/main/java/org/apache/beam/sdk/nexmark/sources/GeneratorConfig.java
@@ -0,0 +1,298 @@
+/*
+ * 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.beam.sdk.nexmark.sources;
+
+import java.io.Serializable;
+import java.util.ArrayList;
+import java.util.List;
+import org.apache.beam.sdk.nexmark.NexmarkConfiguration;
+import org.apache.beam.sdk.nexmark.model.Event;
+import org.apache.beam.sdk.values.KV;
+
+/**
+ * Parameters controlling how {@link Generator} synthesizes {@link Event} elements.
+ */
+public class GeneratorConfig implements Serializable {
+
+  /**
+   * We start the ids at specific values to help ensure the queries find a match even on
+   * small synthesized dataset sizes.
+   */
+  public static final long FIRST_AUCTION_ID = 1000L;
+  public static final long FIRST_PERSON_ID = 1000L;
+  public static final long FIRST_CATEGORY_ID = 10L;
+
+  /**
+   * Proportions of people/auctions/bids to synthesize.
+   */
+  public static final int PERSON_PROPORTION = 1;
+  public static final int AUCTION_PROPORTION = 3;
+  private static final int BID_PROPORTION = 46;
+  public static final int PROPORTION_DENOMINATOR =
+      PERSON_PROPORTION + AUCTION_PROPORTION + BID_PROPORTION;
+
+  /**
+   * Environment options.
+   */
+  public final NexmarkConfiguration configuration;
+
+  /**
+   * Delay between events, in microseconds. If the array has more than one entry then
+   * the rate is changed every {@link #stepLengthSec}, and wraps around.
+   */
+  private final long[] interEventDelayUs;
+
+  /**
+   * Delay before changing the current inter-event delay.
+   */
+  private final long stepLengthSec;
+
+  /**
+   * Time for first event (ms since epoch).
+   */
+  public final long baseTime;
+
+  /**
+   * Event id of first event to be generated. Event ids are unique over all generators, and
+   * are used as a seed to generate each event's data.
+   */
+  public final long firstEventId;
+
+  /**
+   * Maximum number of events to generate.
+   */
+  public final long maxEvents;
+
+  /**
+   * First event number. Generators running in parallel time may share the same event number,
+   * and the event number is used to determine the event timestamp.
+   */
+  public final long firstEventNumber;
+
+  /**
+   * True period of epoch in milliseconds. Derived from above.
+   * (Ie time to run through cycle for all interEventDelayUs entries).
+   */
+  private final long epochPeriodMs;
+
+  /**
+   * Number of events per epoch. Derived from above.
+   * (Ie number of events to run through cycle for all interEventDelayUs entries).
+   */
+  private final long eventsPerEpoch;
+
+  public GeneratorConfig(
+      NexmarkConfiguration configuration, long baseTime, long firstEventId,
+      long maxEventsOrZero, long firstEventNumber) {
+    this.configuration = configuration;
+    this.interEventDelayUs = configuration.rateShape.interEventDelayUs(
+        configuration.firstEventRate, configuration.nextEventRate,
+        configuration.rateUnit, configuration.numEventGenerators);
+    this.stepLengthSec = configuration.rateShape.stepLengthSec(configuration.ratePeriodSec);
+    this.baseTime = baseTime;
+    this.firstEventId = firstEventId;
+    if (maxEventsOrZero == 0) {
+      // Scale maximum down to avoid overflow in getEstimatedSizeBytes.
+      this.maxEvents =
+          Long.MAX_VALUE / (PROPORTION_DENOMINATOR
+                            * Math.max(
+              Math.max(configuration.avgPersonByteSize, configuration.avgAuctionByteSize),
+              configuration.avgBidByteSize));
+    } else {
+      this.maxEvents = maxEventsOrZero;
+    }
+    this.firstEventNumber = firstEventNumber;
+
+    long eventsPerEpoch = 0;
+    long epochPeriodMs = 0;
+    if (interEventDelayUs.length > 1) {
+      for (long interEventDelayU : interEventDelayUs) {
+        long numEventsForThisCycle = (stepLengthSec * 1_000_000L) / interEventDelayU;
+        eventsPerEpoch += numEventsForThisCycle;
+        epochPeriodMs += (numEventsForThisCycle * interEventDelayU) / 1000L;
+      }
+    }
+    this.eventsPerEpoch = eventsPerEpoch;
+    this.epochPeriodMs = epochPeriodMs;
+  }
+
+  /**
+   * Return a copy of this config.
+   */
+  public GeneratorConfig copy() {
+    GeneratorConfig result;
+      result = new GeneratorConfig(configuration, baseTime, firstEventId,
+          maxEvents, firstEventNumber);
+    return result;
+  }
+
+  /**
+   * Split this config into {@code n} sub-configs with roughly equal number of
+   * possible events, but distinct value spaces. The generators will run on parallel timelines.
+   * This config should no longer be used.
+   */
+  public List<GeneratorConfig> split(int n) {
+    List<GeneratorConfig> results = new ArrayList<>();
+    if (n == 1) {
+      // No split required.
+      results.add(this);
+    } else {
+      long subMaxEvents = maxEvents / n;
+      long subFirstEventId = firstEventId;
+      for (int i = 0; i < n; i++) {
+        if (i == n - 1) {
+          // Don't loose any events to round-down.
+          subMaxEvents = maxEvents - subMaxEvents * (n - 1);
+        }
+        results.add(copyWith(subFirstEventId, subMaxEvents, firstEventNumber));
+        subFirstEventId += subMaxEvents;
+      }
+    }
+    return results;
+  }
+
+  /**
+   * Return copy of this config except with given parameters.
+   */
+  public GeneratorConfig copyWith(long firstEventId, long maxEvents, long firstEventNumber) {
+    return new GeneratorConfig(configuration, baseTime, firstEventId, maxEvents, firstEventNumber);
+  }
+
+  /**
+   * Return an estimate of the bytes needed by {@code numEvents}.
+   */
+  public long estimatedBytesForEvents(long numEvents) {
+    long numPersons =
+        (numEvents * GeneratorConfig.PERSON_PROPORTION) / GeneratorConfig.PROPORTION_DENOMINATOR;
+    long numAuctions = (numEvents * AUCTION_PROPORTION) / PROPORTION_DENOMINATOR;
+    long numBids = (numEvents * BID_PROPORTION) / PROPORTION_DENOMINATOR;
+    return numPersons * configuration.avgPersonByteSize
+           + numAuctions * configuration.avgAuctionByteSize
+           + numBids * configuration.avgBidByteSize;
+  }
+
+  /**
+   * Return an estimate of the byte-size of all events a generator for this config would yield.
+   */
+  public long getEstimatedSizeBytes() {
+    return estimatedBytesForEvents(maxEvents);
+  }
+
+  /**
+   * Return the first 'event id' which could be generated from this config. Though events don't
+   * have ids we can simulate them to help bookkeeping.
+   */
+  public long getStartEventId() {
+    return firstEventId + firstEventNumber;
+  }
+
+  /**
+   * Return one past the last 'event id' which could be generated from this config.
+   */
+  public long getStopEventId() {
+    return firstEventId + firstEventNumber + maxEvents;
+  }
+
+  /**
+   * Return the next event number for a generator which has so far emitted {@code numEvents}.
+   */
+  public long nextEventNumber(long numEvents) {
+    return firstEventNumber + numEvents;
+  }
+
+  /**
+   * Return the next event number for a generator which has so far emitted {@code numEvents},
+   * but adjusted to account for {@code outOfOrderGroupSize}.
+   */
+  public long nextAdjustedEventNumber(long numEvents) {
+    long n = configuration.outOfOrderGroupSize;
+    long eventNumber = nextEventNumber(numEvents);
+    long base = (eventNumber / n) * n;
+    long offset = (eventNumber * 953) % n;
+    return base + offset;
+  }
+
+  /**
+   * Return the event number who's event time will be a suitable watermark for
+   * a generator which has so far emitted {@code numEvents}.
+   */
+  public long nextEventNumberForWatermark(long numEvents) {
+    long n = configuration.outOfOrderGroupSize;
+    long eventNumber = nextEventNumber(numEvents);
+    return (eventNumber / n) * n;
+  }
+
+  /**
+   * What timestamp should the event with {@code eventNumber} have for this generator? And
+   * what inter-event delay (in microseconds) is current?
+   */
+  public KV<Long, Long> timestampAndInterEventDelayUsForEvent(long eventNumber) {
+    if (interEventDelayUs.length == 1) {
+      long timestamp = baseTime + (eventNumber * interEventDelayUs[0]) / 1000L;
+      return KV.of(timestamp, interEventDelayUs[0]);
+    }
+
+    long epoch = eventNumber / eventsPerEpoch;
+    long n = eventNumber % eventsPerEpoch;
+    long offsetInEpochMs = 0;
+    for (long interEventDelayU : interEventDelayUs) {
+      long numEventsForThisCycle = (stepLengthSec * 1_000_000L) / interEventDelayU;
+      if (n < numEventsForThisCycle) {
+        long offsetInCycleUs = n * interEventDelayU;
+        long timestamp =
+            baseTime + epoch * epochPeriodMs + offsetInEpochMs + (offsetInCycleUs / 1000L);
+        return KV.of(timestamp, interEventDelayU);
+      }
+      n -= numEventsForThisCycle;
+      offsetInEpochMs += (numEventsForThisCycle * interEventDelayU) / 1000L;
+    }
+    throw new RuntimeException("internal eventsPerEpoch incorrect"); // can't reach
+  }
+
+  @Override
+  public String toString() {
+    StringBuilder sb = new StringBuilder();
+    sb.append("GeneratorConfig");
+    sb.append("{configuration:");
+    sb.append(configuration.toString());
+    sb.append(";interEventDelayUs=[");
+    for (int i = 0; i < interEventDelayUs.length; i++) {
+      if (i > 0) {
+        sb.append(",");
+      }
+      sb.append(interEventDelayUs[i]);
+    }
+    sb.append("]");
+    sb.append(";stepLengthSec:");
+    sb.append(stepLengthSec);
+    sb.append(";baseTime:");
+    sb.append(baseTime);
+    sb.append(";firstEventId:");
+    sb.append(firstEventId);
+    sb.append(";maxEvents:");
+    sb.append(maxEvents);
+    sb.append(";firstEventNumber:");
+    sb.append(firstEventNumber);
+    sb.append(";epochPeriodMs:");
+    sb.append(epochPeriodMs);
+    sb.append(";eventsPerEpoch:");
+    sb.append(eventsPerEpoch);
+    sb.append("}");
+    return sb.toString();
+  }
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/f4333df7/sdks/java/nexmark/src/main/java/org/apache/beam/sdk/nexmark/sources/UnboundedEventSource.java
----------------------------------------------------------------------
diff --git a/sdks/java/nexmark/src/main/java/org/apache/beam/sdk/nexmark/sources/UnboundedEventSource.java b/sdks/java/nexmark/src/main/java/org/apache/beam/sdk/nexmark/sources/UnboundedEventSource.java
new file mode 100644
index 0000000..8f5575c
--- /dev/null
+++ b/sdks/java/nexmark/src/main/java/org/apache/beam/sdk/nexmark/sources/UnboundedEventSource.java
@@ -0,0 +1,329 @@
+/*
+ * 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.beam.sdk.nexmark.sources;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.NoSuchElementException;
+import java.util.PriorityQueue;
+import java.util.Queue;
+import java.util.concurrent.ThreadLocalRandom;
+import javax.annotation.Nullable;
+import org.apache.beam.sdk.coders.Coder;
+import org.apache.beam.sdk.io.UnboundedSource;
+import org.apache.beam.sdk.nexmark.NexmarkUtils;
+import org.apache.beam.sdk.nexmark.model.Event;
+import org.apache.beam.sdk.options.PipelineOptions;
+import org.apache.beam.sdk.transforms.windowing.BoundedWindow;
+import org.apache.beam.sdk.values.TimestampedValue;
+import org.joda.time.Duration;
+import org.joda.time.Instant;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * A custom, unbounded source of event records.
+ *
+ * <p>If {@code isRateLimited} is true, events become available for return from the reader such
+ * that the overall rate respect the {@code interEventDelayUs} period if possible. Otherwise,
+ * events are returned every time the system asks for one.
+ */
+public class UnboundedEventSource extends UnboundedSource<Event, Generator.Checkpoint> {
+  private static final Duration BACKLOG_PERIOD = Duration.standardSeconds(30);
+  private static final Logger LOG = LoggerFactory.getLogger(UnboundedEventSource.class);
+
+  /** Configuration for generator to use when reading synthetic events. May be split. */
+  private final GeneratorConfig config;
+
+  /** How many unbounded sources to create. */
+  private final int numEventGenerators;
+
+  /** How many seconds to hold back the watermark. */
+  private final long watermarkHoldbackSec;
+
+  /** Are we rate limiting the events? */
+  private final boolean isRateLimited;
+
+  public UnboundedEventSource(GeneratorConfig config, int numEventGenerators,
+      long watermarkHoldbackSec, boolean isRateLimited) {
+    this.config = config;
+    this.numEventGenerators = numEventGenerators;
+    this.watermarkHoldbackSec = watermarkHoldbackSec;
+    this.isRateLimited = isRateLimited;
+  }
+
+  /** A reader to pull events from the generator. */
+  private class EventReader extends UnboundedReader<Event> {
+    /** Generator we are reading from. */
+    private final Generator generator;
+
+    /**
+     * Current watermark (ms since epoch). Initially set to beginning of time.
+     * Then updated to be the time of the next generated event.
+     * Then, once all events have been generated, set to the end of time.
+     */
+    private long watermark;
+
+    /**
+     * Current backlog (ms), as delay between timestamp of last returned event and the timestamp
+     * we should be up to according to wall-clock time. Used only for logging.
+     */
+    private long backlogDurationMs;
+
+    /**
+     * Current backlog, as estimated number of event bytes we are behind, or null if
+     * unknown. Reported to callers.
+     */
+    @Nullable
+    private Long backlogBytes;
+
+    /**
+     * Wallclock time (ms since epoch) we last reported the backlog, or -1 if never reported.
+     */
+    private long lastReportedBacklogWallclock;
+
+    /**
+     * Event time (ms since epoch) of pending event at last reported backlog, or -1 if never
+     * calculated.
+     */
+    private long timestampAtLastReportedBacklogMs;
+
+    /** Next event to make 'current' when wallclock time has advanced sufficiently. */
+    @Nullable
+    private TimestampedValue<Event> pendingEvent;
+
+    /** Wallclock time when {@link #pendingEvent} is due, or -1 if no pending event. */
+    private long pendingEventWallclockTime;
+
+    /** Current event to return from getCurrent. */
+    @Nullable
+    private TimestampedValue<Event> currentEvent;
+
+    /** Events which have been held back so as to force them to be late. */
+    private final Queue<Generator.NextEvent> heldBackEvents = new PriorityQueue<>();
+
+    public EventReader(Generator generator) {
+      this.generator = generator;
+      watermark = NexmarkUtils.BEGINNING_OF_TIME.getMillis();
+      lastReportedBacklogWallclock = -1;
+      pendingEventWallclockTime = -1;
+      timestampAtLastReportedBacklogMs = -1;
+    }
+
+    public EventReader(GeneratorConfig config) {
+      this(new Generator(config));
+    }
+
+    @Override
+    public boolean start() {
+      LOG.trace("starting unbounded generator {}", generator);
+      return advance();
+    }
+
+
+    @Override
+    public boolean advance() {
+      long now = System.currentTimeMillis();
+
+      while (pendingEvent == null) {
+        if (!generator.hasNext() && heldBackEvents.isEmpty()) {
+          // No more events, EVER.
+          if (isRateLimited) {
+            updateBacklog(System.currentTimeMillis(), 0);
+          }
+          if (watermark < BoundedWindow.TIMESTAMP_MAX_VALUE.getMillis()) {
+            watermark = BoundedWindow.TIMESTAMP_MAX_VALUE.getMillis();
+            LOG.trace("stopped unbounded generator {}", generator);
+          }
+          return false;
+        }
+
+        Generator.NextEvent next = heldBackEvents.peek();
+        if (next != null && next.wallclockTimestamp <= now) {
+          // Time to use the held-back event.
+          heldBackEvents.poll();
+          LOG.debug("replaying held-back event {}ms behind watermark",
+                             watermark - next.eventTimestamp);
+        } else if (generator.hasNext()) {
+          next = generator.nextEvent();
+          if (isRateLimited && config.configuration.probDelayedEvent > 0.0
+              && config.configuration.occasionalDelaySec > 0
+              && ThreadLocalRandom.current().nextDouble() < config.configuration.probDelayedEvent) {
+            // We'll hold back this event and go around again.
+            long delayMs =
+                ThreadLocalRandom.current().nextLong(config.configuration.occasionalDelaySec * 1000)
+                + 1L;
+            LOG.debug("delaying event by {}ms", delayMs);
+            heldBackEvents.add(next.withDelay(delayMs));
+            continue;
+          }
+        } else {
+          // Waiting for held-back event to fire.
+          if (isRateLimited) {
+            updateBacklog(now, 0);
+          }
+          return false;
+        }
+
+        pendingEventWallclockTime = next.wallclockTimestamp;
+        pendingEvent = TimestampedValue.of(next.event, new Instant(next.eventTimestamp));
+        long newWatermark =
+            next.watermark - Duration.standardSeconds(watermarkHoldbackSec).getMillis();
+        if (newWatermark > watermark) {
+          watermark = newWatermark;
+        }
+      }
+
+      if (isRateLimited) {
+        if (pendingEventWallclockTime > now) {
+          // We want this event to fire in the future. Try again later.
+          updateBacklog(now, 0);
+          return false;
+        }
+        updateBacklog(now, now - pendingEventWallclockTime);
+      }
+
+      // This event is ready to fire.
+      currentEvent = pendingEvent;
+      pendingEvent = null;
+      return true;
+    }
+
+    private void updateBacklog(long now, long newBacklogDurationMs) {
+      backlogDurationMs = newBacklogDurationMs;
+      long interEventDelayUs = generator.currentInterEventDelayUs();
+      if (interEventDelayUs != 0) {
+        long backlogEvents = (backlogDurationMs * 1000 + interEventDelayUs - 1) / interEventDelayUs;
+        backlogBytes = generator.getCurrentConfig().estimatedBytesForEvents(backlogEvents);
+      }
+      if (lastReportedBacklogWallclock < 0
+          || now - lastReportedBacklogWallclock > BACKLOG_PERIOD.getMillis()) {
+        double timeDialation = Double.NaN;
+        if (pendingEvent != null
+            && lastReportedBacklogWallclock >= 0
+            && timestampAtLastReportedBacklogMs >= 0) {
+          long wallclockProgressionMs = now - lastReportedBacklogWallclock;
+          long eventTimeProgressionMs =
+              pendingEvent.getTimestamp().getMillis() - timestampAtLastReportedBacklogMs;
+          timeDialation = (double) eventTimeProgressionMs / (double) wallclockProgressionMs;
+        }
+        LOG.debug(
+            "unbounded generator backlog now {}ms ({} bytes) at {}us interEventDelay "
+            + "with {} time dilation",
+            backlogDurationMs, backlogBytes, interEventDelayUs, timeDialation);
+        lastReportedBacklogWallclock = now;
+        if (pendingEvent != null) {
+          timestampAtLastReportedBacklogMs = pendingEvent.getTimestamp().getMillis();
+        }
+      }
+    }
+
+    @Override
+    public Event getCurrent() {
+      if (currentEvent == null) {
+        throw new NoSuchElementException();
+      }
+      return currentEvent.getValue();
+    }
+
+    @Override
+    public Instant getCurrentTimestamp() {
+      if (currentEvent == null) {
+        throw new NoSuchElementException();
+      }
+      return currentEvent.getTimestamp();
+    }
+
+    @Override
+    public void close() {
+      // Nothing to close.
+    }
+
+    @Override
+    public UnboundedEventSource getCurrentSource() {
+      return UnboundedEventSource.this;
+    }
+
+    @Override
+    public Instant getWatermark() {
+      return new Instant(watermark);
+    }
+
+    @Override
+    public Generator.Checkpoint getCheckpointMark() {
+      return generator.toCheckpoint();
+    }
+
+    @Override
+    public long getSplitBacklogBytes() {
+      return backlogBytes == null ? BACKLOG_UNKNOWN : backlogBytes;
+    }
+
+    @Override
+    public String toString() {
+      return String.format("EventReader(%d, %d, %d)",
+          generator.getCurrentConfig().getStartEventId(), generator.getNextEventId(),
+          generator.getCurrentConfig().getStopEventId());
+    }
+  }
+
+  @Override
+  public Coder<Generator.Checkpoint> getCheckpointMarkCoder() {
+    return Generator.Checkpoint.CODER_INSTANCE;
+  }
+
+  @Override
+  public List<UnboundedEventSource> split(
+      int desiredNumSplits, PipelineOptions options) {
+    LOG.trace("splitting unbounded source into {} sub-sources", numEventGenerators);
+    List<UnboundedEventSource> results = new ArrayList<>();
+    // Ignore desiredNumSplits and use numEventGenerators instead.
+    for (GeneratorConfig subConfig : config.split(numEventGenerators)) {
+      results.add(new UnboundedEventSource(subConfig, 1, watermarkHoldbackSec, isRateLimited));
+    }
+    return results;
+  }
+
+  @Override
+  public EventReader createReader(
+      PipelineOptions options, @Nullable Generator.Checkpoint checkpoint) {
+    if (checkpoint == null) {
+      LOG.trace("creating initial unbounded reader for {}", config);
+      return new EventReader(config);
+    } else {
+      LOG.trace("resuming unbounded reader from {}", checkpoint);
+      return new EventReader(checkpoint.toGenerator(config));
+    }
+  }
+
+  @Override
+  public void validate() {
+    // Nothing to validate.
+  }
+
+  @Override
+  public Coder<Event> getDefaultOutputCoder() {
+    return Event.CODER;
+  }
+
+  @Override
+  public String toString() {
+    return String.format(
+        "UnboundedEventSource(%d, %d)", config.getStartEventId(), config.getStopEventId());
+  }
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/f4333df7/sdks/java/nexmark/src/main/java/org/apache/beam/sdk/nexmark/sources/package-info.java
----------------------------------------------------------------------
diff --git a/sdks/java/nexmark/src/main/java/org/apache/beam/sdk/nexmark/sources/package-info.java b/sdks/java/nexmark/src/main/java/org/apache/beam/sdk/nexmark/sources/package-info.java
new file mode 100644
index 0000000..266af10
--- /dev/null
+++ b/sdks/java/nexmark/src/main/java/org/apache/beam/sdk/nexmark/sources/package-info.java
@@ -0,0 +1,22 @@
+/*
+ * 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.
+ */
+
+/**
+ * Nexmark Synthetic Sources.
+ */
+package org.apache.beam.sdk.nexmark.sources;

http://git-wip-us.apache.org/repos/asf/beam/blob/f4333df7/sdks/java/nexmark/src/main/resources/log4j.properties
----------------------------------------------------------------------
diff --git a/sdks/java/nexmark/src/main/resources/log4j.properties b/sdks/java/nexmark/src/main/resources/log4j.properties
new file mode 100644
index 0000000..7dd57b5
--- /dev/null
+++ b/sdks/java/nexmark/src/main/resources/log4j.properties
@@ -0,0 +1,55 @@
+#
+# 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.
+#
+
+# Set everything to be logged to the console
+log4j.rootCategory=DEBUG, console
+log4j.appender.console=org.apache.log4j.ConsoleAppender
+log4j.appender.console.target=System.err
+log4j.appender.console.layout=org.apache.log4j.PatternLayout
+log4j.appender.console.layout.ConversionPattern=%d{yy/MM/dd HH:mm:ss} %p %c: %m%n
+
+# General Beam loggers
+log4j.logger.org.apache.beam.runners.direct=WARN
+log4j.logger.org.apache.beam.sdk=WARN
+
+# Nexmark specific
+log4j.logger.org.apache.beam.integration.nexmark=WARN
+
+# Settings to quiet third party logs that are too verbose
+log4j.logger.org.spark_project.jetty=WARN
+log4j.logger.org.spark_project.jetty.util.component.AbstractLifeCycle=ERROR
+
+# Setting to quiet spark logs, Beam logs should standout
+log4j.logger.org.apache.beam.runners.spark=INFO
+log4j.logger.org.apache.spark=WARN
+log4j.logger.org.spark-project=WARN
+log4j.logger.io.netty=INFO
+
+# Settings to quiet flink logs
+log4j.logger.org.apache.flink=WARN
+
+# Settings to quiet apex logs
+log4j.logger.org.apache.beam.runners.apex=INFO
+log4j.logger.com.datatorrent=ERROR
+log4j.logger.org.apache.hadoop.metrics2=WARN
+log4j.logger.org.apache.commons=WARN
+log4j.logger.org.apache.hadoop.security=WARN
+log4j.logger.org.apache.hadoop.util=WARN
+
+# SPARK-9183: Settings to avoid annoying messages when looking up nonexistent UDFs in SparkSQL with Hive support
+log4j.logger.org.apache.hadoop.hive.metastore.RetryingHMSHandler=FATAL
+log4j.logger.org.apache.hadoop.hive.ql.exec.FunctionRegistry=ERROR

http://git-wip-us.apache.org/repos/asf/beam/blob/f4333df7/sdks/java/nexmark/src/test/java/org/apache/beam/sdk/nexmark/queries/QueryTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/nexmark/src/test/java/org/apache/beam/sdk/nexmark/queries/QueryTest.java b/sdks/java/nexmark/src/test/java/org/apache/beam/sdk/nexmark/queries/QueryTest.java
new file mode 100644
index 0000000..d8ac057
--- /dev/null
+++ b/sdks/java/nexmark/src/test/java/org/apache/beam/sdk/nexmark/queries/QueryTest.java
@@ -0,0 +1,185 @@
+/*
+ * 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.beam.sdk.nexmark.queries;
+
+import org.apache.beam.sdk.PipelineResult;
+import org.apache.beam.sdk.nexmark.NexmarkConfiguration;
+import org.apache.beam.sdk.nexmark.NexmarkUtils;
+import org.apache.beam.sdk.nexmark.model.KnownSize;
+import org.apache.beam.sdk.testing.NeedsRunner;
+import org.apache.beam.sdk.testing.PAssert;
+import org.apache.beam.sdk.testing.TestPipeline;
+import org.apache.beam.sdk.testing.UsesStatefulParDo;
+import org.apache.beam.sdk.testing.UsesTimersInParDo;
+import org.apache.beam.sdk.values.PCollection;
+import org.apache.beam.sdk.values.TimestampedValue;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+import org.junit.runner.RunWith;
+import org.junit.runners.JUnit4;
+
+/** Test the various NEXMark queries yield results coherent with their models. */
+@RunWith(JUnit4.class)
+public class QueryTest {
+  private static final NexmarkConfiguration CONFIG = NexmarkConfiguration.DEFAULT.copy();
+
+  static {
+    // careful, results of tests are linked to numEventGenerators because of timestamp generation
+    CONFIG.numEventGenerators = 1;
+    CONFIG.numEvents = 1000;
+  }
+
+  @Rule public TestPipeline p = TestPipeline.create();
+
+  /** Test {@code query} matches {@code model}. */
+  private void queryMatchesModel(
+      String name, NexmarkQuery query, NexmarkQueryModel model, boolean streamingMode) {
+    NexmarkUtils.setupPipeline(NexmarkUtils.CoderStrategy.HAND, p);
+    PCollection<TimestampedValue<KnownSize>> results;
+    if (streamingMode) {
+      results =
+          p.apply(name + ".ReadUnBounded", NexmarkUtils.streamEventsSource(CONFIG)).apply(query);
+    } else {
+      results = p.apply(name + ".ReadBounded", NexmarkUtils.batchEventsSource(CONFIG)).apply(query);
+    }
+    PAssert.that(results).satisfies(model.assertionFor());
+    PipelineResult result = p.run();
+    result.waitUntilFinish();
+  }
+
+  @Test
+  @Category(NeedsRunner.class)
+  public void query0MatchesModelBatch() {
+    queryMatchesModel("Query0TestBatch", new Query0(CONFIG), new Query0Model(CONFIG), false);
+  }
+
+  @Test
+  @Category(NeedsRunner.class)
+  public void query0MatchesModelStreaming() {
+    queryMatchesModel("Query0TestStreaming", new Query0(CONFIG), new Query0Model(CONFIG), true);
+  }
+
+  @Test
+  @Category(NeedsRunner.class)
+  public void query1MatchesModelBatch() {
+    queryMatchesModel("Query1TestBatch", new Query1(CONFIG), new Query1Model(CONFIG), false);
+  }
+
+  @Test
+  @Category(NeedsRunner.class)
+  public void query1MatchesModelStreaming() {
+    queryMatchesModel("Query1TestStreaming", new Query1(CONFIG), new Query1Model(CONFIG), true);
+  }
+
+  @Test
+  @Category(NeedsRunner.class)
+  public void query2MatchesModelBatch() {
+    queryMatchesModel("Query2TestBatch", new Query2(CONFIG), new Query2Model(CONFIG), false);
+  }
+
+  @Test
+  @Category(NeedsRunner.class)
+  public void query2MatchesModelStreaming() {
+    queryMatchesModel("Query2TestStreaming", new Query2(CONFIG), new Query2Model(CONFIG), true);
+  }
+
+  @Test
+  @Category({NeedsRunner.class, UsesStatefulParDo.class, UsesTimersInParDo.class})
+  public void query3MatchesModelBatch() {
+    queryMatchesModel("Query3TestBatch", new Query3(CONFIG), new Query3Model(CONFIG), false);
+  }
+
+  @Test
+  @Category({NeedsRunner.class, UsesStatefulParDo.class, UsesTimersInParDo.class})
+  public void query3MatchesModelStreaming() {
+    queryMatchesModel("Query3TestStreaming", new Query3(CONFIG), new Query3Model(CONFIG), true);
+  }
+
+  @Test
+  @Category(NeedsRunner.class)
+  public void query4MatchesModelBatch() {
+    queryMatchesModel("Query4TestBatch", new Query4(CONFIG), new Query4Model(CONFIG), false);
+  }
+
+  @Test
+  @Category(NeedsRunner.class)
+  public void query4MatchesModelStreaming() {
+    queryMatchesModel("Query4TestStreaming", new Query4(CONFIG), new Query4Model(CONFIG), true);
+  }
+
+  @Test
+  @Category(NeedsRunner.class)
+  public void query5MatchesModelBatch() {
+    queryMatchesModel("Query5TestBatch", new Query5(CONFIG), new Query5Model(CONFIG), false);
+  }
+
+  @Test
+  @Category(NeedsRunner.class)
+  public void query5MatchesModelStreaming() {
+    queryMatchesModel("Query5TestStreaming", new Query5(CONFIG), new Query5Model(CONFIG), true);
+  }
+
+  @Test
+  @Category(NeedsRunner.class)
+  public void query6MatchesModelBatch() {
+    queryMatchesModel("Query6TestBatch", new Query6(CONFIG), new Query6Model(CONFIG), false);
+  }
+
+  @Test
+  @Category(NeedsRunner.class)
+  public void query6MatchesModelStreaming() {
+    queryMatchesModel("Query6TestStreaming", new Query6(CONFIG), new Query6Model(CONFIG), true);
+  }
+
+  @Test
+  @Category(NeedsRunner.class)
+  public void query7MatchesModelBatch() {
+    queryMatchesModel("Query7TestBatch", new Query7(CONFIG), new Query7Model(CONFIG), false);
+  }
+
+  @Test
+  @Category(NeedsRunner.class)
+  public void query7MatchesModelStreaming() {
+    queryMatchesModel("Query7TestStreaming", new Query7(CONFIG), new Query7Model(CONFIG), true);
+  }
+
+  @Test
+  @Category(NeedsRunner.class)
+  public void query8MatchesModelBatch() {
+    queryMatchesModel("Query8TestBatch", new Query8(CONFIG), new Query8Model(CONFIG), false);
+  }
+
+  @Test
+  @Category(NeedsRunner.class)
+  public void query8MatchesModelStreaming() {
+    queryMatchesModel("Query8TestStreaming", new Query8(CONFIG), new Query8Model(CONFIG), true);
+  }
+
+  @Test
+  @Category(NeedsRunner.class)
+  public void query9MatchesModelBatch() {
+    queryMatchesModel("Query9TestBatch", new Query9(CONFIG), new Query9Model(CONFIG), false);
+  }
+
+  @Test
+  @Category(NeedsRunner.class)
+  public void query9MatchesModelStreaming() {
+    queryMatchesModel("Query9TestStreaming", new Query9(CONFIG), new Query9Model(CONFIG), true);
+  }
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/f4333df7/sdks/java/nexmark/src/test/java/org/apache/beam/sdk/nexmark/sources/BoundedEventSourceTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/nexmark/src/test/java/org/apache/beam/sdk/nexmark/sources/BoundedEventSourceTest.java b/sdks/java/nexmark/src/test/java/org/apache/beam/sdk/nexmark/sources/BoundedEventSourceTest.java
new file mode 100644
index 0000000..3590d64
--- /dev/null
+++ b/sdks/java/nexmark/src/test/java/org/apache/beam/sdk/nexmark/sources/BoundedEventSourceTest.java
@@ -0,0 +1,70 @@
+/*
+ * 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.beam.sdk.nexmark.sources;
+
+import org.apache.beam.sdk.nexmark.NexmarkConfiguration;
+import org.apache.beam.sdk.nexmark.NexmarkOptions;
+import org.apache.beam.sdk.options.PipelineOptionsFactory;
+import org.apache.beam.sdk.testing.SourceTestUtils;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.JUnit4;
+
+/**
+ * Test {@link BoundedEventSource}.
+ */
+@RunWith(JUnit4.class)
+public class BoundedEventSourceTest {
+  private GeneratorConfig makeConfig(long n) {
+    return new GeneratorConfig(
+        NexmarkConfiguration.DEFAULT, System.currentTimeMillis(), 0, n, 0);
+  }
+
+  @Test
+  public void sourceAndReadersWork() throws Exception {
+    NexmarkOptions options = PipelineOptionsFactory.as(NexmarkOptions.class);
+    long n = 200L;
+    BoundedEventSource source = new BoundedEventSource(makeConfig(n), 1);
+
+    SourceTestUtils.assertUnstartedReaderReadsSameAsItsSource(
+        source.createReader(options), options);
+  }
+
+  @Test
+  public void splitAtFractionRespectsContract() throws Exception {
+    NexmarkOptions options = PipelineOptionsFactory.as(NexmarkOptions.class);
+    long n = 20L;
+    BoundedEventSource source = new BoundedEventSource(makeConfig(n), 1);
+
+    // Can't split if already consumed.
+    SourceTestUtils.assertSplitAtFractionFails(source, 10, 0.3, options);
+
+    SourceTestUtils.assertSplitAtFractionSucceedsAndConsistent(source, 5, 0.3, options);
+
+    SourceTestUtils.assertSplitAtFractionExhaustive(source, options);
+  }
+
+  @Test
+  public void splitIntoBundlesRespectsContract() throws Exception {
+    NexmarkOptions options = PipelineOptionsFactory.as(NexmarkOptions.class);
+    long n = 200L;
+    BoundedEventSource source = new BoundedEventSource(makeConfig(n), 1);
+    SourceTestUtils.assertSourcesEqualReferenceSource(
+        source, source.split(10, options), options);
+  }
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/f4333df7/sdks/java/nexmark/src/test/java/org/apache/beam/sdk/nexmark/sources/GeneratorTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/nexmark/src/test/java/org/apache/beam/sdk/nexmark/sources/GeneratorTest.java b/sdks/java/nexmark/src/test/java/org/apache/beam/sdk/nexmark/sources/GeneratorTest.java
new file mode 100644
index 0000000..9553d22
--- /dev/null
+++ b/sdks/java/nexmark/src/test/java/org/apache/beam/sdk/nexmark/sources/GeneratorTest.java
@@ -0,0 +1,110 @@
+/*
+ * 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.beam.sdk.nexmark.sources;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+
+import java.util.ArrayList;
+import java.util.Iterator;
+import java.util.List;
+import org.apache.beam.sdk.nexmark.NexmarkConfiguration;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.JUnit4;
+
+/**
+ * Test {@link Generator}.
+ */
+@RunWith(JUnit4.class)
+public class GeneratorTest {
+  private GeneratorConfig makeConfig(long n) {
+    return new GeneratorConfig(
+        NexmarkConfiguration.DEFAULT, System.currentTimeMillis(), 0, n, 0);
+  }
+
+  private <T> long consume(long n, Iterator<T> itr) {
+    for (long i = 0; i < n; i++) {
+      assertTrue(itr.hasNext());
+      itr.next();
+    }
+    return n;
+  }
+
+  private <T> long consume(Iterator<T> itr) {
+    long n = 0;
+    while (itr.hasNext()) {
+      itr.next();
+      n++;
+    }
+    return n;
+  }
+
+  @Test
+  public void splitAtFractionPreservesOverallEventCount() {
+    long n = 55729L;
+    GeneratorConfig initialConfig = makeConfig(n);
+    long expected = initialConfig.getStopEventId() - initialConfig.getStartEventId();
+
+    long actual = 0;
+
+    Generator initialGenerator = new Generator(initialConfig);
+
+    // Consume some events.
+    actual += consume(5000, initialGenerator);
+
+
+    // Split once.
+    GeneratorConfig remainConfig1 = initialGenerator.splitAtEventId(9000L);
+    Generator remainGenerator1 = new Generator(remainConfig1);
+
+    // Consume some more events.
+    actual += consume(2000, initialGenerator);
+    actual += consume(3000, remainGenerator1);
+
+    // Split again.
+    GeneratorConfig remainConfig2 = remainGenerator1.splitAtEventId(30000L);
+    Generator remainGenerator2 = new Generator(remainConfig2);
+
+    // Run to completion.
+    actual += consume(initialGenerator);
+    actual += consume(remainGenerator1);
+    actual += consume(remainGenerator2);
+
+    assertEquals(expected, actual);
+  }
+
+  @Test
+  public void splitPreservesOverallEventCount() {
+    long n = 51237L;
+    GeneratorConfig initialConfig = makeConfig(n);
+    long expected = initialConfig.getStopEventId() - initialConfig.getStartEventId();
+
+    List<Generator> generators = new ArrayList<>();
+    for (GeneratorConfig subConfig : initialConfig.split(20)) {
+      generators.add(new Generator(subConfig));
+    }
+
+    long actual = 0;
+    for (Generator generator : generators) {
+      actual += consume(generator);
+    }
+
+    assertEquals(expected, actual);
+  }
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/f4333df7/sdks/java/nexmark/src/test/java/org/apache/beam/sdk/nexmark/sources/UnboundedEventSourceTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/nexmark/src/test/java/org/apache/beam/sdk/nexmark/sources/UnboundedEventSourceTest.java b/sdks/java/nexmark/src/test/java/org/apache/beam/sdk/nexmark/sources/UnboundedEventSourceTest.java
new file mode 100644
index 0000000..3853ede
--- /dev/null
+++ b/sdks/java/nexmark/src/test/java/org/apache/beam/sdk/nexmark/sources/UnboundedEventSourceTest.java
@@ -0,0 +1,105 @@
+/*
+ * 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.beam.sdk.nexmark.sources;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+
+import java.io.IOException;
+import java.util.HashSet;
+import java.util.Random;
+import java.util.Set;
+import org.apache.beam.sdk.io.UnboundedSource.CheckpointMark;
+import org.apache.beam.sdk.io.UnboundedSource.UnboundedReader;
+import org.apache.beam.sdk.nexmark.NexmarkConfiguration;
+import org.apache.beam.sdk.nexmark.model.Event;
+import org.apache.beam.sdk.options.PipelineOptions;
+import org.apache.beam.sdk.testing.TestPipeline;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.JUnit4;
+
+/**
+ * Test UnboundedEventSource.
+ */
+@RunWith(JUnit4.class)
+public class UnboundedEventSourceTest {
+  private GeneratorConfig makeConfig(long n) {
+    return new GeneratorConfig(
+        NexmarkConfiguration.DEFAULT, System.currentTimeMillis(), 0, n, 0);
+  }
+
+  /**
+   * Helper for tracking which ids we've seen (so we can detect dups) and
+   * confirming reading events match the model events.
+   */
+  private static class EventIdChecker {
+    private final Set<Long> seenPersonIds = new HashSet<>();
+    private final Set<Long> seenAuctionIds = new HashSet<>();
+
+    public void add(Event event) {
+      if (event.newAuction != null) {
+        assertTrue(seenAuctionIds.add(event.newAuction.id));
+      } else if (event.newPerson != null) {
+        assertTrue(seenPersonIds.add(event.newPerson.id));
+      }
+    }
+
+    public void add(int n, UnboundedReader<Event> reader, Generator modelGenerator)
+        throws IOException {
+      for (int i = 0; i < n; i++) {
+        assertTrue(modelGenerator.hasNext());
+        Event modelEvent = modelGenerator.next().getValue();
+        assertTrue(reader.advance());
+        Event actualEvent = reader.getCurrent();
+        assertEquals(modelEvent.toString(), actualEvent.toString());
+        add(actualEvent);
+      }
+    }
+  }
+
+  /**
+   * Check aggressively checkpointing and resuming a reader gives us exactly the
+   * same event stream as reading directly.
+   */
+  @Test
+  public void resumeFromCheckpoint() throws IOException {
+    Random random = new Random(297);
+    int n = 47293;
+    GeneratorConfig config = makeConfig(n);
+    Generator modelGenerator = new Generator(config);
+
+    EventIdChecker checker = new EventIdChecker();
+    PipelineOptions options = TestPipeline.testingPipelineOptions();
+    UnboundedEventSource source = new UnboundedEventSource(config, 1, 0, false);
+    UnboundedReader<Event> reader = source.createReader(options, null);
+
+    while (n > 0) {
+      int m = Math.min(459 + random.nextInt(455), n);
+      System.out.printf("reading %d...%n", m);
+      checker.add(m, reader, modelGenerator);
+      n -= m;
+      System.out.printf("splitting with %d remaining...%n", n);
+      CheckpointMark checkpointMark = reader.getCheckpointMark();
+      reader = source.createReader(options, (Generator.Checkpoint) checkpointMark);
+    }
+
+    assertFalse(reader.advance());
+  }
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/f4333df7/sdks/java/pom.xml
----------------------------------------------------------------------
diff --git a/sdks/java/pom.xml b/sdks/java/pom.xml
index 3144193..f0cf8d9 100644
--- a/sdks/java/pom.xml
+++ b/sdks/java/pom.xml
@@ -43,6 +43,7 @@
     <!-- javadoc runs directly from the root parent as the last module
          in the build to be able to capture runner-specific javadoc.
     <module>javadoc</module> -->
+    <module>nexmark</module>
   </modules>
 
   <profiles>


[28/55] [abbrv] beam git commit: Move WinningBids into the queries package

Posted by ie...@apache.org.
http://git-wip-us.apache.org/repos/asf/beam/blob/a39cb800/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query3.java
----------------------------------------------------------------------
diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query3.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query3.java
index 12b16f1..71364ba 100644
--- a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query3.java
+++ b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query3.java
@@ -21,7 +21,6 @@ import java.io.IOException;
 import java.util.ArrayList;
 import java.util.List;
 import org.apache.beam.integration.nexmark.NexmarkConfiguration;
-import org.apache.beam.integration.nexmark.NexmarkQuery;
 import org.apache.beam.integration.nexmark.NexmarkUtils;
 import org.apache.beam.integration.nexmark.model.Auction;
 import org.apache.beam.integration.nexmark.model.Event;

http://git-wip-us.apache.org/repos/asf/beam/blob/a39cb800/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query3Model.java
----------------------------------------------------------------------
diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query3Model.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query3Model.java
index e4b72d2..6b98e2a 100644
--- a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query3Model.java
+++ b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query3Model.java
@@ -26,9 +26,7 @@ import java.util.HashMap;
 import java.util.Iterator;
 import java.util.Map;
 
-import org.apache.beam.integration.nexmark.AbstractSimulator;
 import org.apache.beam.integration.nexmark.NexmarkConfiguration;
-import org.apache.beam.integration.nexmark.NexmarkQueryModel;
 import org.apache.beam.integration.nexmark.NexmarkUtils;
 import org.apache.beam.integration.nexmark.model.Auction;
 import org.apache.beam.integration.nexmark.model.Event;

http://git-wip-us.apache.org/repos/asf/beam/blob/a39cb800/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query4.java
----------------------------------------------------------------------
diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query4.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query4.java
index 61991c8..9c0fe6d 100644
--- a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query4.java
+++ b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query4.java
@@ -19,9 +19,7 @@ package org.apache.beam.integration.nexmark.queries;
 
 import org.apache.beam.integration.nexmark.Monitor;
 import org.apache.beam.integration.nexmark.NexmarkConfiguration;
-import org.apache.beam.integration.nexmark.NexmarkQuery;
 import org.apache.beam.integration.nexmark.NexmarkUtils;
-import org.apache.beam.integration.nexmark.WinningBids;
 import org.apache.beam.integration.nexmark.model.Auction;
 import org.apache.beam.integration.nexmark.model.AuctionBid;
 import org.apache.beam.integration.nexmark.model.Bid;

http://git-wip-us.apache.org/repos/asf/beam/blob/a39cb800/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query4Model.java
----------------------------------------------------------------------
diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query4Model.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query4Model.java
index 9405ac8..634a58e 100644
--- a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query4Model.java
+++ b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query4Model.java
@@ -24,11 +24,8 @@ import java.util.Iterator;
 import java.util.List;
 import java.util.Map;
 import java.util.TreeMap;
-import org.apache.beam.integration.nexmark.AbstractSimulator;
 import org.apache.beam.integration.nexmark.NexmarkConfiguration;
-import org.apache.beam.integration.nexmark.NexmarkQueryModel;
 import org.apache.beam.integration.nexmark.NexmarkUtils;
-import org.apache.beam.integration.nexmark.WinningBidsSimulator;
 import org.apache.beam.integration.nexmark.model.Auction;
 import org.apache.beam.integration.nexmark.model.AuctionBid;
 import org.apache.beam.integration.nexmark.model.Bid;

http://git-wip-us.apache.org/repos/asf/beam/blob/a39cb800/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query5.java
----------------------------------------------------------------------
diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query5.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query5.java
index 34b7b50..18ce578 100644
--- a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query5.java
+++ b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query5.java
@@ -21,7 +21,6 @@ import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.List;
 import org.apache.beam.integration.nexmark.NexmarkConfiguration;
-import org.apache.beam.integration.nexmark.NexmarkQuery;
 import org.apache.beam.integration.nexmark.NexmarkUtils;
 import org.apache.beam.integration.nexmark.model.AuctionCount;
 import org.apache.beam.integration.nexmark.model.Bid;

http://git-wip-us.apache.org/repos/asf/beam/blob/a39cb800/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query5Model.java
----------------------------------------------------------------------
diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query5Model.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query5Model.java
index 6bf65dc..24d9a00 100644
--- a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query5Model.java
+++ b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query5Model.java
@@ -24,9 +24,7 @@ import java.util.Iterator;
 import java.util.List;
 import java.util.Map;
 import java.util.TreeMap;
-import org.apache.beam.integration.nexmark.AbstractSimulator;
 import org.apache.beam.integration.nexmark.NexmarkConfiguration;
-import org.apache.beam.integration.nexmark.NexmarkQueryModel;
 import org.apache.beam.integration.nexmark.NexmarkUtils;
 import org.apache.beam.integration.nexmark.model.AuctionCount;
 import org.apache.beam.integration.nexmark.model.Bid;

http://git-wip-us.apache.org/repos/asf/beam/blob/a39cb800/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query6.java
----------------------------------------------------------------------
diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query6.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query6.java
index 2a5ab702..65789ab 100644
--- a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query6.java
+++ b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query6.java
@@ -22,9 +22,7 @@ import java.util.ArrayList;
 import java.util.Collections;
 import java.util.List;
 import org.apache.beam.integration.nexmark.NexmarkConfiguration;
-import org.apache.beam.integration.nexmark.NexmarkQuery;
 import org.apache.beam.integration.nexmark.NexmarkUtils;
-import org.apache.beam.integration.nexmark.WinningBids;
 import org.apache.beam.integration.nexmark.model.Auction;
 import org.apache.beam.integration.nexmark.model.AuctionBid;
 import org.apache.beam.integration.nexmark.model.Bid;

http://git-wip-us.apache.org/repos/asf/beam/blob/a39cb800/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query6Model.java
----------------------------------------------------------------------
diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query6Model.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query6Model.java
index 4325337..0691714 100644
--- a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query6Model.java
+++ b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query6Model.java
@@ -22,11 +22,8 @@ import java.util.Collection;
 import java.util.Iterator;
 import java.util.Map;
 import java.util.TreeMap;
-import org.apache.beam.integration.nexmark.AbstractSimulator;
 import org.apache.beam.integration.nexmark.NexmarkConfiguration;
-import org.apache.beam.integration.nexmark.NexmarkQueryModel;
 import org.apache.beam.integration.nexmark.NexmarkUtils;
-import org.apache.beam.integration.nexmark.WinningBidsSimulator;
 import org.apache.beam.integration.nexmark.model.Auction;
 import org.apache.beam.integration.nexmark.model.AuctionBid;
 import org.apache.beam.integration.nexmark.model.Bid;

http://git-wip-us.apache.org/repos/asf/beam/blob/a39cb800/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query7.java
----------------------------------------------------------------------
diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query7.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query7.java
index f3d1ba4..2a94ca9 100644
--- a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query7.java
+++ b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query7.java
@@ -18,7 +18,6 @@
 package org.apache.beam.integration.nexmark.queries;
 
 import org.apache.beam.integration.nexmark.NexmarkConfiguration;
-import org.apache.beam.integration.nexmark.NexmarkQuery;
 import org.apache.beam.integration.nexmark.NexmarkUtils;
 import org.apache.beam.integration.nexmark.model.Bid;
 import org.apache.beam.integration.nexmark.model.Event;

http://git-wip-us.apache.org/repos/asf/beam/blob/a39cb800/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query7Model.java
----------------------------------------------------------------------
diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query7Model.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query7Model.java
index 0a80e59..5c039f9 100644
--- a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query7Model.java
+++ b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query7Model.java
@@ -23,9 +23,7 @@ import java.util.Collection;
 import java.util.Iterator;
 import java.util.List;
 
-import org.apache.beam.integration.nexmark.AbstractSimulator;
 import org.apache.beam.integration.nexmark.NexmarkConfiguration;
-import org.apache.beam.integration.nexmark.NexmarkQueryModel;
 import org.apache.beam.integration.nexmark.NexmarkUtils;
 import org.apache.beam.integration.nexmark.model.Bid;
 import org.apache.beam.integration.nexmark.model.Event;

http://git-wip-us.apache.org/repos/asf/beam/blob/a39cb800/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query8.java
----------------------------------------------------------------------
diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query8.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query8.java
index e7daccd..603841b 100644
--- a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query8.java
+++ b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query8.java
@@ -18,7 +18,6 @@
 package org.apache.beam.integration.nexmark.queries;
 
 import org.apache.beam.integration.nexmark.NexmarkConfiguration;
-import org.apache.beam.integration.nexmark.NexmarkQuery;
 import org.apache.beam.integration.nexmark.NexmarkUtils;
 import org.apache.beam.integration.nexmark.model.Auction;
 import org.apache.beam.integration.nexmark.model.Event;

http://git-wip-us.apache.org/repos/asf/beam/blob/a39cb800/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query8Model.java
----------------------------------------------------------------------
diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query8Model.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query8Model.java
index 1161994..8c76bc6 100644
--- a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query8Model.java
+++ b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query8Model.java
@@ -24,9 +24,7 @@ import java.util.Collection;
 import java.util.HashMap;
 import java.util.Iterator;
 import java.util.Map;
-import org.apache.beam.integration.nexmark.AbstractSimulator;
 import org.apache.beam.integration.nexmark.NexmarkConfiguration;
-import org.apache.beam.integration.nexmark.NexmarkQueryModel;
 import org.apache.beam.integration.nexmark.NexmarkUtils;
 import org.apache.beam.integration.nexmark.model.Auction;
 import org.apache.beam.integration.nexmark.model.Event;

http://git-wip-us.apache.org/repos/asf/beam/blob/a39cb800/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query9.java
----------------------------------------------------------------------
diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query9.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query9.java
index aed827b..6dd189d 100644
--- a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query9.java
+++ b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query9.java
@@ -18,9 +18,7 @@
 package org.apache.beam.integration.nexmark.queries;
 
 import org.apache.beam.integration.nexmark.NexmarkConfiguration;
-import org.apache.beam.integration.nexmark.NexmarkQuery;
 import org.apache.beam.integration.nexmark.NexmarkUtils;
-import org.apache.beam.integration.nexmark.WinningBids;
 import org.apache.beam.integration.nexmark.model.AuctionBid;
 import org.apache.beam.integration.nexmark.model.Event;
 import org.apache.beam.integration.nexmark.model.KnownSize;

http://git-wip-us.apache.org/repos/asf/beam/blob/a39cb800/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query9Model.java
----------------------------------------------------------------------
diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query9Model.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query9Model.java
index b88d60a..d117e2d 100644
--- a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query9Model.java
+++ b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query9Model.java
@@ -21,10 +21,7 @@ import java.io.Serializable;
 import java.util.Collection;
 import java.util.Iterator;
 
-import org.apache.beam.integration.nexmark.AbstractSimulator;
 import org.apache.beam.integration.nexmark.NexmarkConfiguration;
-import org.apache.beam.integration.nexmark.NexmarkQueryModel;
-import org.apache.beam.integration.nexmark.WinningBidsSimulator;
 import org.apache.beam.sdk.values.TimestampedValue;
 
 /**

http://git-wip-us.apache.org/repos/asf/beam/blob/a39cb800/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/WinningBids.java
----------------------------------------------------------------------
diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/WinningBids.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/WinningBids.java
new file mode 100644
index 0000000..11a4d38
--- /dev/null
+++ b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/WinningBids.java
@@ -0,0 +1,379 @@
+/*
+ * 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.beam.integration.nexmark.queries;
+
+import static com.google.common.base.Preconditions.checkState;
+
+import com.fasterxml.jackson.annotation.JsonCreator;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
+import java.io.Serializable;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.List;
+import java.util.Map;
+import java.util.TreeMap;
+import org.apache.beam.integration.nexmark.NexmarkConfiguration;
+import org.apache.beam.integration.nexmark.NexmarkUtils;
+import org.apache.beam.integration.nexmark.model.Auction;
+import org.apache.beam.integration.nexmark.model.AuctionBid;
+import org.apache.beam.integration.nexmark.model.Bid;
+import org.apache.beam.integration.nexmark.model.Event;
+import org.apache.beam.integration.nexmark.sources.GeneratorConfig;
+import org.apache.beam.sdk.coders.Coder;
+import org.apache.beam.sdk.coders.CoderException;
+import org.apache.beam.sdk.coders.CustomCoder;
+import org.apache.beam.sdk.coders.VarIntCoder;
+import org.apache.beam.sdk.coders.VarLongCoder;
+import org.apache.beam.sdk.metrics.Counter;
+import org.apache.beam.sdk.metrics.Metrics;
+import org.apache.beam.sdk.transforms.DoFn;
+import org.apache.beam.sdk.transforms.PTransform;
+import org.apache.beam.sdk.transforms.ParDo;
+import org.apache.beam.sdk.transforms.join.CoGbkResult;
+import org.apache.beam.sdk.transforms.join.CoGroupByKey;
+import org.apache.beam.sdk.transforms.join.KeyedPCollectionTuple;
+import org.apache.beam.sdk.transforms.windowing.IntervalWindow;
+import org.apache.beam.sdk.transforms.windowing.Window;
+import org.apache.beam.sdk.transforms.windowing.WindowFn;
+import org.apache.beam.sdk.transforms.windowing.WindowMappingFn;
+import org.apache.beam.sdk.values.KV;
+import org.apache.beam.sdk.values.PCollection;
+import org.joda.time.Instant;
+
+/**
+ * A transform to find the winning bid for each closed auction. In pseudo CQL syntax:
+ *
+ * <pre>{@code
+ * SELECT Rstream(A.*, B.auction, B.bidder, MAX(B.price), B.dateTime)
+ * FROM Auction A [ROWS UNBOUNDED], Bid B [ROWS UNBOUNDED]
+ * WHERE A.id = B.auction AND B.datetime < A.expires AND A.expires < CURRENT_TIME
+ * GROUP BY A.id
+ * }</pre>
+ *
+ * <p>We will also check that the winning bid is above the auction reserve. Note that
+ * we ignore the auction opening bid value since it has no impact on which bid eventually wins,
+ * if any.
+ *
+ * <p>Our implementation will use a custom windowing function in order to bring bids and
+ * auctions together without requiring global state.
+ */
+public class WinningBids extends PTransform<PCollection<Event>, PCollection<AuctionBid>> {
+  /** Windows for open auctions and bids. */
+  private static class AuctionOrBidWindow extends IntervalWindow implements Serializable {
+    /** Id of auction this window is for. */
+    public final long auction;
+
+    /**
+     * True if this window represents an actual auction, and thus has a start/end
+     * time matching that of the auction. False if this window represents a bid, and
+     * thus has an unbounded start/end time.
+     */
+    public final boolean isAuctionWindow;
+
+    /** For avro only. */
+    private AuctionOrBidWindow() {
+      super(TIMESTAMP_MIN_VALUE, TIMESTAMP_MAX_VALUE);
+      auction = 0;
+      isAuctionWindow = false;
+    }
+
+    private AuctionOrBidWindow(
+        Instant start, Instant end, long auctionId, boolean isAuctionWindow) {
+      super(start, end);
+      this.auction = auctionId;
+      this.isAuctionWindow = isAuctionWindow;
+    }
+
+    /** Return an auction window for {@code auction}. */
+    public static AuctionOrBidWindow forAuction(Instant timestamp, Auction auction) {
+      AuctionOrBidWindow result =
+          new AuctionOrBidWindow(timestamp, new Instant(auction.expires), auction.id, true);
+      return result;
+    }
+
+    /**
+     * Return a bid window for {@code bid}. It should later be merged into
+     * the corresponding auction window. However, it is possible this bid is for an already
+     * expired auction, or for an auction which the system has not yet seen. So we
+     * give the bid a bit of wiggle room in its interval.
+     */
+    public static AuctionOrBidWindow forBid(
+        long expectedAuctionDurationMs, Instant timestamp, Bid bid) {
+      // At this point we don't know which auctions are still valid, and the bid may
+      // be for an auction which won't start until some unknown time in the future
+      // (due to Generator.AUCTION_ID_LEAD in Generator.nextBid).
+      // A real system would atomically reconcile bids and auctions by a separate mechanism.
+      // If we give bids an unbounded window it is possible a bid for an auction which
+      // has already expired would cause the system watermark to stall, since that window
+      // would never be retired.
+      // Instead, we will just give the bid a finite window which expires at
+      // the upper bound of auctions assuming the auction starts at the same time as the bid,
+      // and assuming the system is running at its lowest event rate (as per interEventDelayUs).
+      AuctionOrBidWindow result = new AuctionOrBidWindow(
+          timestamp, timestamp.plus(expectedAuctionDurationMs * 2), bid.auction, false);
+      return result;
+    }
+
+    /** Is this an auction window? */
+    public boolean isAuctionWindow() {
+      return isAuctionWindow;
+    }
+
+    @Override
+    public String toString() {
+      return String.format("AuctionOrBidWindow{start:%s; end:%s; auction:%d; isAuctionWindow:%s}",
+          start(), end(), auction, isAuctionWindow);
+    }
+  }
+
+  /**
+   * Encodes an {@link AuctionOrBidWindow} as an {@link IntervalWindow} and an auction id long.
+   */
+  private static class AuctionOrBidWindowCoder extends CustomCoder<AuctionOrBidWindow> {
+    private static final AuctionOrBidWindowCoder INSTANCE = new AuctionOrBidWindowCoder();
+    private static final Coder<IntervalWindow> SUPER_CODER = IntervalWindow.getCoder();
+    private static final Coder<Long> ID_CODER = VarLongCoder.of();
+    private static final Coder<Integer> INT_CODER = VarIntCoder.of();
+
+    @JsonCreator
+    public static AuctionOrBidWindowCoder of() {
+      return INSTANCE;
+    }
+
+    @Override
+    public void encode(AuctionOrBidWindow window, OutputStream outStream, Coder.Context context)
+        throws IOException, CoderException {
+      SUPER_CODER.encode(window, outStream, Coder.Context.NESTED);
+      ID_CODER.encode(window.auction, outStream, Coder.Context.NESTED);
+      INT_CODER.encode(window.isAuctionWindow ? 1 : 0, outStream, Coder.Context.NESTED);
+    }
+
+    @Override
+    public AuctionOrBidWindow decode(InputStream inStream, Coder.Context context)
+        throws IOException, CoderException {
+      IntervalWindow superWindow = SUPER_CODER.decode(inStream, Coder.Context.NESTED);
+      long auction = ID_CODER.decode(inStream, Coder.Context.NESTED);
+      boolean isAuctionWindow =
+          INT_CODER.decode(inStream, Coder.Context.NESTED) == 0 ? false : true;
+      return new AuctionOrBidWindow(
+          superWindow.start(), superWindow.end(), auction, isAuctionWindow);
+    }
+
+    @Override public void verifyDeterministic() throws NonDeterministicException {}
+  }
+
+  /** Assign events to auction windows and merges them intelligently. */
+  private static class AuctionOrBidWindowFn extends WindowFn<Event, AuctionOrBidWindow> {
+    /** Expected duration of auctions in ms. */
+    private final long expectedAuctionDurationMs;
+
+    public AuctionOrBidWindowFn(long expectedAuctionDurationMs) {
+      this.expectedAuctionDurationMs = expectedAuctionDurationMs;
+    }
+
+    @Override
+    public Collection<AuctionOrBidWindow> assignWindows(AssignContext c) {
+      Event event = c.element();
+      if (event.newAuction != null) {
+        // Assign auctions to an auction window which expires at the auction's close.
+        return Arrays.asList(AuctionOrBidWindow.forAuction(c.timestamp(), event.newAuction));
+      } else if (event.bid != null) {
+        // Assign bids to a temporary bid window which will later be merged into the appropriate
+        // auction window.
+        return Arrays.asList(
+            AuctionOrBidWindow.forBid(expectedAuctionDurationMs, c.timestamp(), event.bid));
+      } else {
+        // Don't assign people to any window. They will thus be dropped.
+        return Arrays.asList();
+      }
+    }
+
+    @Override
+    public void mergeWindows(MergeContext c) throws Exception {
+      // Split and index the auction and bid windows by auction id.
+      Map<Long, AuctionOrBidWindow> idToTrueAuctionWindow = new TreeMap<>();
+      Map<Long, List<AuctionOrBidWindow>> idToBidAuctionWindows = new TreeMap<>();
+      for (AuctionOrBidWindow window : c.windows()) {
+        if (window.isAuctionWindow()) {
+          idToTrueAuctionWindow.put(window.auction, window);
+        } else {
+          List<AuctionOrBidWindow> bidWindows = idToBidAuctionWindows.get(window.auction);
+          if (bidWindows == null) {
+            bidWindows = new ArrayList<>();
+            idToBidAuctionWindows.put(window.auction, bidWindows);
+          }
+          bidWindows.add(window);
+        }
+      }
+
+      // Merge all 'bid' windows into their corresponding 'auction' window, provided the
+      // auction has not expired.
+      for (long auction : idToTrueAuctionWindow.keySet()) {
+        AuctionOrBidWindow auctionWindow = idToTrueAuctionWindow.get(auction);
+        List<AuctionOrBidWindow> bidWindows = idToBidAuctionWindows.get(auction);
+        if (bidWindows != null) {
+          List<AuctionOrBidWindow> toBeMerged = new ArrayList<>();
+          for (AuctionOrBidWindow bidWindow : bidWindows) {
+            if (bidWindow.start().isBefore(auctionWindow.end())) {
+              toBeMerged.add(bidWindow);
+            }
+            // else: This bid window will remain until its expire time, at which point it
+            // will expire without ever contributing to an output.
+          }
+          if (!toBeMerged.isEmpty()) {
+            toBeMerged.add(auctionWindow);
+            c.merge(toBeMerged, auctionWindow);
+          }
+        }
+      }
+    }
+
+    @Override
+    public boolean isCompatible(WindowFn<?, ?> other) {
+      return other instanceof AuctionOrBidWindowFn;
+    }
+
+    @Override
+    public Coder<AuctionOrBidWindow> windowCoder() {
+      return AuctionOrBidWindowCoder.of();
+    }
+
+    @Override
+    public WindowMappingFn<AuctionOrBidWindow> getDefaultWindowMappingFn() {
+      throw new UnsupportedOperationException("AuctionWindowFn not supported for side inputs");
+    }
+
+    /**
+     * Below we will GBK auctions and bids on their auction ids. Then we will reduce those
+     * per id to emit {@code (auction, winning bid)} pairs for auctions which have expired with at
+     * least one valid bid. We would like those output pairs to have a timestamp of the auction's
+     * expiry (since that's the earliest we know for sure we have the correct winner). We would
+     * also like to make that winning results are available to following stages at the auction's
+     * expiry.
+     *
+     * <p>Each result of the GBK will have a timestamp of the min of the result of this object's
+     * assignOutputTime over all records which end up in one of its iterables. Thus we get the
+     * desired behavior if we ignore each record's timestamp and always return the auction window's
+     * 'maxTimestamp', which will correspond to the auction's expiry.
+     *
+     * <p>In contrast, if this object's assignOutputTime were to return 'inputTimestamp'
+     * (the usual implementation), then each GBK record will take as its timestamp the minimum of
+     * the timestamps of all bids and auctions within it, which will always be the auction's
+     * timestamp. An auction which expires well into the future would thus hold up the watermark
+     * of the GBK results until that auction expired. That in turn would hold up all winning pairs.
+     */
+    @Override
+    public Instant getOutputTime(
+        Instant inputTimestamp, AuctionOrBidWindow window) {
+      return window.maxTimestamp();
+    }
+  }
+
+  private final AuctionOrBidWindowFn auctionOrBidWindowFn;
+
+  public WinningBids(String name, NexmarkConfiguration configuration) {
+    super(name);
+    // What's the expected auction time (when the system is running at the lowest event rate).
+    long[] interEventDelayUs = configuration.rateShape.interEventDelayUs(
+        configuration.firstEventRate, configuration.nextEventRate,
+        configuration.rateUnit, configuration.numEventGenerators);
+    long longestDelayUs = 0;
+    for (int i = 0; i < interEventDelayUs.length; i++) {
+      longestDelayUs = Math.max(longestDelayUs, interEventDelayUs[i]);
+    }
+    // Adjust for proportion of auction events amongst all events.
+    longestDelayUs =
+        (longestDelayUs * GeneratorConfig.PROPORTION_DENOMINATOR)
+        / GeneratorConfig.AUCTION_PROPORTION;
+    // Adjust for number of in-flight auctions.
+    longestDelayUs = longestDelayUs * configuration.numInFlightAuctions;
+    long expectedAuctionDurationMs = (longestDelayUs + 999) / 1000;
+    NexmarkUtils.console("Expected auction duration is %d ms", expectedAuctionDurationMs);
+    auctionOrBidWindowFn = new AuctionOrBidWindowFn(expectedAuctionDurationMs);
+  }
+
+  @Override
+  public PCollection<AuctionBid> expand(PCollection<Event> events) {
+    // Window auctions and bids into custom auction windows. New people events will be discarded.
+    // This will allow us to bring bids and auctions together irrespective of how long
+    // each auction is open for.
+    events = events.apply("Window", Window.into(auctionOrBidWindowFn));
+
+    // Key auctions by their id.
+    PCollection<KV<Long, Auction>> auctionsById =
+        events.apply(NexmarkQuery.JUST_NEW_AUCTIONS)
+              .apply("AuctionById:", NexmarkQuery.AUCTION_BY_ID);
+
+    // Key bids by their auction id.
+    PCollection<KV<Long, Bid>> bidsByAuctionId =
+        events.apply(NexmarkQuery.JUST_BIDS).apply("BidByAuction", NexmarkQuery.BID_BY_AUCTION);
+
+    // Find the highest price valid bid for each closed auction.
+    return
+      // Join auctions and bids.
+      KeyedPCollectionTuple.of(NexmarkQuery.AUCTION_TAG, auctionsById)
+        .and(NexmarkQuery.BID_TAG, bidsByAuctionId)
+        .apply(CoGroupByKey.<Long>create())
+        // Filter and select.
+        .apply(name + ".Join",
+          ParDo.of(new DoFn<KV<Long, CoGbkResult>, AuctionBid>() {
+            private final Counter noAuctionCounter = Metrics.counter(name, "noAuction");
+            private final Counter underReserveCounter = Metrics.counter(name, "underReserve");
+            private final Counter noValidBidsCounter = Metrics.counter(name, "noValidBids");
+
+            @ProcessElement
+            public void processElement(ProcessContext c) {
+              Auction auction =
+                  c.element().getValue().getOnly(NexmarkQuery.AUCTION_TAG, null);
+              if (auction == null) {
+                // We have bids without a matching auction. Give up.
+                noAuctionCounter.inc();
+                return;
+              }
+              // Find the current winning bid for auction.
+              // The earliest bid with the maximum price above the reserve wins.
+              Bid bestBid = null;
+              for (Bid bid : c.element().getValue().getAll(NexmarkQuery.BID_TAG)) {
+                // Bids too late for their auction will have been
+                // filtered out by the window merge function.
+                checkState(bid.dateTime < auction.expires);
+                if (bid.price < auction.reserve) {
+                  // Bid price is below auction reserve.
+                  underReserveCounter.inc();
+                  continue;
+                }
+
+                if (bestBid == null
+                    || Bid.PRICE_THEN_DESCENDING_TIME.compare(bid, bestBid) > 0) {
+                  bestBid = bid;
+                }
+              }
+              if (bestBid == null) {
+                // We don't have any valid bids for auction.
+                noValidBidsCounter.inc();
+                return;
+              }
+              c.output(new AuctionBid(auction, bestBid));
+            }
+          }
+        ));
+  }
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/a39cb800/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/WinningBidsSimulator.java
----------------------------------------------------------------------
diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/WinningBidsSimulator.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/WinningBidsSimulator.java
new file mode 100644
index 0000000..7d74f8f
--- /dev/null
+++ b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/WinningBidsSimulator.java
@@ -0,0 +1,207 @@
+/*
+ * 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.beam.integration.nexmark.queries;
+
+import java.util.ArrayList;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.TreeMap;
+import java.util.TreeSet;
+import javax.annotation.Nullable;
+
+import org.apache.beam.integration.nexmark.NexmarkConfiguration;
+import org.apache.beam.integration.nexmark.NexmarkUtils;
+import org.apache.beam.integration.nexmark.model.Auction;
+import org.apache.beam.integration.nexmark.model.AuctionBid;
+import org.apache.beam.integration.nexmark.model.Bid;
+import org.apache.beam.integration.nexmark.model.Event;
+import org.apache.beam.sdk.transforms.windowing.BoundedWindow;
+import org.apache.beam.sdk.values.TimestampedValue;
+import org.joda.time.Instant;
+
+/**
+ * A simulator of the {@code WinningBids} query.
+ */
+public class WinningBidsSimulator extends AbstractSimulator<Event, AuctionBid> {
+  /** Auctions currently still open, indexed by auction id. */
+  private final Map<Long, Auction> openAuctions;
+
+  /** The ids of auctions known to be closed. */
+  private final Set<Long> closedAuctions;
+
+  /** Current best valid bids for open auctions, indexed by auction id. */
+  private final Map<Long, Bid> bestBids;
+
+  /** Bids for auctions we havn't seen yet. */
+  private final List<Bid> bidsWithoutAuctions;
+
+  /**
+   * Timestamp of last new auction or bid event (ms since epoch).
+   */
+  private long lastTimestamp;
+
+  public WinningBidsSimulator(NexmarkConfiguration configuration) {
+    super(NexmarkUtils.standardEventIterator(configuration));
+    openAuctions = new TreeMap<>();
+    closedAuctions = new TreeSet<>();
+    bestBids = new TreeMap<>();
+    bidsWithoutAuctions = new ArrayList<>();
+    lastTimestamp = BoundedWindow.TIMESTAMP_MIN_VALUE.getMillis();
+  }
+
+  /**
+   * Try to account for {@code bid} in state. Return true if bid has now been
+   * accounted for by {@code bestBids}.
+   */
+  private boolean captureBestBid(Bid bid, boolean shouldLog) {
+    if (closedAuctions.contains(bid.auction)) {
+      // Ignore bids for known, closed auctions.
+      if (shouldLog) {
+        NexmarkUtils.info("closed auction: %s", bid);
+      }
+      return true;
+    }
+    Auction auction = openAuctions.get(bid.auction);
+    if (auction == null) {
+      // We don't have an auction for this bid yet, so can't determine if it is
+      // winning or not.
+      if (shouldLog) {
+        NexmarkUtils.info("pending auction: %s", bid);
+      }
+      return false;
+    }
+    if (bid.price < auction.reserve) {
+      // Bid price is too low.
+      if (shouldLog) {
+        NexmarkUtils.info("below reserve: %s", bid);
+      }
+      return true;
+    }
+    Bid existingBid = bestBids.get(bid.auction);
+    if (existingBid == null || Bid.PRICE_THEN_DESCENDING_TIME.compare(existingBid, bid) < 0) {
+      // We've found a (new) best bid for a known auction.
+      bestBids.put(bid.auction, bid);
+      if (shouldLog) {
+        NexmarkUtils.info("new winning bid: %s", bid);
+      }
+    } else {
+      if (shouldLog) {
+        NexmarkUtils.info("ignoring low bid: %s", bid);
+      }
+    }
+    return true;
+  }
+
+  /**
+   * Try to match bids without auctions to auctions.
+   */
+  private void flushBidsWithoutAuctions() {
+    Iterator<Bid> itr = bidsWithoutAuctions.iterator();
+    while (itr.hasNext()) {
+      Bid bid = itr.next();
+      if (captureBestBid(bid, false)) {
+        NexmarkUtils.info("bid now accounted for: %s", bid);
+        itr.remove();
+      }
+    }
+  }
+
+  /**
+   * Return the next winning bid for an expired auction relative to {@code timestamp}.
+   * Return null if no more winning bids, in which case all expired auctions will
+   * have been removed from our state. Retire auctions in order of expire time.
+   */
+  @Nullable
+  private TimestampedValue<AuctionBid> nextWinningBid(long timestamp) {
+    Map<Long, List<Long>> toBeRetired = new TreeMap<>();
+    for (Map.Entry<Long, Auction> entry : openAuctions.entrySet()) {
+      if (entry.getValue().expires <= timestamp) {
+        List<Long> idsAtTime = toBeRetired.get(entry.getValue().expires);
+        if (idsAtTime == null) {
+          idsAtTime = new ArrayList<>();
+          toBeRetired.put(entry.getValue().expires, idsAtTime);
+        }
+        idsAtTime.add(entry.getKey());
+      }
+    }
+    for (Map.Entry<Long, List<Long>> entry : toBeRetired.entrySet()) {
+      for (long id : entry.getValue()) {
+        Auction auction = openAuctions.get(id);
+        NexmarkUtils.info("retiring auction: %s", auction);
+        openAuctions.remove(id);
+        Bid bestBid = bestBids.get(id);
+        if (bestBid != null) {
+          TimestampedValue<AuctionBid> result =
+              TimestampedValue.of(new AuctionBid(auction, bestBid), new Instant(auction.expires));
+          NexmarkUtils.info("winning: %s", result);
+          return result;
+        }
+      }
+    }
+    return null;
+  }
+
+  @Override
+  protected void run() {
+    if (lastTimestamp > BoundedWindow.TIMESTAMP_MIN_VALUE.getMillis()) {
+      // We may have finally seen the auction a bid was intended for.
+      flushBidsWithoutAuctions();
+      TimestampedValue<AuctionBid> result = nextWinningBid(lastTimestamp);
+      if (result != null) {
+        addResult(result);
+        return;
+      }
+    }
+
+    TimestampedValue<Event> timestampedEvent = nextInput();
+    if (timestampedEvent == null) {
+      // No more events. Flush any still open auctions.
+      TimestampedValue<AuctionBid> result =
+          nextWinningBid(BoundedWindow.TIMESTAMP_MAX_VALUE.getMillis());
+      if (result == null) {
+        // We are done.
+        allDone();
+        return;
+      }
+      addResult(result);
+      //TODO test fails because offset of some hundreds of ms beween expect and actual
+      return;
+    }
+
+    Event event = timestampedEvent.getValue();
+    if (event.newPerson != null) {
+      // Ignore new person events.
+      return;
+    }
+
+    lastTimestamp = timestampedEvent.getTimestamp().getMillis();
+    if (event.newAuction != null) {
+      // Add this new open auction to our state.
+      openAuctions.put(event.newAuction.id, event.newAuction);
+    } else {
+      if (!captureBestBid(event.bid, true)) {
+        // We don't know what to do with this bid yet.
+        NexmarkUtils.info("bid not yet accounted for: %s", event.bid);
+        bidsWithoutAuctions.add(event.bid);
+      }
+    }
+    // Keep looking for winning bids.
+  }
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/a39cb800/integration/java/nexmark/src/test/java/org/apache/beam/integration/nexmark/queries/QueryTest.java
----------------------------------------------------------------------
diff --git a/integration/java/nexmark/src/test/java/org/apache/beam/integration/nexmark/queries/QueryTest.java b/integration/java/nexmark/src/test/java/org/apache/beam/integration/nexmark/queries/QueryTest.java
index 284aa7e..b005d65 100644
--- a/integration/java/nexmark/src/test/java/org/apache/beam/integration/nexmark/queries/QueryTest.java
+++ b/integration/java/nexmark/src/test/java/org/apache/beam/integration/nexmark/queries/QueryTest.java
@@ -18,8 +18,6 @@
 package org.apache.beam.integration.nexmark.queries;
 
 import org.apache.beam.integration.nexmark.NexmarkConfiguration;
-import org.apache.beam.integration.nexmark.NexmarkQuery;
-import org.apache.beam.integration.nexmark.NexmarkQueryModel;
 import org.apache.beam.integration.nexmark.NexmarkUtils;
 import org.apache.beam.integration.nexmark.model.KnownSize;
 import org.apache.beam.sdk.PipelineResult;


[52/55] [abbrv] beam git commit: Move module beam-integration-java-nexmark to beam-sdks-java-nexmark

Posted by ie...@apache.org.
Move module beam-integration-java-nexmark to beam-sdks-java-nexmark


Project: http://git-wip-us.apache.org/repos/asf/beam/repo
Commit: http://git-wip-us.apache.org/repos/asf/beam/commit/f4333df7
Tree: http://git-wip-us.apache.org/repos/asf/beam/tree/f4333df7
Diff: http://git-wip-us.apache.org/repos/asf/beam/diff/f4333df7

Branch: refs/heads/master
Commit: f4333df77267d5207f0f23ae62e79b171a00e8a7
Parents: 2f9b494
Author: Etienne Chauchot <ec...@gmail.com>
Authored: Thu Jun 15 11:55:26 2017 +0200
Committer: Ismaël Mejía <ie...@gmail.com>
Committed: Wed Aug 23 19:07:29 2017 +0200

----------------------------------------------------------------------
 integration/java/nexmark/README.md              |  340 -----
 integration/java/nexmark/pom.xml                |  292 -----
 .../apache/beam/integration/nexmark/Main.java   |  304 -----
 .../beam/integration/nexmark/Monitor.java       |   79 --
 .../nexmark/NexmarkConfiguration.java           |  721 -----------
 .../integration/nexmark/NexmarkLauncher.java    | 1158 ------------------
 .../integration/nexmark/NexmarkOptions.java     |  403 ------
 .../beam/integration/nexmark/NexmarkPerf.java   |  208 ----
 .../beam/integration/nexmark/NexmarkSuite.java  |  112 --
 .../beam/integration/nexmark/NexmarkUtils.java  |  672 ----------
 .../beam/integration/nexmark/model/Auction.java |  187 ---
 .../integration/nexmark/model/AuctionBid.java   |   84 --
 .../integration/nexmark/model/AuctionCount.java |   84 --
 .../integration/nexmark/model/AuctionPrice.java |   88 --
 .../beam/integration/nexmark/model/Bid.java     |  177 ---
 .../nexmark/model/BidsPerSession.java           |   87 --
 .../nexmark/model/CategoryPrice.java            |   97 --
 .../beam/integration/nexmark/model/Done.java    |   80 --
 .../beam/integration/nexmark/model/Event.java   |  171 ---
 .../nexmark/model/IdNameReserve.java            |   98 --
 .../integration/nexmark/model/KnownSize.java    |   26 -
 .../nexmark/model/NameCityStateId.java          |  103 --
 .../beam/integration/nexmark/model/Person.java  |  163 ---
 .../integration/nexmark/model/SellerPrice.java  |   89 --
 .../integration/nexmark/model/package-info.java |   22 -
 .../beam/integration/nexmark/package-info.java  |   21 -
 .../nexmark/queries/AbstractSimulator.java      |  211 ----
 .../nexmark/queries/NexmarkQuery.java           |  270 ----
 .../nexmark/queries/NexmarkQueryModel.java      |  118 --
 .../integration/nexmark/queries/Query0.java     |   71 --
 .../nexmark/queries/Query0Model.java            |   64 -
 .../integration/nexmark/queries/Query1.java     |   67 -
 .../integration/nexmark/queries/Query10.java    |  367 ------
 .../integration/nexmark/queries/Query11.java    |   79 --
 .../integration/nexmark/queries/Query12.java    |   80 --
 .../nexmark/queries/Query1Model.java            |   76 --
 .../integration/nexmark/queries/Query2.java     |   79 --
 .../nexmark/queries/Query2Model.java            |   80 --
 .../integration/nexmark/queries/Query3.java     |  301 -----
 .../nexmark/queries/Query3Model.java            |  124 --
 .../integration/nexmark/queries/Query4.java     |  116 --
 .../nexmark/queries/Query4Model.java            |  186 ---
 .../integration/nexmark/queries/Query5.java     |  138 ---
 .../nexmark/queries/Query5Model.java            |  176 ---
 .../integration/nexmark/queries/Query6.java     |  155 ---
 .../nexmark/queries/Query6Model.java            |  133 --
 .../integration/nexmark/queries/Query7.java     |   90 --
 .../nexmark/queries/Query7Model.java            |  130 --
 .../integration/nexmark/queries/Query8.java     |   97 --
 .../nexmark/queries/Query8Model.java            |  148 ---
 .../integration/nexmark/queries/Query9.java     |   44 -
 .../nexmark/queries/Query9Model.java            |   44 -
 .../nexmark/queries/WinningBids.java            |  412 -------
 .../nexmark/queries/WinningBidsSimulator.java   |  206 ----
 .../nexmark/queries/package-info.java           |   22 -
 .../nexmark/sources/BoundedEventSource.java     |  190 ---
 .../integration/nexmark/sources/Generator.java  |  609 ---------
 .../nexmark/sources/GeneratorConfig.java        |  301 -----
 .../nexmark/sources/UnboundedEventSource.java   |  330 -----
 .../nexmark/sources/package-info.java           |   22 -
 .../nexmark/src/main/resources/log4j.properties |   55 -
 .../integration/nexmark/queries/QueryTest.java  |  185 ---
 .../nexmark/sources/BoundedEventSourceTest.java |   71 --
 .../nexmark/sources/GeneratorTest.java          |  111 --
 .../sources/UnboundedEventSourceTest.java       |  107 --
 integration/java/pom.xml                        |   37 -
 integration/pom.xml                             |   51 -
 pom.xml                                         |    1 -
 sdks/java/nexmark/README.md                     |  340 +++++
 sdks/java/nexmark/pom.xml                       |  292 +++++
 .../java/org/apache/beam/sdk/nexmark/Main.java  |  303 +++++
 .../org/apache/beam/sdk/nexmark/Monitor.java    |   78 ++
 .../beam/sdk/nexmark/NexmarkConfiguration.java  |  721 +++++++++++
 .../beam/sdk/nexmark/NexmarkLauncher.java       | 1157 +++++++++++++++++
 .../apache/beam/sdk/nexmark/NexmarkOptions.java |  403 ++++++
 .../apache/beam/sdk/nexmark/NexmarkPerf.java    |  207 ++++
 .../apache/beam/sdk/nexmark/NexmarkSuite.java   |  112 ++
 .../apache/beam/sdk/nexmark/NexmarkUtils.java   |  674 ++++++++++
 .../apache/beam/sdk/nexmark/model/Auction.java  |  187 +++
 .../beam/sdk/nexmark/model/AuctionBid.java      |   85 ++
 .../beam/sdk/nexmark/model/AuctionCount.java    |   84 ++
 .../beam/sdk/nexmark/model/AuctionPrice.java    |   88 ++
 .../org/apache/beam/sdk/nexmark/model/Bid.java  |  177 +++
 .../beam/sdk/nexmark/model/BidsPerSession.java  |   87 ++
 .../beam/sdk/nexmark/model/CategoryPrice.java   |   97 ++
 .../org/apache/beam/sdk/nexmark/model/Done.java |   80 ++
 .../apache/beam/sdk/nexmark/model/Event.java    |  171 +++
 .../beam/sdk/nexmark/model/IdNameReserve.java   |   98 ++
 .../beam/sdk/nexmark/model/KnownSize.java       |   26 +
 .../beam/sdk/nexmark/model/NameCityStateId.java |  103 ++
 .../apache/beam/sdk/nexmark/model/Person.java   |  163 +++
 .../beam/sdk/nexmark/model/SellerPrice.java     |   89 ++
 .../beam/sdk/nexmark/model/package-info.java    |   22 +
 .../apache/beam/sdk/nexmark/package-info.java   |   21 +
 .../sdk/nexmark/queries/AbstractSimulator.java  |  211 ++++
 .../beam/sdk/nexmark/queries/NexmarkQuery.java  |  270 ++++
 .../sdk/nexmark/queries/NexmarkQueryModel.java  |  118 ++
 .../apache/beam/sdk/nexmark/queries/Query0.java |   70 ++
 .../beam/sdk/nexmark/queries/Query0Model.java   |   64 +
 .../apache/beam/sdk/nexmark/queries/Query1.java |   67 +
 .../beam/sdk/nexmark/queries/Query10.java       |  367 ++++++
 .../beam/sdk/nexmark/queries/Query11.java       |   79 ++
 .../beam/sdk/nexmark/queries/Query12.java       |   80 ++
 .../beam/sdk/nexmark/queries/Query1Model.java   |   76 ++
 .../apache/beam/sdk/nexmark/queries/Query2.java |   79 ++
 .../beam/sdk/nexmark/queries/Query2Model.java   |   80 ++
 .../apache/beam/sdk/nexmark/queries/Query3.java |  301 +++++
 .../beam/sdk/nexmark/queries/Query3Model.java   |  124 ++
 .../apache/beam/sdk/nexmark/queries/Query4.java |  116 ++
 .../beam/sdk/nexmark/queries/Query4Model.java   |  186 +++
 .../apache/beam/sdk/nexmark/queries/Query5.java |  138 +++
 .../beam/sdk/nexmark/queries/Query5Model.java   |  176 +++
 .../apache/beam/sdk/nexmark/queries/Query6.java |  155 +++
 .../beam/sdk/nexmark/queries/Query6Model.java   |  133 ++
 .../apache/beam/sdk/nexmark/queries/Query7.java |   90 ++
 .../beam/sdk/nexmark/queries/Query7Model.java   |  130 ++
 .../apache/beam/sdk/nexmark/queries/Query8.java |   97 ++
 .../beam/sdk/nexmark/queries/Query8Model.java   |  148 +++
 .../apache/beam/sdk/nexmark/queries/Query9.java |   44 +
 .../beam/sdk/nexmark/queries/Query9Model.java   |   44 +
 .../beam/sdk/nexmark/queries/WinningBids.java   |  412 +++++++
 .../nexmark/queries/WinningBidsSimulator.java   |  206 ++++
 .../beam/sdk/nexmark/queries/package-info.java  |   22 +
 .../sdk/nexmark/sources/BoundedEventSource.java |  190 +++
 .../beam/sdk/nexmark/sources/Generator.java     |  609 +++++++++
 .../sdk/nexmark/sources/GeneratorConfig.java    |  298 +++++
 .../nexmark/sources/UnboundedEventSource.java   |  329 +++++
 .../beam/sdk/nexmark/sources/package-info.java  |   22 +
 .../nexmark/src/main/resources/log4j.properties |   55 +
 .../beam/sdk/nexmark/queries/QueryTest.java     |  185 +++
 .../nexmark/sources/BoundedEventSourceTest.java |   70 ++
 .../beam/sdk/nexmark/sources/GeneratorTest.java |  110 ++
 .../sources/UnboundedEventSourceTest.java       |  105 ++
 sdks/java/pom.xml                               |    1 +
 134 files changed, 11922 insertions(+), 12020 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/beam/blob/f4333df7/integration/java/nexmark/README.md
----------------------------------------------------------------------
diff --git a/integration/java/nexmark/README.md b/integration/java/nexmark/README.md
deleted file mode 100644
index a9acd63..0000000
--- a/integration/java/nexmark/README.md
+++ /dev/null
@@ -1,340 +0,0 @@
-<!--
-    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.
--->
-
-# NEXMark integration suite
-
-This is a suite of pipelines inspired by the 'continuous data stream'
-queries in [http://datalab.cs.pdx.edu/niagaraST/NEXMark/]
-(http://datalab.cs.pdx.edu/niagaraST/NEXMark/).
-
-These are multiple queries over a three entities model representing on online auction system:
-
- - **Person** represents a person submitting an item for auction and/or making a bid
-    on an auction.
- - **Auction** represents an item under auction.
- - **Bid** represents a bid for an item under auction.
-
-The queries exercise many aspects of Beam model:
-
-* **Query1**: What are the bid values in Euro's?
-  Illustrates a simple map.
-* **Query2**: What are the auctions with particular auction numbers?
-  Illustrates a simple filter.
-* **Query3**: Who is selling in particular US states?
-  Illustrates an incremental join (using per-key state and timer) and filter.
-* **Query4**: What is the average selling price for each auction
-  category?
-  Illustrates complex join (using custom window functions) and
-  aggregation.
-* **Query5**: Which auctions have seen the most bids in the last period?
-  Illustrates sliding windows and combiners.
-* **Query6**: What is the average selling price per seller for their
-  last 10 closed auctions.
-  Shares the same 'winning bids' core as for **Query4**, and
-  illustrates a specialized combiner.
-* **Query7**: What are the highest bids per period?
-  Deliberately implemented using a side input to illustrate fanout.
-* **Query8**: Who has entered the system and created an auction in
-  the last period?
-  Illustrates a simple join.
-
-We have augmented the original queries with five more:
-
-* **Query0**: Pass-through.
-  Allows us to measure the monitoring overhead.
-* **Query9**: Winning-bids.
-  A common sub-query shared by **Query4** and **Query6**.
-* **Query10**: Log all events to GCS files.
-  Illustrates windows with large side effects on firing.
-* **Query11**: How many bids did a user make in each session they
-  were active?
-  Illustrates session windows.
-* **Query12**: How many bids does a user make within a fixed
-  processing time limit?
-  Illustrates working in processing time in the Global window, as
-  compared with event time in non-Global windows for all the other
-  queries.
-
-We can specify the Beam runner to use with maven profiles, available profiles are:
-
-* direct-runner
-* spark-runner
-* flink-runner
-* apex-runner
-
-The runner must also be specified like in any other Beam pipeline using
-
-    --runner
-
-
-Test data is deterministically synthesized on demand. The test
-data may be synthesized in the same pipeline as the query itself,
-or may be published to Pubsub.
-
-The query results may be:
-
-* Published to Pubsub.
-* Written to text files as plain text.
-* Written to text files using an Avro encoding.
-* Send to BigQuery.
-* Discarded.
-
-# Configuration
-
-## Common configuration parameters
-
-Decide if batch or streaming:
-
-    --streaming=true
-
-Number of events generators
-
-    --numEventGenerators=4
-
-Run query N
-
-    --query=N
-
-## Available Suites
-The suite to run can be chosen using this configuration parameter:
-
-    --suite=SUITE
-
-Available suites are:
-* DEFAULT: Test default configuration with query 0.
-* SMOKE: Run the 12 default configurations.
-* STRESS: Like smoke but for 1m events.
-* FULL_THROTTLE: Like SMOKE but 100m events.
-
-   
-
-## Apex specific configuration
-
-    --manageResources=false --monitorJobs=false
-
-## Dataflow specific configuration
-
-    --manageResources=false --monitorJobs=true \
-    --enforceEncodability=false --enforceImmutability=false
-    --project=<your project> \
-    --zone=<your zone> \
-    --workerMachineType=n1-highmem-8 \
-    --stagingLocation=<a gs path for staging> \
-    --runner=DataflowRunner \
-    --tempLocation=gs://talend-imejia/nexmark/temp/ \
-    --stagingLocation=gs://talend-imejia/nexmark/temp/staging/ \
-    --filesToStage=target/beam-integration-java-0.7.0-SNAPSHOT.jar
-
-## Direct specific configuration
-
-    --manageResources=false --monitorJobs=true \
-    --enforceEncodability=false --enforceImmutability=false
-
-## Flink specific configuration
-
-    --manageResources=false --monitorJobs=true \
-    --flinkMaster=local --parallelism=#numcores
-
-## Spark specific configuration
-
-    --manageResources=false --monitorJobs=true \
-    --sparkMaster=local \
-    -Dspark.ui.enabled=false -DSPARK_LOCAL_IP=localhost -Dsun.io.serialization.extendedDebugInfo=true
-
-# Current Status
-
-Open issues are tracked [here](https://github.com../../../../../issues):
-
-## Batch / Synthetic / Local
-
-| Query | Direct | Spark                                                        | Flink                                                      | Apex                                                         |
-| ----: | ------ | ------------------------------------------------------------ | ---------------------------------------------------------- | ------------------------------------------------------------ |
-|     0 | ok     | ok                                                           | ok                                                         | ok                                                           |
-|     1 | ok     | ok                                                           | ok                                                         | ok                                                           |
-|     2 | ok     | ok                                                           | ok                                                         | ok                                                           |
-|     3 | ok     | [BEAM-1035](https://issues.apache.org/jira/browse/BEAM-1035) | ok                                                         | [BEAM-1037](https://issues.apache.org/jira/browse/BEAM-1037) |
-|     4 | ok     | ok                                                           | ok                                                         | ok                                                           |
-|     5 | ok     | ok                                                           | ok                                                         | ok                                                           |
-|     6 | ok     | ok                                                           | ok                                                         | ok                                                           |
-|     7 | ok     | ok                                                           | ok                                                         | ok                                                           |
-|     8 | ok     | ok                                                           | ok                                                         | ok                                                           |
-|     9 | ok     | ok                                                           | ok                                                         | ok                                                           |
-|    10 | ok     | ok                                                           | ok                                                         | ok                                                           |
-|    11 | ok     | ok                                                           | ok                                                         | ok                                                           |
-|    12 | ok     | ok                                                           | ok                                                         | ok                                                           |
-
-## Streaming / Synthetic / Local
-
-| Query | Direct | Spark                                                        | Flink                                                      | Apex                                                         |
-| ----: | ------ | ------------------------------------------------------------ | ---------------------------------------------------------- | ------------------------------------------------------------ |
-|     0 | ok     | ok                                                           | [BEAM-593](https://issues.apache.org/jira/browse/BEAM-593) | ok                                                           |
-|     1 | ok     | ok                                                           | [BEAM-593](https://issues.apache.org/jira/browse/BEAM-593) | ok                                                           |
-|     2 | ok     | ok                                                           | [BEAM-593](https://issues.apache.org/jira/browse/BEAM-593) | ok                                                           |
-|     3 | ok     | [BEAM-1035](https://issues.apache.org/jira/browse/BEAM-1035) | [BEAM-593](https://issues.apache.org/jira/browse/BEAM-593) | [BEAM-1037](https://issues.apache.org/jira/browse/BEAM-1037) |
-|     4 | ok     | ok                                                           | [BEAM-593](https://issues.apache.org/jira/browse/BEAM-593) | ok                                                           |
-|     5 | ok     | ok                                                           | [BEAM-593](https://issues.apache.org/jira/browse/BEAM-593) | ok                                                           |
-|     6 | ok     | ok                                                           | [BEAM-593](https://issues.apache.org/jira/browse/BEAM-593) | ok                                                           |
-|     7 | ok     | [BEAM-2112](https://issues.apache.org/jira/browse/BEAM-2112) | [BEAM-593](https://issues.apache.org/jira/browse/BEAM-593) | ok                                                           |
-|     8 | ok     | ok                                                           | [BEAM-593](https://issues.apache.org/jira/browse/BEAM-593) | ok                                                           |
-|     9 | ok     | ok                                                           | [BEAM-593](https://issues.apache.org/jira/browse/BEAM-593) | ok                                                           |
-|    10 | ok     | ok                                                           | [BEAM-593](https://issues.apache.org/jira/browse/BEAM-593) | ok                                                           |
-|    11 | ok     | ok                                                           | [BEAM-593](https://issues.apache.org/jira/browse/BEAM-593) | ok                                                           |
-|    12 | ok     | ok                                                           | [BEAM-593](https://issues.apache.org/jira/browse/BEAM-593) | ok                                                           |
-
-## Batch / Synthetic / Cluster
-
-TODO
-
-| Query | Dataflow                       | Spark                          | Flink                          | Apex                           |
-| ----: | ------------------------------ | ------------------------------ | ------------------------------ | ------------------------------ |
-|     0 |                                |                                |                                |                                |
-
-## Streaming / Synthetic / Cluster
-
-TODO
-
-| Query | Dataflow                       | Spark                          | Flink                          | Apex                           |
-| ----: | ------------------------------ | ------------------------------ | ------------------------------ | ------------------------------ |
-|     0 |                                |                                |                                |                                |
-
-# Running Nexmark
-
-## Running SMOKE suite on the DirectRunner (local)
-
-Batch Mode
-
-    mvn exec:java -Dexec.mainClass=org.apache.beam.integration.nexmark.Main -Pdirect-runner -Dexec.args="--runner=DirectRunner --suite=SMOKE --streaming=false --manageResources=false --monitorJobs=true --enforceEncodability=true --enforceImmutability=true"
-
-Streaming Mode
-
-    mvn exec:java -Dexec.mainClass=org.apache.beam.integration.nexmark.Main -Pdirect-runner -Dexec.args="--runner=DirectRunner --suite=SMOKE --streaming=true --manageResources=false --monitorJobs=true --enforceEncodability=true --enforceImmutability=true"
-
-
-## Running SMOKE suite on the SparkRunner (local)
-
-Batch Mode
-
-    mvn exec:java -Dexec.mainClass=org.apache.beam.integration.nexmark.Main -Pspark-runner "-Dexec.args=--runner=SparkRunner --suite=SMOKE --streamTimeout=60 --streaming=false --manageResources=false --monitorJobs=true"
-
-Streaming Mode
-
-    mvn exec:java -Dexec.mainClass=org.apache.beam.integration.nexmark.Main -Pspark-runner "-Dexec.args=--runner=SparkRunner --suite=SMOKE --streamTimeout=60 --streaming=true --manageResources=false --monitorJobs=true"
-
-
-## Running SMOKE suite on the FlinkRunner (local)
-
-Batch Mode
-
-    mvn exec:java -Dexec.mainClass=org.apache.beam.integration.nexmark.Main -Pflink-runner "-Dexec.args=--runner=FlinkRunner --suite=SMOKE --streamTimeout=60 --streaming=false --manageResources=false --monitorJobs=true  --flinkMaster=local"
-
-Streaming Mode
-
-    mvn exec:java -Dexec.mainClass=org.apache.beam.integration.nexmark.Main -Pflink-runner "-Dexec.args=--runner=FlinkRunner --suite=SMOKE --streamTimeout=60 --streaming=true --manageResources=false --monitorJobs=true  --flinkMaster=local"
-
-
-## Running SMOKE suite on the ApexRunner (local)
-
-Batch Mode
-
-    mvn exec:java -Dexec.mainClass=org.apache.beam.integration.nexmark.Main -Papex-runner "-Dexec.args=--runner=ApexRunner --suite=SMOKE --streamTimeout=60 --streaming=false --manageResources=false --monitorJobs=false"
-
-Streaming Mode
-
-    mvn exec:java -Dexec.mainClass=org.apache.beam.integration.nexmark.Main -Papex-runner "-Dexec.args=--runner=ApexRunner --suite=SMOKE --streamTimeout=60 --streaming=true --manageResources=false --monitorJobs=false"
-
-
-## Running SMOKE suite on Google Cloud Dataflow
-
-Building package
-
-    mvn clean package -Pdataflow-runner
-
-Submit to Google Dataflow service
-
-
-```
-java -cp integration/java/target/beam-integration-java-nexmark-bundled-2.1.0-SNAPSHOT.jar \
-  org.apache.beam.integration.nexmark.Main \
-  --runner=DataflowRunner
-  --project=<your project> \
-  --zone=<your zone> \
-  --workerMachineType=n1-highmem-8 \
-  --stagingLocation=<a gs path for staging> \
-  --streaming=true \
-  --sourceType=PUBSUB \
-  --pubSubMode=PUBLISH_ONLY \
-  --pubsubTopic=<an existing Pubsub topic> \
-  --resourceNameMode=VERBATIM \
-  --manageResources=false \
-  --monitorJobs=false \
-  --numEventGenerators=64 \
-  --numWorkers=16 \
-  --maxNumWorkers=16 \
-  --suite=SMOKE \
-  --firstEventRate=100000 \
-  --nextEventRate=100000 \
-  --ratePeriodSec=3600 \
-  --isRateLimited=true \
-  --avgPersonByteSize=500 \
-  --avgAuctionByteSize=500 \
-  --avgBidByteSize=500 \
-  --probDelayedEvent=0.000001 \
-  --occasionalDelaySec=3600 \
-  --numEvents=0 \
-  --useWallclockEventTime=true \
-  --usePubsubPublishTime=true \
-  --experiments=enable_custom_pubsub_sink
-```
-
-```
-java -cp integration/java/target/beam-integration-java-nexmark-bundled-2.1.0-SNAPSHOT.jar \
-  org.apache.beam.integration.nexmark.Main \
-  --runner=DataflowRunner
-  --project=<your project> \
-  --zone=<your zone> \
-  --workerMachineType=n1-highmem-8 \
-  --stagingLocation=<a gs path for staging> \
-  --streaming=true \
-  --sourceType=PUBSUB \
-  --pubSubMode=SUBSCRIBE_ONLY \
-  --pubsubSubscription=<an existing Pubsub subscription to above topic> \
-  --resourceNameMode=VERBATIM \
-  --manageResources=false \
-  --monitorJobs=false \
-  --numWorkers=64 \
-  --maxNumWorkers=64 \
-  --suite=SMOKE \
-  --usePubsubPublishTime=true \
-  --outputPath=<a gs path under which log files will be written> \
-  --windowSizeSec=600 \
-  --occasionalDelaySec=3600 \
-  --maxLogEvents=10000 \
-  --experiments=enable_custom_pubsub_source
-```
-
-## Running query 0 on a Spark cluster with yarn
-
-Building package
-
-    mvn clean package -Pspark-runner
-
-Submit to the cluster
-
-    spark-submit --master yarn-client --class org.apache.beam.integration.nexmark.Main --driver-memory 512m --executor-memory 512m --executor-cores 1 beam-integration-java-nexmark-bundled-2.1.0-SNAPSHOT.jar --runner=SparkRunner --query=0 --streamTimeout=60 --streaming=false --manageResources=false --monitorJobs=true
-

http://git-wip-us.apache.org/repos/asf/beam/blob/f4333df7/integration/java/nexmark/pom.xml
----------------------------------------------------------------------
diff --git a/integration/java/nexmark/pom.xml b/integration/java/nexmark/pom.xml
deleted file mode 100644
index 664a410..0000000
--- a/integration/java/nexmark/pom.xml
+++ /dev/null
@@ -1,292 +0,0 @@
-<?xml version="1.0" encoding="UTF-8"?>
-<!--
-    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.
--->
-<project xmlns="http://maven.apache.org/POM/4.0.0"
-         xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
-         xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
-  <modelVersion>4.0.0</modelVersion>
-
-  <parent>
-    <groupId>org.apache.beam</groupId>
-    <artifactId>beam-integration-java-parent</artifactId>
-    <version>2.1.0-SNAPSHOT</version>
-    <relativePath>../pom.xml</relativePath>
-  </parent>
-
-  <artifactId>beam-integration-java-nexmark</artifactId>
-  <name>Apache Beam :: Integration Tests :: Java :: Nexmark</name>
-  <packaging>jar</packaging>
-
-  <profiles>
-
-    <!--
-      The direct runner is available by default.
-      You can also include it on the classpath explicitly with -P direct-runner
-    -->
-    <profile>
-      <id>direct-runner</id>
-      <activation>
-        <activeByDefault>true</activeByDefault>
-      </activation>
-      <dependencies>
-        <dependency>
-          <groupId>org.apache.beam</groupId>
-          <artifactId>beam-runners-direct-java</artifactId>
-          <scope>runtime</scope>
-        </dependency>
-      </dependencies>
-    </profile>
-
-    <!-- Include the Apache Apex runner with -P apex-runner -->
-    <profile>
-      <id>apex-runner</id>
-      <dependencies>
-        <dependency>
-          <groupId>org.apache.beam</groupId>
-          <artifactId>beam-runners-apex</artifactId>
-          <scope>runtime</scope>
-        </dependency>
-        <!--
-          Apex depends on httpclient version 4.3.5, project has a transitive dependency to httpclient 4.0.1 from
-          google-http-client. Apex dependency version being specified explicitly so that it gets picked up. This
-          can be removed when the project no longer has a dependency on a different httpclient version.
-        -->
-        <dependency>
-          <groupId>org.apache.httpcomponents</groupId>
-          <artifactId>httpclient</artifactId>
-          <version>4.3.5</version>
-          <scope>runtime</scope>
-          <exclusions>
-            <exclusion>
-              <groupId>commons-codec</groupId>
-              <artifactId>commons-codec</artifactId>
-            </exclusion>
-          </exclusions>
-        </dependency>
-      </dependencies>
-    </profile>
-
-    <!-- Include the Apache Flink runner with -P flink-runner -->
-    <profile>
-      <id>flink-runner</id>
-      <dependencies>
-        <dependency>
-          <groupId>org.apache.beam</groupId>
-          <artifactId>beam-runners-flink_2.10</artifactId>
-          <scope>runtime</scope>
-        </dependency>
-      </dependencies>
-    </profile>
-
-    <!-- Include the Apache Spark runner -P spark-runner -->
-    <profile>
-      <id>spark-runner</id>
-      <dependencies>
-        <dependency>
-          <groupId>org.apache.beam</groupId>
-          <artifactId>beam-runners-spark</artifactId>
-          <scope>runtime</scope>
-        </dependency>
-        <dependency>
-          <groupId>org.apache.spark</groupId>
-          <artifactId>spark-streaming_2.10</artifactId>
-          <version>${spark.version}</version>
-          <scope>runtime</scope>
-        </dependency>
-        <dependency>
-          <groupId>org.apache.spark</groupId>
-          <artifactId>spark-core_2.10</artifactId>
-          <version>${spark.version}</version>
-          <scope>runtime</scope>
-          <exclusions>
-            <exclusion>
-              <groupId>org.slf4j</groupId>
-              <artifactId>jul-to-slf4j</artifactId>
-            </exclusion>
-          </exclusions>
-        </dependency>
-      </dependencies>
-    </profile>
-
-    <!-- Include the Google Cloud Dataflow runner -P dataflow-runner -->
-    <profile>
-      <id>dataflow-runner</id>
-      <dependencies>
-        <dependency>
-          <groupId>org.apache.beam</groupId>
-          <artifactId>beam-runners-google-cloud-dataflow-java</artifactId>
-          <scope>runtime</scope>
-        </dependency>
-      </dependencies>
-    </profile>
-  </profiles>
-
-  <build>
-    <plugins>
-      <plugin>
-        <groupId>org.apache.maven.plugins</groupId>
-        <artifactId>maven-shade-plugin</artifactId>
-        <executions>
-          <execution>
-            <phase>package</phase>
-            <goals>
-              <goal>shade</goal>
-            </goals>
-            <configuration>
-              <finalName>${project.artifactId}-bundled-${project.version}</finalName>
-              <artifactSet>
-                <includes>
-                  <include>*:*</include>
-                </includes>
-              </artifactSet>
-              <filters>
-                <filter>
-                  <artifact>*:*</artifact>
-                  <excludes>
-                    <exclude>META-INF/*.SF</exclude>
-                    <exclude>META-INF/*.DSA</exclude>
-                    <exclude>META-INF/*.RSA</exclude>
-                  </excludes>
-                </filter>
-              </filters>
-            </configuration>
-          </execution>
-        </executions>
-      </plugin>
-
-      <!-- Avro plugin for automatic code generation -->
-      <plugin>
-        <groupId>org.apache.avro</groupId>
-        <artifactId>avro-maven-plugin</artifactId>
-        <executions>
-          <execution>
-            <id>schemas</id>
-            <phase>generate-sources</phase>
-            <goals>
-              <goal>schema</goal>
-            </goals>
-            <configuration>
-              <sourceDirectory>${project.basedir}/src/main/</sourceDirectory>
-              <outputDirectory>${project.build.directory}/generated-sources/java</outputDirectory>
-            </configuration>
-          </execution>
-        </executions>
-      </plugin>
-
-      <!-- Coverage analysis for unit tests. -->
-      <plugin>
-        <groupId>org.jacoco</groupId>
-        <artifactId>jacoco-maven-plugin</artifactId>
-      </plugin>
-    </plugins>
-  </build>
-
-  <dependencies>
-    <!-- Java SDK -->
-    <dependency>
-      <groupId>org.apache.beam</groupId>
-      <artifactId>beam-sdks-java-core</artifactId>
-    </dependency>
-
-    <!-- IOs -->
-    <dependency>
-      <groupId>org.apache.beam</groupId>
-      <artifactId>beam-sdks-java-io-google-cloud-platform</artifactId>
-    </dependency>
-
-    <dependency>
-      <groupId>org.apache.beam</groupId>
-      <artifactId>beam-sdks-java-extensions-google-cloud-platform-core</artifactId>
-    </dependency>
-
-    <dependency>
-      <groupId>com.google.apis</groupId>
-      <artifactId>google-api-services-bigquery</artifactId>
-    </dependency>
-
-    <!-- Extra libraries -->
-    <dependency>
-      <groupId>com.fasterxml.jackson.core</groupId>
-      <artifactId>jackson-core</artifactId>
-    </dependency>
-
-    <dependency>
-      <groupId>com.fasterxml.jackson.core</groupId>
-      <artifactId>jackson-annotations</artifactId>
-    </dependency>
-
-    <dependency>
-      <groupId>com.fasterxml.jackson.core</groupId>
-      <artifactId>jackson-databind</artifactId>
-    </dependency>
-
-    <dependency>
-      <groupId>org.apache.avro</groupId>
-      <artifactId>avro</artifactId>
-    </dependency>
-
-    <dependency>
-      <groupId>joda-time</groupId>
-      <artifactId>joda-time</artifactId>
-    </dependency>
-
-    <dependency>
-      <groupId>com.google.guava</groupId>
-      <artifactId>guava</artifactId>
-    </dependency>
-
-    <dependency>
-      <groupId>org.slf4j</groupId>
-      <artifactId>slf4j-api</artifactId>
-    </dependency>
-
-    <dependency>
-      <groupId>org.slf4j</groupId>
-      <artifactId>slf4j-jdk14</artifactId>
-      <scope>runtime</scope>
-    </dependency>
-
-    <dependency>
-      <groupId>com.google.code.findbugs</groupId>
-      <artifactId>jsr305</artifactId>
-    </dependency>
-
-    <dependency>
-      <groupId>junit</groupId>
-      <artifactId>junit</artifactId>
-      <scope>compile</scope>
-    </dependency>
-
-    <dependency>
-      <groupId>org.hamcrest</groupId>
-      <artifactId>hamcrest-core</artifactId>
-      <scope>compile</scope>
-    </dependency>
-
-    <dependency>
-      <groupId>org.hamcrest</groupId>
-      <artifactId>hamcrest-all</artifactId>
-    </dependency>
-
-    <!-- Test -->
-    <dependency>
-      <groupId>org.apache.beam</groupId>
-      <artifactId>beam-runners-direct-java</artifactId>
-      <scope>test</scope>
-    </dependency>
-  </dependencies>
-</project>

http://git-wip-us.apache.org/repos/asf/beam/blob/f4333df7/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/Main.java
----------------------------------------------------------------------
diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/Main.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/Main.java
deleted file mode 100644
index 4c23651..0000000
--- a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/Main.java
+++ /dev/null
@@ -1,304 +0,0 @@
-/*
- * 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.beam.integration.nexmark;
-
-import java.io.IOException;
-import java.nio.charset.StandardCharsets;
-import java.nio.file.Files;
-import java.nio.file.Paths;
-import java.nio.file.StandardOpenOption;
-import java.util.ArrayList;
-import java.util.LinkedHashMap;
-import java.util.List;
-import java.util.Map;
-import javax.annotation.Nullable;
-
-import org.apache.beam.integration.nexmark.model.Auction;
-import org.apache.beam.integration.nexmark.model.Bid;
-import org.apache.beam.integration.nexmark.model.Person;
-import org.apache.beam.sdk.options.PipelineOptionsFactory;
-import org.joda.time.Duration;
-import org.joda.time.Instant;
-
-/**
- * An implementation of the 'NEXMark queries' for Beam.
- * These are multiple queries over a three table schema representing an online auction system:
- * <ul>
- * <li>{@link Person} represents a person submitting an item for auction and/or making a bid
- * on an auction.
- * <li>{@link Auction} represents an item under auction.
- * <li>{@link Bid} represents a bid for an item under auction.
- * </ul>
- * The queries exercise many aspects of the Beam model.
- *
- * <p>We synthesize the creation of people, auctions and bids in real-time. The data is not
- * particularly sensible.
- *
- * <p>See <a href="http://datalab.cs.pdx.edu/niagaraST/NEXMark/">
- * http://datalab.cs.pdx.edu/niagaraST/NEXMark/</a>
- */
-public class Main<OptionT extends NexmarkOptions> {
-
-  /**
-   * Entry point.
-   */
-  void runAll(OptionT options, NexmarkLauncher nexmarkLauncher) {
-    Instant start = Instant.now();
-    Map<NexmarkConfiguration, NexmarkPerf> baseline = loadBaseline(options.getBaselineFilename());
-    Map<NexmarkConfiguration, NexmarkPerf> actual = new LinkedHashMap<>();
-    Iterable<NexmarkConfiguration> configurations = options.getSuite().getConfigurations(options);
-
-    boolean successful = true;
-    try {
-      // Run all the configurations.
-      for (NexmarkConfiguration configuration : configurations) {
-        NexmarkPerf perf = nexmarkLauncher.run(configuration);
-        if (perf != null) {
-          if (perf.errors == null || perf.errors.size() > 0) {
-            successful = false;
-          }
-          appendPerf(options.getPerfFilename(), configuration, perf);
-          actual.put(configuration, perf);
-          // Summarize what we've run so far.
-          saveSummary(null, configurations, actual, baseline, start);
-        }
-      }
-    } finally {
-      if (options.getMonitorJobs()) {
-        // Report overall performance.
-        saveSummary(options.getSummaryFilename(), configurations, actual, baseline, start);
-        saveJavascript(options.getJavascriptFilename(), configurations, actual, baseline, start);
-      }
-    }
-
-    if (!successful) {
-      throw new RuntimeException("Execution was not successful");
-    }
-  }
-
-  /**
-   * Append the pair of {@code configuration} and {@code perf} to perf file.
-   */
-  private void appendPerf(
-      @Nullable String perfFilename, NexmarkConfiguration configuration,
-      NexmarkPerf perf) {
-    if (perfFilename == null) {
-      return;
-    }
-    List<String> lines = new ArrayList<>();
-    lines.add("");
-    lines.add(String.format("# %s", Instant.now()));
-    lines.add(String.format("# %s", configuration.toShortString()));
-    lines.add(configuration.toString());
-    lines.add(perf.toString());
-    try {
-      Files.write(Paths.get(perfFilename), lines, StandardCharsets.UTF_8, StandardOpenOption.CREATE,
-          StandardOpenOption.APPEND);
-    } catch (IOException e) {
-      throw new RuntimeException("Unable to write perf file: ", e);
-    }
-    NexmarkUtils.console("appended results to perf file %s.", perfFilename);
-  }
-
-  /**
-   * Load the baseline perf.
-   */
-  @Nullable
-  private static Map<NexmarkConfiguration, NexmarkPerf> loadBaseline(
-      @Nullable String baselineFilename) {
-    if (baselineFilename == null) {
-      return null;
-    }
-    Map<NexmarkConfiguration, NexmarkPerf> baseline = new LinkedHashMap<>();
-    List<String> lines;
-    try {
-      lines = Files.readAllLines(Paths.get(baselineFilename), StandardCharsets.UTF_8);
-    } catch (IOException e) {
-      throw new RuntimeException("Unable to read baseline perf file: ", e);
-    }
-    for (int i = 0; i < lines.size(); i++) {
-      if (lines.get(i).startsWith("#") || lines.get(i).trim().isEmpty()) {
-        continue;
-      }
-      NexmarkConfiguration configuration = NexmarkConfiguration.fromString(lines.get(i++));
-      NexmarkPerf perf = NexmarkPerf.fromString(lines.get(i));
-      baseline.put(configuration, perf);
-    }
-    NexmarkUtils.console("loaded %d entries from baseline file %s.", baseline.size(),
-        baselineFilename);
-    return baseline;
-  }
-
-  private static final String LINE =
-      "==========================================================================================";
-
-  /**
-   * Print summary  of {@code actual} vs (if non-null) {@code baseline}.
-   */
-  private static void saveSummary(
-      @Nullable String summaryFilename,
-      Iterable<NexmarkConfiguration> configurations, Map<NexmarkConfiguration, NexmarkPerf> actual,
-      @Nullable Map<NexmarkConfiguration, NexmarkPerf> baseline, Instant start) {
-    List<String> lines = new ArrayList<>();
-
-    lines.add("");
-    lines.add(LINE);
-
-    lines.add(
-        String.format("Run started %s and ran for %s", start, new Duration(start, Instant.now())));
-    lines.add("");
-
-    lines.add("Default configuration:");
-    lines.add(NexmarkConfiguration.DEFAULT.toString());
-    lines.add("");
-
-    lines.add("Configurations:");
-    lines.add("  Conf  Description");
-    int conf = 0;
-    for (NexmarkConfiguration configuration : configurations) {
-      lines.add(String.format("  %04d  %s", conf++, configuration.toShortString()));
-      NexmarkPerf actualPerf = actual.get(configuration);
-      if (actualPerf != null && actualPerf.jobId != null) {
-        lines.add(String.format("  %4s  [Ran as job %s]", "", actualPerf.jobId));
-      }
-    }
-
-    lines.add("");
-    lines.add("Performance:");
-    lines.add(String.format("  %4s  %12s  %12s  %12s  %12s  %12s  %12s", "Conf", "Runtime(sec)",
-        "(Baseline)", "Events(/sec)", "(Baseline)", "Results", "(Baseline)"));
-    conf = 0;
-    for (NexmarkConfiguration configuration : configurations) {
-      String line = String.format("  %04d  ", conf++);
-      NexmarkPerf actualPerf = actual.get(configuration);
-      if (actualPerf == null) {
-        line += "*** not run ***";
-      } else {
-        NexmarkPerf baselinePerf = baseline == null ? null : baseline.get(configuration);
-        double runtimeSec = actualPerf.runtimeSec;
-        line += String.format("%12.1f  ", runtimeSec);
-        if (baselinePerf == null) {
-          line += String.format("%12s  ", "");
-        } else {
-          double baselineRuntimeSec = baselinePerf.runtimeSec;
-          double diff = ((runtimeSec - baselineRuntimeSec) / baselineRuntimeSec) * 100.0;
-          line += String.format("%+11.2f%%  ", diff);
-        }
-
-        double eventsPerSec = actualPerf.eventsPerSec;
-        line += String.format("%12.1f  ", eventsPerSec);
-        if (baselinePerf == null) {
-          line += String.format("%12s  ", "");
-        } else {
-          double baselineEventsPerSec = baselinePerf.eventsPerSec;
-          double diff = ((eventsPerSec - baselineEventsPerSec) / baselineEventsPerSec) * 100.0;
-          line += String.format("%+11.2f%%  ", diff);
-        }
-
-        long numResults = actualPerf.numResults;
-        line += String.format("%12d  ", numResults);
-        if (baselinePerf == null) {
-          line += String.format("%12s", "");
-        } else {
-          long baselineNumResults = baselinePerf.numResults;
-          long diff = numResults - baselineNumResults;
-          line += String.format("%+12d", diff);
-        }
-      }
-      lines.add(line);
-
-      if (actualPerf != null) {
-        List<String> errors = actualPerf.errors;
-        if (errors == null) {
-          errors = new ArrayList<>();
-          errors.add("NexmarkGoogleRunner returned null errors list");
-        }
-        for (String error : errors) {
-          lines.add(String.format("  %4s  *** %s ***", "", error));
-        }
-      }
-    }
-
-    lines.add(LINE);
-    lines.add("");
-
-    for (String line : lines) {
-      System.out.println(line);
-    }
-
-    if (summaryFilename != null) {
-      try {
-        Files.write(Paths.get(summaryFilename), lines, StandardCharsets.UTF_8,
-            StandardOpenOption.CREATE, StandardOpenOption.APPEND);
-      } catch (IOException e) {
-        throw new RuntimeException("Unable to save summary file: ", e);
-      }
-      NexmarkUtils.console("appended summary to summary file %s.", summaryFilename);
-    }
-  }
-
-  /**
-   * Write all perf data and any baselines to a javascript file which can be used by
-   * graphing page etc.
-   */
-  private static void saveJavascript(
-      @Nullable String javascriptFilename,
-      Iterable<NexmarkConfiguration> configurations, Map<NexmarkConfiguration, NexmarkPerf> actual,
-      @Nullable Map<NexmarkConfiguration, NexmarkPerf> baseline, Instant start) {
-    if (javascriptFilename == null) {
-      return;
-    }
-
-    List<String> lines = new ArrayList<>();
-    lines.add(String.format(
-        "// Run started %s and ran for %s", start, new Duration(start, Instant.now())));
-    lines.add("var all = [");
-
-    for (NexmarkConfiguration configuration : configurations) {
-      lines.add("  {");
-      lines.add(String.format("    config: %s", configuration));
-      NexmarkPerf actualPerf = actual.get(configuration);
-      if (actualPerf != null) {
-        lines.add(String.format("    ,perf: %s", actualPerf));
-      }
-      NexmarkPerf baselinePerf = baseline == null ? null : baseline.get(configuration);
-      if (baselinePerf != null) {
-        lines.add(String.format("    ,baseline: %s", baselinePerf));
-      }
-      lines.add("  },");
-    }
-
-    lines.add("];");
-
-    try {
-      Files.write(Paths.get(javascriptFilename), lines, StandardCharsets.UTF_8,
-          StandardOpenOption.CREATE, StandardOpenOption.TRUNCATE_EXISTING);
-    } catch (IOException e) {
-      throw new RuntimeException("Unable to save javascript file: ", e);
-    }
-    NexmarkUtils.console("saved javascript to file %s.", javascriptFilename);
-  }
-
-  public static void main(String[] args) {
-    NexmarkOptions options = PipelineOptionsFactory.fromArgs(args)
-      .withValidation()
-      .as(NexmarkOptions.class);
-    NexmarkLauncher<NexmarkOptions> nexmarkLauncher = new NexmarkLauncher<>(options);
-    new Main<>().runAll(options, nexmarkLauncher);
-  }
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/f4333df7/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/Monitor.java
----------------------------------------------------------------------
diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/Monitor.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/Monitor.java
deleted file mode 100644
index 2f0c56a..0000000
--- a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/Monitor.java
+++ /dev/null
@@ -1,79 +0,0 @@
-/*
- * 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.beam.integration.nexmark;
-
-import java.io.Serializable;
-
-import org.apache.beam.integration.nexmark.model.KnownSize;
-import org.apache.beam.sdk.metrics.Counter;
-import org.apache.beam.sdk.metrics.Distribution;
-import org.apache.beam.sdk.metrics.Metrics;
-import org.apache.beam.sdk.transforms.DoFn;
-import org.apache.beam.sdk.transforms.PTransform;
-import org.apache.beam.sdk.transforms.ParDo;
-import org.apache.beam.sdk.values.PCollection;
-
-/**
- * A monitor of elements with support for later retrieving their metrics.
- *
- * @param <T> Type of element we are monitoring.
- */
-public class Monitor<T extends KnownSize> implements Serializable {
-  private class MonitorDoFn extends DoFn<T, T> {
-    final Counter elementCounter =
-      Metrics.counter(name , prefix + ".elements");
-    final Counter bytesCounter =
-      Metrics.counter(name , prefix + ".bytes");
-    final Distribution startTime =
-      Metrics.distribution(name , prefix + ".startTime");
-    final Distribution endTime =
-      Metrics.distribution(name , prefix + ".endTime");
-    final Distribution startTimestamp =
-      Metrics.distribution(name , prefix + ".startTimestamp");
-    final Distribution endTimestamp =
-      Metrics.distribution(name , prefix + ".endTimestamp");
-
-    @ProcessElement
-    public void processElement(ProcessContext c) {
-      elementCounter.inc();
-      bytesCounter.inc(c.element().sizeInBytes());
-      long now = System.currentTimeMillis();
-      startTime.update(now);
-      endTime.update(now);
-      startTimestamp.update(c.timestamp().getMillis());
-      endTimestamp.update(c.timestamp().getMillis());
-      c.output(c.element());
-    }
-  }
-
-  public final String name;
-  public final String prefix;
-  private final MonitorDoFn doFn;
-  private final PTransform<PCollection<? extends T>, PCollection<T>> transform;
-
-  public Monitor(String name, String prefix) {
-    this.name = name;
-    this.prefix = prefix;
-    doFn = new MonitorDoFn();
-    transform = ParDo.of(doFn);
-  }
-
-  public PTransform<PCollection<? extends T>, PCollection<T>> getTransform() {
-    return transform;
-  }
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/f4333df7/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkConfiguration.java
----------------------------------------------------------------------
diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkConfiguration.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkConfiguration.java
deleted file mode 100644
index 2faf3f5..0000000
--- a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkConfiguration.java
+++ /dev/null
@@ -1,721 +0,0 @@
-/*
- * 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.beam.integration.nexmark;
-
-import com.fasterxml.jackson.annotation.JsonProperty;
-import com.fasterxml.jackson.core.JsonProcessingException;
-import java.io.IOException;
-import java.io.Serializable;
-import java.util.Objects;
-
-/**
- * Configuration controlling how a query is run. May be supplied by command line or
- * programmatically. We only capture properties which may influence the resulting
- * pipeline performance, as captured by {@link NexmarkPerf}.
- */
-public class NexmarkConfiguration implements Serializable {
-  public static final NexmarkConfiguration DEFAULT = new NexmarkConfiguration();
-
-  /** If {@literal true}, include additional debugging and monitoring stats. */
-  @JsonProperty
-  public boolean debug = true;
-
-  /** Which query to run, in [0,9]. */
-  @JsonProperty
-  public int query = 0;
-
-  /** Where events come from. */
-  @JsonProperty
-  public NexmarkUtils.SourceType sourceType = NexmarkUtils.SourceType.DIRECT;
-
-  /** Where results go to. */
-  @JsonProperty
-  public NexmarkUtils.SinkType sinkType = NexmarkUtils.SinkType.DEVNULL;
-
-  /**
-   * Control whether pub/sub publishing is done in a stand-alone pipeline or is integrated
-   * into the overall query pipeline.
-   */
-  @JsonProperty
-  public NexmarkUtils.PubSubMode pubSubMode = NexmarkUtils.PubSubMode.COMBINED;
-
-  /**
-   * Number of events to generate. If zero, generate as many as possible without overflowing
-   * internal counters etc.
-   */
-  @JsonProperty
-  public long numEvents = 100000;
-
-  /**
-   * Number of event generators to use. Each generates events in its own timeline.
-   */
-  @JsonProperty
-  public int numEventGenerators = 100;
-
-  /**
-   * Shape of event rate curve.
-   */
-  @JsonProperty
-  public NexmarkUtils.RateShape rateShape = NexmarkUtils.RateShape.SINE;
-
-  /**
-   * Initial overall event rate (in {@link #rateUnit}).
-   */
-  @JsonProperty
-  public int firstEventRate = 10000;
-
-  /**
-   * Next overall event rate (in {@link #rateUnit}).
-   */
-  @JsonProperty
-  public int nextEventRate = 10000;
-
-  /**
-   * Unit for rates.
-   */
-  @JsonProperty
-  public NexmarkUtils.RateUnit rateUnit = NexmarkUtils.RateUnit.PER_SECOND;
-
-  /**
-   * Overall period of rate shape, in seconds.
-   */
-  @JsonProperty
-  public int ratePeriodSec = 600;
-
-  /**
-   * Time in seconds to preload the subscription with data, at the initial input rate of the
-   * pipeline.
-   */
-  @JsonProperty
-  public int preloadSeconds = 0;
-
-  /**
-   * Timeout for stream pipelines to stop in seconds.
-   */
-  @JsonProperty
-  public int streamTimeout = 240;
-
-  /**
-   * If true, and in streaming mode, generate events only when they are due according to their
-   * timestamp.
-   */
-  @JsonProperty
-  public boolean isRateLimited = false;
-
-  /**
-   * If true, use wallclock time as event time. Otherwise, use a deterministic
-   * time in the past so that multiple runs will see exactly the same event streams
-   * and should thus have exactly the same results.
-   */
-  @JsonProperty
-  public boolean useWallclockEventTime = false;
-
-  /** Average idealized size of a 'new person' event, in bytes. */
-  @JsonProperty
-  public int avgPersonByteSize = 200;
-
-  /** Average idealized size of a 'new auction' event, in bytes. */
-  @JsonProperty
-  public int avgAuctionByteSize = 500;
-
-  /** Average idealized size of a 'bid' event, in bytes. */
-  @JsonProperty
-  public int avgBidByteSize = 100;
-
-  /** Ratio of bids to 'hot' auctions compared to all other auctions. */
-  @JsonProperty
-  public int hotAuctionRatio = 2;
-
-  /** Ratio of auctions for 'hot' sellers compared to all other people. */
-  @JsonProperty
-  public int hotSellersRatio = 4;
-
-  /** Ratio of bids for 'hot' bidders compared to all other people. */
-  @JsonProperty
-  public int hotBiddersRatio = 4;
-
-  /** Window size, in seconds, for queries 3, 5, 7 and 8. */
-  @JsonProperty
-  public long windowSizeSec = 10;
-
-  /** Sliding window period, in seconds, for query 5. */
-  @JsonProperty
-  public long windowPeriodSec = 5;
-
-  /** Number of seconds to hold back events according to their reported timestamp. */
-  @JsonProperty
-  public long watermarkHoldbackSec = 0;
-
-  /** Average number of auction which should be inflight at any time, per generator. */
-  @JsonProperty
-  public int numInFlightAuctions = 100;
-
-  /** Maximum number of people to consider as active for placing auctions or bids. */
-  @JsonProperty
-  public int numActivePeople = 1000;
-
-  /** Coder strategy to follow. */
-  @JsonProperty
-  public NexmarkUtils.CoderStrategy coderStrategy = NexmarkUtils.CoderStrategy.HAND;
-
-  /**
-   * Delay, in milliseconds, for each event. This will peg one core for this number
-   * of milliseconds to simulate CPU-bound computation.
-   */
-  @JsonProperty
-  public long cpuDelayMs = 0;
-
-  /**
-   * Extra data, in bytes, to save to persistent state for each event. This will force
-   * i/o all the way to durable storage to simulate an I/O-bound computation.
-   */
-  @JsonProperty
-  public long diskBusyBytes = 0;
-
-  /**
-   * Skip factor for query 2. We select bids for every {@code auctionSkip}'th auction.
-   */
-  @JsonProperty
-  public int auctionSkip = 123;
-
-  /**
-   * Fanout for queries 4 (groups by category id), 5 and 7 (find a global maximum).
-   */
-  @JsonProperty
-  public int fanout = 5;
-
-  /**
-   * Maximum waiting time to clean personState in query3
-   * (ie maximum waiting of the auctions related to person in state in seconds in event time).
-   */
-  @JsonProperty
-  public int maxAuctionsWaitingTime = 600;
-
-  /**
-   * Length of occasional delay to impose on events (in seconds).
-   */
-  @JsonProperty
-  public long occasionalDelaySec = 3;
-
-  /**
-   * Probability that an event will be delayed by delayS.
-   */
-  @JsonProperty
-  public double probDelayedEvent = 0.1;
-
-  /**
-   * Maximum size of each log file (in events). For Query10 only.
-   */
-  @JsonProperty
-  public int maxLogEvents = 100_000;
-
-  /**
-   * If true, use pub/sub publish time instead of event time.
-   */
-  @JsonProperty
-  public boolean usePubsubPublishTime = false;
-
-  /**
-   * Number of events in out-of-order groups. 1 implies no out-of-order events. 1000 implies
-   * every 1000 events per generator are emitted in pseudo-random order.
-   */
-  @JsonProperty
-  public long outOfOrderGroupSize = 1;
-
-  /**
-   * Replace any properties of this configuration which have been supplied by the command line.
-   */
-  public void overrideFromOptions(NexmarkOptions options) {
-    if (options.getDebug() != null) {
-      debug = options.getDebug();
-    }
-    if (options.getQuery() != null) {
-      query = options.getQuery();
-    }
-    if (options.getSourceType() != null) {
-      sourceType = options.getSourceType();
-    }
-    if (options.getSinkType() != null) {
-      sinkType = options.getSinkType();
-    }
-    if (options.getPubSubMode() != null) {
-      pubSubMode = options.getPubSubMode();
-    }
-    if (options.getNumEvents() != null) {
-      numEvents = options.getNumEvents();
-    }
-    if (options.getNumEventGenerators() != null) {
-      numEventGenerators = options.getNumEventGenerators();
-    }
-    if (options.getRateShape() != null) {
-      rateShape = options.getRateShape();
-    }
-    if (options.getFirstEventRate() != null) {
-      firstEventRate = options.getFirstEventRate();
-    }
-    if (options.getNextEventRate() != null) {
-      nextEventRate = options.getNextEventRate();
-    }
-    if (options.getRateUnit() != null) {
-      rateUnit = options.getRateUnit();
-    }
-    if (options.getRatePeriodSec() != null) {
-      ratePeriodSec = options.getRatePeriodSec();
-    }
-    if (options.getPreloadSeconds() != null) {
-      preloadSeconds = options.getPreloadSeconds();
-    }
-    if (options.getStreamTimeout() != null) {
-      streamTimeout = options.getStreamTimeout();
-    }
-    if (options.getIsRateLimited() != null) {
-      isRateLimited = options.getIsRateLimited();
-    }
-    if (options.getUseWallclockEventTime() != null) {
-      useWallclockEventTime = options.getUseWallclockEventTime();
-    }
-    if (options.getAvgPersonByteSize() != null) {
-      avgPersonByteSize = options.getAvgPersonByteSize();
-    }
-    if (options.getAvgAuctionByteSize() != null) {
-      avgAuctionByteSize = options.getAvgAuctionByteSize();
-    }
-    if (options.getAvgBidByteSize() != null) {
-      avgBidByteSize = options.getAvgBidByteSize();
-    }
-    if (options.getHotAuctionRatio() != null) {
-      hotAuctionRatio = options.getHotAuctionRatio();
-    }
-    if (options.getHotSellersRatio() != null) {
-      hotSellersRatio = options.getHotSellersRatio();
-    }
-    if (options.getHotBiddersRatio() != null) {
-      hotBiddersRatio = options.getHotBiddersRatio();
-    }
-    if (options.getWindowSizeSec() != null) {
-      windowSizeSec = options.getWindowSizeSec();
-    }
-    if (options.getWindowPeriodSec() != null) {
-      windowPeriodSec = options.getWindowPeriodSec();
-    }
-    if (options.getWatermarkHoldbackSec() != null) {
-      watermarkHoldbackSec = options.getWatermarkHoldbackSec();
-    }
-    if (options.getNumInFlightAuctions() != null) {
-      numInFlightAuctions = options.getNumInFlightAuctions();
-    }
-    if (options.getNumActivePeople() != null) {
-      numActivePeople = options.getNumActivePeople();
-    }
-    if (options.getCoderStrategy() != null) {
-      coderStrategy = options.getCoderStrategy();
-    }
-    if (options.getCpuDelayMs() != null) {
-      cpuDelayMs = options.getCpuDelayMs();
-    }
-    if (options.getDiskBusyBytes() != null) {
-      diskBusyBytes = options.getDiskBusyBytes();
-    }
-    if (options.getAuctionSkip() != null) {
-      auctionSkip = options.getAuctionSkip();
-    }
-    if (options.getFanout() != null) {
-      fanout = options.getFanout();
-    }
-    if (options.getMaxAuctionsWaitingTime() != null) {
-      fanout = options.getMaxAuctionsWaitingTime();
-    }
-    if (options.getOccasionalDelaySec() != null) {
-      occasionalDelaySec = options.getOccasionalDelaySec();
-    }
-    if (options.getProbDelayedEvent() != null) {
-      probDelayedEvent = options.getProbDelayedEvent();
-    }
-    if (options.getMaxLogEvents() != null) {
-      maxLogEvents = options.getMaxLogEvents();
-    }
-    if (options.getUsePubsubPublishTime() != null) {
-      usePubsubPublishTime = options.getUsePubsubPublishTime();
-    }
-    if (options.getOutOfOrderGroupSize() != null) {
-      outOfOrderGroupSize = options.getOutOfOrderGroupSize();
-    }
-  }
-
-  /**
-   * Return copy of configuration with given label.
-   */
-  public NexmarkConfiguration copy() {
-    NexmarkConfiguration result;
-    result = new NexmarkConfiguration();
-    result.debug = debug;
-    result.query = query;
-    result.sourceType = sourceType;
-    result.sinkType = sinkType;
-    result.pubSubMode = pubSubMode;
-    result.numEvents = numEvents;
-    result.numEventGenerators = numEventGenerators;
-    result.rateShape = rateShape;
-    result.firstEventRate = firstEventRate;
-    result.nextEventRate = nextEventRate;
-    result.rateUnit = rateUnit;
-    result.ratePeriodSec = ratePeriodSec;
-    result.preloadSeconds = preloadSeconds;
-    result.streamTimeout = streamTimeout;
-    result.isRateLimited = isRateLimited;
-    result.useWallclockEventTime = useWallclockEventTime;
-    result.avgPersonByteSize = avgPersonByteSize;
-    result.avgAuctionByteSize = avgAuctionByteSize;
-    result.avgBidByteSize = avgBidByteSize;
-    result.hotAuctionRatio = hotAuctionRatio;
-    result.hotSellersRatio = hotSellersRatio;
-    result.hotBiddersRatio = hotBiddersRatio;
-    result.windowSizeSec = windowSizeSec;
-    result.windowPeriodSec = windowPeriodSec;
-    result.watermarkHoldbackSec = watermarkHoldbackSec;
-    result.numInFlightAuctions = numInFlightAuctions;
-    result.numActivePeople = numActivePeople;
-    result.coderStrategy = coderStrategy;
-    result.cpuDelayMs = cpuDelayMs;
-    result.diskBusyBytes = diskBusyBytes;
-    result.auctionSkip = auctionSkip;
-    result.fanout = fanout;
-    result.maxAuctionsWaitingTime = maxAuctionsWaitingTime;
-    result.occasionalDelaySec = occasionalDelaySec;
-    result.probDelayedEvent = probDelayedEvent;
-    result.maxLogEvents = maxLogEvents;
-    result.usePubsubPublishTime = usePubsubPublishTime;
-    result.outOfOrderGroupSize = outOfOrderGroupSize;
-    return result;
-  }
-
-  /**
-   * Return short description of configuration (suitable for use in logging). We only render
-   * the core fields plus those which do not have default values.
-   */
-  public String toShortString() {
-    StringBuilder sb = new StringBuilder();
-    sb.append(String.format("query:%d", query));
-    if (debug != DEFAULT.debug) {
-      sb.append(String.format("; debug:%s", debug));
-    }
-    if (sourceType != DEFAULT.sourceType) {
-      sb.append(String.format("; sourceType:%s", sourceType));
-    }
-    if (sinkType != DEFAULT.sinkType) {
-      sb.append(String.format("; sinkType:%s", sinkType));
-    }
-    if (pubSubMode != DEFAULT.pubSubMode) {
-      sb.append(String.format("; pubSubMode:%s", pubSubMode));
-    }
-    if (numEvents != DEFAULT.numEvents) {
-      sb.append(String.format("; numEvents:%d", numEvents));
-    }
-    if (numEventGenerators != DEFAULT.numEventGenerators) {
-      sb.append(String.format("; numEventGenerators:%d", numEventGenerators));
-    }
-    if (rateShape != DEFAULT.rateShape) {
-      sb.append(String.format("; rateShape:%s", rateShape));
-    }
-    if (firstEventRate != DEFAULT.firstEventRate || nextEventRate != DEFAULT.nextEventRate) {
-      sb.append(String.format("; firstEventRate:%d", firstEventRate));
-      sb.append(String.format("; nextEventRate:%d", nextEventRate));
-    }
-    if (rateUnit != DEFAULT.rateUnit) {
-      sb.append(String.format("; rateUnit:%s", rateUnit));
-    }
-    if (ratePeriodSec != DEFAULT.ratePeriodSec) {
-      sb.append(String.format("; ratePeriodSec:%d", ratePeriodSec));
-    }
-    if (preloadSeconds != DEFAULT.preloadSeconds) {
-      sb.append(String.format("; preloadSeconds:%d", preloadSeconds));
-    }
-    if (streamTimeout != DEFAULT.streamTimeout) {
-      sb.append(String.format("; streamTimeout:%d", streamTimeout));
-    }
-    if (isRateLimited != DEFAULT.isRateLimited) {
-      sb.append(String.format("; isRateLimited:%s", isRateLimited));
-    }
-    if (useWallclockEventTime != DEFAULT.useWallclockEventTime) {
-      sb.append(String.format("; useWallclockEventTime:%s", useWallclockEventTime));
-    }
-    if (avgPersonByteSize != DEFAULT.avgPersonByteSize) {
-      sb.append(String.format("; avgPersonByteSize:%d", avgPersonByteSize));
-    }
-    if (avgAuctionByteSize != DEFAULT.avgAuctionByteSize) {
-      sb.append(String.format("; avgAuctionByteSize:%d", avgAuctionByteSize));
-    }
-    if (avgBidByteSize != DEFAULT.avgBidByteSize) {
-      sb.append(String.format("; avgBidByteSize:%d", avgBidByteSize));
-    }
-    if (hotAuctionRatio != DEFAULT.hotAuctionRatio) {
-      sb.append(String.format("; hotAuctionRatio:%d", hotAuctionRatio));
-    }
-    if (hotSellersRatio != DEFAULT.hotSellersRatio) {
-      sb.append(String.format("; hotSellersRatio:%d", hotSellersRatio));
-    }
-    if (hotBiddersRatio != DEFAULT.hotBiddersRatio) {
-      sb.append(String.format("; hotBiddersRatio:%d", hotBiddersRatio));
-    }
-    if (windowSizeSec != DEFAULT.windowSizeSec) {
-      sb.append(String.format("; windowSizeSec:%d", windowSizeSec));
-    }
-    if (windowPeriodSec != DEFAULT.windowPeriodSec) {
-      sb.append(String.format("; windowPeriodSec:%d", windowPeriodSec));
-    }
-    if (watermarkHoldbackSec != DEFAULT.watermarkHoldbackSec) {
-      sb.append(String.format("; watermarkHoldbackSec:%d", watermarkHoldbackSec));
-    }
-    if (numInFlightAuctions != DEFAULT.numInFlightAuctions) {
-      sb.append(String.format("; numInFlightAuctions:%d", numInFlightAuctions));
-    }
-    if (numActivePeople != DEFAULT.numActivePeople) {
-      sb.append(String.format("; numActivePeople:%d", numActivePeople));
-    }
-    if (coderStrategy != DEFAULT.coderStrategy) {
-      sb.append(String.format("; coderStrategy:%s", coderStrategy));
-    }
-    if (cpuDelayMs != DEFAULT.cpuDelayMs) {
-      sb.append(String.format("; cpuSlowdownMs:%d", cpuDelayMs));
-    }
-    if (diskBusyBytes != DEFAULT.diskBusyBytes) {
-      sb.append(String.format("; diskBuysBytes:%d", diskBusyBytes));
-    }
-    if (auctionSkip != DEFAULT.auctionSkip) {
-      sb.append(String.format("; auctionSkip:%d", auctionSkip));
-    }
-    if (fanout != DEFAULT.fanout) {
-      sb.append(String.format("; fanout:%d", fanout));
-    }
-    if (maxAuctionsWaitingTime != DEFAULT.maxAuctionsWaitingTime) {
-      sb.append(String.format("; maxAuctionsWaitingTime:%d", fanout));
-    }
-    if (occasionalDelaySec != DEFAULT.occasionalDelaySec) {
-      sb.append(String.format("; occasionalDelaySec:%d", occasionalDelaySec));
-    }
-    if (probDelayedEvent != DEFAULT.probDelayedEvent) {
-      sb.append(String.format("; probDelayedEvent:%f", probDelayedEvent));
-    }
-    if (maxLogEvents != DEFAULT.maxLogEvents) {
-      sb.append(String.format("; maxLogEvents:%d", maxLogEvents));
-    }
-    if (usePubsubPublishTime != DEFAULT.usePubsubPublishTime) {
-      sb.append(String.format("; usePubsubPublishTime:%s", usePubsubPublishTime));
-    }
-    if (outOfOrderGroupSize != DEFAULT.outOfOrderGroupSize) {
-      sb.append(String.format("; outOfOrderGroupSize:%d", outOfOrderGroupSize));
-    }
-    return sb.toString();
-  }
-
-  /**
-   * Return full description as a string.
-   */
-  @Override
-  public String toString() {
-    try {
-      return NexmarkUtils.MAPPER.writeValueAsString(this);
-    } catch (JsonProcessingException e) {
-      throw new RuntimeException(e);
-    }
-  }
-
-  /**
-   * Parse an object from {@code string}.
-   */
-  public static NexmarkConfiguration fromString(String string) {
-    try {
-      return NexmarkUtils.MAPPER.readValue(string, NexmarkConfiguration.class);
-    } catch (IOException e) {
-      throw new RuntimeException("Unable to parse nexmark configuration: ", e);
-    }
-  }
-
-  @Override
-  public int hashCode() {
-    return Objects.hash(
-        debug,
-        query,
-        sourceType,
-        sinkType,
-        pubSubMode,
-        numEvents,
-        numEventGenerators,
-        rateShape,
-        firstEventRate,
-        nextEventRate,
-        rateUnit,
-        ratePeriodSec,
-        preloadSeconds,
-        streamTimeout,
-        isRateLimited,
-        useWallclockEventTime,
-        avgPersonByteSize,
-        avgAuctionByteSize,
-        avgBidByteSize,
-        hotAuctionRatio,
-        hotSellersRatio,
-        hotBiddersRatio,
-        windowSizeSec,
-        windowPeriodSec,
-        watermarkHoldbackSec,
-        numInFlightAuctions,
-        numActivePeople,
-        coderStrategy,
-        cpuDelayMs,
-        diskBusyBytes,
-        auctionSkip,
-        fanout,
-        maxAuctionsWaitingTime,
-        occasionalDelaySec,
-        probDelayedEvent,
-        maxLogEvents,
-        usePubsubPublishTime,
-        outOfOrderGroupSize);
-  }
-
-  @Override
-  public boolean equals(Object obj) {
-    if (this == obj) {
-      return true;
-    }
-    if (obj == null) {
-      return false;
-    }
-    if (getClass() != obj.getClass()) {
-      return false;
-    }
-    NexmarkConfiguration other = (NexmarkConfiguration) obj;
-    if (debug != other.debug) {
-      return false;
-    }
-    if (auctionSkip != other.auctionSkip) {
-      return false;
-    }
-    if (avgAuctionByteSize != other.avgAuctionByteSize) {
-      return false;
-    }
-    if (avgBidByteSize != other.avgBidByteSize) {
-      return false;
-    }
-    if (avgPersonByteSize != other.avgPersonByteSize) {
-      return false;
-    }
-    if (coderStrategy != other.coderStrategy) {
-      return false;
-    }
-    if (cpuDelayMs != other.cpuDelayMs) {
-      return false;
-    }
-    if (diskBusyBytes != other.diskBusyBytes) {
-      return false;
-    }
-    if (fanout != other.fanout) {
-      return false;
-    }
-    if (maxAuctionsWaitingTime != other.maxAuctionsWaitingTime) {
-      return false;
-    }
-    if (firstEventRate != other.firstEventRate) {
-      return false;
-    }
-    if (hotAuctionRatio != other.hotAuctionRatio) {
-      return false;
-    }
-    if (hotBiddersRatio != other.hotBiddersRatio) {
-      return false;
-    }
-    if (hotSellersRatio != other.hotSellersRatio) {
-      return false;
-    }
-    if (isRateLimited != other.isRateLimited) {
-      return false;
-    }
-    if (maxLogEvents != other.maxLogEvents) {
-      return false;
-    }
-    if (nextEventRate != other.nextEventRate) {
-      return false;
-    }
-    if (rateUnit != other.rateUnit) {
-      return false;
-    }
-    if (numEventGenerators != other.numEventGenerators) {
-      return false;
-    }
-    if (numEvents != other.numEvents) {
-      return false;
-    }
-    if (numInFlightAuctions != other.numInFlightAuctions) {
-      return false;
-    }
-    if (numActivePeople != other.numActivePeople) {
-      return false;
-    }
-    if (occasionalDelaySec != other.occasionalDelaySec) {
-      return false;
-    }
-    if (preloadSeconds != other.preloadSeconds) {
-      return false;
-    }
-    if (streamTimeout != other.streamTimeout) {
-      return false;
-    }
-    if (Double.doubleToLongBits(probDelayedEvent)
-        != Double.doubleToLongBits(other.probDelayedEvent)) {
-      return false;
-    }
-    if (pubSubMode != other.pubSubMode) {
-      return false;
-    }
-    if (ratePeriodSec != other.ratePeriodSec) {
-      return false;
-    }
-    if (rateShape != other.rateShape) {
-      return false;
-    }
-    if (query != other.query) {
-      return false;
-    }
-    if (sinkType != other.sinkType) {
-      return false;
-    }
-    if (sourceType != other.sourceType) {
-      return false;
-    }
-    if (useWallclockEventTime != other.useWallclockEventTime) {
-      return false;
-    }
-    if (watermarkHoldbackSec != other.watermarkHoldbackSec) {
-      return false;
-    }
-    if (windowPeriodSec != other.windowPeriodSec) {
-      return false;
-    }
-    if (windowSizeSec != other.windowSizeSec) {
-      return false;
-    }
-    if (usePubsubPublishTime != other.usePubsubPublishTime) {
-      return false;
-    }
-    if (outOfOrderGroupSize != other.outOfOrderGroupSize) {
-      return false;
-    }
-    return true;
-  }
-}


[54/55] [abbrv] beam git commit: Improve NexmarkUtils: improve diskBusy() and remove unneeded randomization code

Posted by ie...@apache.org.
Improve NexmarkUtils: improve diskBusy() and remove unneeded randomization code

- Use state API in NexmarkUtils.diskBusy()
- Remove commented code for direct runner randomization disabling: direct runner no more allows disabling randomization and queries and UT pass


Project: http://git-wip-us.apache.org/repos/asf/beam/repo
Commit: http://git-wip-us.apache.org/repos/asf/beam/commit/6c116709
Tree: http://git-wip-us.apache.org/repos/asf/beam/tree/6c116709
Diff: http://git-wip-us.apache.org/repos/asf/beam/diff/6c116709

Branch: refs/heads/master
Commit: 6c116709fff06f7faa491a090f441f618931d256
Parents: ee500b2
Author: Etienne Chauchot <ec...@gmail.com>
Authored: Tue May 30 18:00:00 2017 +0100
Committer: Ismaël Mejía <ie...@gmail.com>
Committed: Wed Aug 23 19:07:29 2017 +0200

----------------------------------------------------------------------
 .../beam/integration/nexmark/NexmarkUtils.java  | 87 ++++++++++++--------
 .../nexmark/queries/NexmarkQuery.java           |  2 +-
 2 files changed, 52 insertions(+), 37 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/beam/blob/6c116709/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkUtils.java
----------------------------------------------------------------------
diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkUtils.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkUtils.java
index 7707429..7926690 100644
--- a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkUtils.java
+++ b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkUtils.java
@@ -18,14 +18,12 @@
 package org.apache.beam.integration.nexmark;
 
 import com.fasterxml.jackson.databind.ObjectMapper;
-import com.google.common.collect.ImmutableList;
 import com.google.common.hash.Hashing;
 import java.io.IOException;
 import java.io.InputStream;
 import java.io.OutputStream;
 import java.nio.charset.StandardCharsets;
 import java.util.Iterator;
-import java.util.List;
 import org.apache.beam.integration.nexmark.model.Auction;
 import org.apache.beam.integration.nexmark.model.AuctionBid;
 import org.apache.beam.integration.nexmark.model.AuctionCount;
@@ -66,6 +64,7 @@ import org.apache.beam.sdk.transforms.windowing.AfterPane;
 import org.apache.beam.sdk.transforms.windowing.BoundedWindow;
 import org.apache.beam.sdk.transforms.windowing.GlobalWindows;
 import org.apache.beam.sdk.transforms.windowing.Window;
+import org.apache.beam.sdk.values.KV;
 import org.apache.beam.sdk.values.PBegin;
 import org.apache.beam.sdk.values.PCollection;
 import org.apache.beam.sdk.values.TimestampedValue;
@@ -289,8 +288,8 @@ public class NexmarkUtils {
   private static final boolean LOG_ERROR = true;
 
   /**
-   * Set to true to log directly to stdout. If run using Google Dataflow, you can watch the results in real-time with:
-   * tail -f /var/log/dataflow/streaming-harness/harness-stdout.log
+   * Set to true to log directly to stdout. If run using Google Dataflow, you can watch the results
+   * in real-time with: tail -f /var/log/dataflow/streaming-harness/harness-stdout.log
    */
   private static final boolean LOG_TO_CONSOLE = false;
 
@@ -340,14 +339,6 @@ public class NexmarkUtils {
    * Setup pipeline with codes and some other options.
    */
   public static void setupPipeline(CoderStrategy coderStrategy, Pipeline p) {
-    //TODO Ismael check
-//    PipelineRunner<?> runner = p.getRunner();
-//    if (runner instanceof DirectRunner) {
-//      // Disable randomization of output since we want to check batch and streaming match the
-//      // model both locally and on the cloud.
-//      ((DirectRunner) runner).withUnorderednessTesting(false);
-//    }
-
     CoderRegistry registry = p.getCoderRegistry();
     switch (coderStrategy) {
       case HAND:
@@ -565,35 +556,59 @@ public class NexmarkUtils {
                 });
   }
 
-  private static final StateSpec<ValueState<byte[]>> DUMMY_TAG =
-          StateSpecs.value(ByteArrayCoder.of());
   private static final int MAX_BUFFER_SIZE = 1 << 24;
 
+  private static class DiskBusyTransform<T> extends PTransform<PCollection<T>, PCollection<T>>{
+
+    private long bytes;
+
+    private DiskBusyTransform(long bytes) {
+      this.bytes = bytes;
+    }
+
+    @Override public PCollection<T> expand(PCollection<T> input) {
+      // Add dummy key to be able to use State API
+      PCollection<KV<Integer, T>> kvCollection = input.apply("diskBusy.keyElements", ParDo.of(new DoFn<T, KV<Integer, T>>() {
+
+        @ProcessElement public void processElement(ProcessContext context) {
+          context.output(KV.of(0, context.element()));
+        }
+      }));
+      // Apply actual transform that generates disk IO using state API
+      PCollection<T> output = kvCollection.apply("diskBusy.generateIO", ParDo.of(new DoFn<KV<Integer, T>, T>() {
+
+        private static final String DISK_BUSY = "diskBusy";
+
+        @StateId(DISK_BUSY) private final StateSpec<ValueState<byte[]>> spec = StateSpecs
+            .value(ByteArrayCoder.of());
+
+        @ProcessElement public void processElement(ProcessContext c,
+            @StateId(DISK_BUSY) ValueState<byte[]> state) {
+          long remain = bytes;
+          long now = System.currentTimeMillis();
+          while (remain > 0) {
+            long thisBytes = Math.min(remain, MAX_BUFFER_SIZE);
+            remain -= thisBytes;
+            byte[] arr = new byte[(int) thisBytes];
+            for (int i = 0; i < thisBytes; i++) {
+              arr[i] = (byte) now;
+            }
+            state.write(arr);
+            now = System.currentTimeMillis();
+          }
+          c.output(c.element().getValue());
+        }
+      }));
+      return output;
+    }
+  }
+
+
   /**
    * Return a transform to write given number of bytes to durable store on every record.
    */
-  public static <T> ParDo.SingleOutput<T, T> diskBusy(String name, final long bytes) {
-    return ParDo.of(new DoFn<T, T>() {
-                  @ProcessElement
-                  public void processElement(ProcessContext c) {
-                    long remain = bytes;
-//                    long now = System.currentTimeMillis();
-                    while (remain > 0) {
-                      //TODO Ismael google on state
-                      long thisBytes = Math.min(remain, MAX_BUFFER_SIZE);
-                      remain -= thisBytes;
-//                      byte[] arr = new byte[(int) thisBytes];
-//                      for (int i = 0; i < thisBytes; i++) {
-//                        arr[i] = (byte) now;
-//                      }
-//                      ValueState<byte[]> state = c.windowingInternals().stateInternals().state(
-//                          StateNamespaces.global(), DUMMY_TAG);
-//                      state.write(arr);
-//                      now = System.currentTimeMillis();
-                    }
-                    c.output(c.element());
-                  }
-                });
+  public static <T> PTransform<PCollection<T>, PCollection<T>> diskBusy(final long bytes) {
+    return new DiskBusyTransform<>(bytes);
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/beam/blob/6c116709/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/NexmarkQuery.java
----------------------------------------------------------------------
diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/NexmarkQuery.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/NexmarkQuery.java
index 09415c0..8b74282 100644
--- a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/NexmarkQuery.java
+++ b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/NexmarkQuery.java
@@ -253,7 +253,7 @@ public abstract class NexmarkQuery
     if (configuration.diskBusyBytes > 0) {
       // Slow down by forcing bytes to durable store.
       events = events.apply(name + ".DiskBusy",
-              NexmarkUtils.<Event>diskBusy(name, configuration.diskBusyBytes));
+              NexmarkUtils.<Event>diskBusy(configuration.diskBusyBytes));
     }
 
     // Run the query.


[36/55] [abbrv] beam git commit: Rename NexmarkDriver to Main and NexmarkRunner to NexmarkLauncher

Posted by ie...@apache.org.
Rename NexmarkDriver to Main and NexmarkRunner to NexmarkLauncher


Project: http://git-wip-us.apache.org/repos/asf/beam/repo
Commit: http://git-wip-us.apache.org/repos/asf/beam/commit/683680b1
Tree: http://git-wip-us.apache.org/repos/asf/beam/tree/683680b1
Diff: http://git-wip-us.apache.org/repos/asf/beam/diff/683680b1

Branch: refs/heads/master
Commit: 683680b1655e79d696a1d0f4588753a7d8ff2b82
Parents: 77eabba
Author: Etienne Chauchot <ec...@gmail.com>
Authored: Tue May 9 10:17:06 2017 +0200
Committer: Ismaël Mejía <ie...@gmail.com>
Committed: Wed Aug 23 19:07:28 2017 +0200

----------------------------------------------------------------------
 .../apache/beam/integration/nexmark/Main.java   |  304 +++++
 .../beam/integration/nexmark/NexmarkDriver.java |  304 -----
 .../integration/nexmark/NexmarkLauncher.java    | 1172 ++++++++++++++++++
 .../beam/integration/nexmark/NexmarkRunner.java | 1172 ------------------
 4 files changed, 1476 insertions(+), 1476 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/beam/blob/683680b1/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/Main.java
----------------------------------------------------------------------
diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/Main.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/Main.java
new file mode 100644
index 0000000..da4d446
--- /dev/null
+++ b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/Main.java
@@ -0,0 +1,304 @@
+/*
+ * 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.beam.integration.nexmark;
+
+import java.io.IOException;
+import java.nio.charset.StandardCharsets;
+import java.nio.file.Files;
+import java.nio.file.Paths;
+import java.nio.file.StandardOpenOption;
+import java.util.ArrayList;
+import java.util.LinkedHashMap;
+import java.util.List;
+import java.util.Map;
+import javax.annotation.Nullable;
+
+import org.apache.beam.integration.nexmark.model.Auction;
+import org.apache.beam.integration.nexmark.model.Bid;
+import org.apache.beam.integration.nexmark.model.Person;
+import org.apache.beam.sdk.options.PipelineOptionsFactory;
+import org.joda.time.Duration;
+import org.joda.time.Instant;
+
+/**
+ * An implementation of the 'NEXMark queries' for Google Dataflow.
+ * These are multiple queries over a three table schema representing an online auction system:
+ * <ul>
+ * <li>{@link Person} represents a person submitting an item for auction and/or making a bid
+ * on an auction.
+ * <li>{@link Auction} represents an item under auction.
+ * <li>{@link Bid} represents a bid for an item under auction.
+ * </ul>
+ * The queries exercise many aspects of streaming dataflow.
+ *
+ * <p>We synthesize the creation of people, auctions and bids in real-time. The data is not
+ * particularly sensible.
+ *
+ * <p>See <a href="http://datalab.cs.pdx.edu/niagaraST/NEXMark/">
+ * http://datalab.cs.pdx.edu/niagaraST/NEXMark/</a>
+ */
+public class Main<OptionT extends NexmarkOptions> {
+
+  /**
+   * Entry point.
+   */
+  void runAll(OptionT options, NexmarkLauncher nexmarkLauncher) {
+    Instant start = Instant.now();
+    Map<NexmarkConfiguration, NexmarkPerf> baseline = loadBaseline(options.getBaselineFilename());
+    Map<NexmarkConfiguration, NexmarkPerf> actual = new LinkedHashMap<>();
+    Iterable<NexmarkConfiguration> configurations = options.getSuite().getConfigurations(options);
+
+    boolean successful = true;
+    try {
+      // Run all the configurations.
+      for (NexmarkConfiguration configuration : configurations) {
+        NexmarkPerf perf = nexmarkLauncher.run(configuration);
+        if (perf != null) {
+          if (perf.errors == null || perf.errors.size() > 0) {
+            successful = false;
+          }
+          appendPerf(options.getPerfFilename(), configuration, perf);
+          actual.put(configuration, perf);
+          // Summarize what we've run so far.
+          saveSummary(null, configurations, actual, baseline, start);
+        }
+      }
+    } finally {
+      if (options.getMonitorJobs()) {
+        // Report overall performance.
+        saveSummary(options.getSummaryFilename(), configurations, actual, baseline, start);
+        saveJavascript(options.getJavascriptFilename(), configurations, actual, baseline, start);
+      }
+    }
+
+    if (!successful) {
+      throw new RuntimeException("Execution was not successful");
+    }
+  }
+
+  /**
+   * Append the pair of {@code configuration} and {@code perf} to perf file.
+   */
+  private void appendPerf(
+      @Nullable String perfFilename, NexmarkConfiguration configuration,
+      NexmarkPerf perf) {
+    if (perfFilename == null) {
+      return;
+    }
+    List<String> lines = new ArrayList<>();
+    lines.add("");
+    lines.add(String.format("# %s", Instant.now()));
+    lines.add(String.format("# %s", configuration.toShortString()));
+    lines.add(configuration.toString());
+    lines.add(perf.toString());
+    try {
+      Files.write(Paths.get(perfFilename), lines, StandardCharsets.UTF_8, StandardOpenOption.CREATE,
+          StandardOpenOption.APPEND);
+    } catch (IOException e) {
+      throw new RuntimeException("Unable to write perf file: ", e);
+    }
+    NexmarkUtils.console("appended results to perf file %s.", perfFilename);
+  }
+
+  /**
+   * Load the baseline perf.
+   */
+  @Nullable
+  private static Map<NexmarkConfiguration, NexmarkPerf> loadBaseline(
+      @Nullable String baselineFilename) {
+    if (baselineFilename == null) {
+      return null;
+    }
+    Map<NexmarkConfiguration, NexmarkPerf> baseline = new LinkedHashMap<>();
+    List<String> lines;
+    try {
+      lines = Files.readAllLines(Paths.get(baselineFilename), StandardCharsets.UTF_8);
+    } catch (IOException e) {
+      throw new RuntimeException("Unable to read baseline perf file: ", e);
+    }
+    for (int i = 0; i < lines.size(); i++) {
+      if (lines.get(i).startsWith("#") || lines.get(i).trim().isEmpty()) {
+        continue;
+      }
+      NexmarkConfiguration configuration = NexmarkConfiguration.fromString(lines.get(i++));
+      NexmarkPerf perf = NexmarkPerf.fromString(lines.get(i));
+      baseline.put(configuration, perf);
+    }
+    NexmarkUtils.console("loaded %d entries from baseline file %s.", baseline.size(),
+        baselineFilename);
+    return baseline;
+  }
+
+  private static final String LINE =
+      "==========================================================================================";
+
+  /**
+   * Print summary  of {@code actual} vs (if non-null) {@code baseline}.
+   */
+  private static void saveSummary(
+      @Nullable String summaryFilename,
+      Iterable<NexmarkConfiguration> configurations, Map<NexmarkConfiguration, NexmarkPerf> actual,
+      @Nullable Map<NexmarkConfiguration, NexmarkPerf> baseline, Instant start) {
+    List<String> lines = new ArrayList<>();
+
+    lines.add("");
+    lines.add(LINE);
+
+    lines.add(
+        String.format("Run started %s and ran for %s", start, new Duration(start, Instant.now())));
+    lines.add("");
+
+    lines.add("Default configuration:");
+    lines.add(NexmarkConfiguration.DEFAULT.toString());
+    lines.add("");
+
+    lines.add("Configurations:");
+    lines.add("  Conf  Description");
+    int conf = 0;
+    for (NexmarkConfiguration configuration : configurations) {
+      lines.add(String.format("  %04d  %s", conf++, configuration.toShortString()));
+      NexmarkPerf actualPerf = actual.get(configuration);
+      if (actualPerf != null && actualPerf.jobId != null) {
+        lines.add(String.format("  %4s  [Ran as job %s]", "", actualPerf.jobId));
+      }
+    }
+
+    lines.add("");
+    lines.add("Performance:");
+    lines.add(String.format("  %4s  %12s  %12s  %12s  %12s  %12s  %12s", "Conf", "Runtime(sec)",
+        "(Baseline)", "Events(/sec)", "(Baseline)", "Results", "(Baseline)"));
+    conf = 0;
+    for (NexmarkConfiguration configuration : configurations) {
+      String line = String.format("  %04d  ", conf++);
+      NexmarkPerf actualPerf = actual.get(configuration);
+      if (actualPerf == null) {
+        line += "*** not run ***";
+      } else {
+        NexmarkPerf baselinePerf = baseline == null ? null : baseline.get(configuration);
+        double runtimeSec = actualPerf.runtimeSec;
+        line += String.format("%12.1f  ", runtimeSec);
+        if (baselinePerf == null) {
+          line += String.format("%12s  ", "");
+        } else {
+          double baselineRuntimeSec = baselinePerf.runtimeSec;
+          double diff = ((runtimeSec - baselineRuntimeSec) / baselineRuntimeSec) * 100.0;
+          line += String.format("%+11.2f%%  ", diff);
+        }
+
+        double eventsPerSec = actualPerf.eventsPerSec;
+        line += String.format("%12.1f  ", eventsPerSec);
+        if (baselinePerf == null) {
+          line += String.format("%12s  ", "");
+        } else {
+          double baselineEventsPerSec = baselinePerf.eventsPerSec;
+          double diff = ((eventsPerSec - baselineEventsPerSec) / baselineEventsPerSec) * 100.0;
+          line += String.format("%+11.2f%%  ", diff);
+        }
+
+        long numResults = actualPerf.numResults;
+        line += String.format("%12d  ", numResults);
+        if (baselinePerf == null) {
+          line += String.format("%12s", "");
+        } else {
+          long baselineNumResults = baselinePerf.numResults;
+          long diff = numResults - baselineNumResults;
+          line += String.format("%+12d", diff);
+        }
+      }
+      lines.add(line);
+
+      if (actualPerf != null) {
+        List<String> errors = actualPerf.errors;
+        if (errors == null) {
+          errors = new ArrayList<>();
+          errors.add("NexmarkGoogleRunner returned null errors list");
+        }
+        for (String error : errors) {
+          lines.add(String.format("  %4s  *** %s ***", "", error));
+        }
+      }
+    }
+
+    lines.add(LINE);
+    lines.add("");
+
+    for (String line : lines) {
+      System.out.println(line);
+    }
+
+    if (summaryFilename != null) {
+      try {
+        Files.write(Paths.get(summaryFilename), lines, StandardCharsets.UTF_8,
+            StandardOpenOption.CREATE, StandardOpenOption.APPEND);
+      } catch (IOException e) {
+        throw new RuntimeException("Unable to save summary file: ", e);
+      }
+      NexmarkUtils.console("appended summary to summary file %s.", summaryFilename);
+    }
+  }
+
+  /**
+   * Write all perf data and any baselines to a javascript file which can be used by
+   * graphing page etc.
+   */
+  private static void saveJavascript(
+      @Nullable String javascriptFilename,
+      Iterable<NexmarkConfiguration> configurations, Map<NexmarkConfiguration, NexmarkPerf> actual,
+      @Nullable Map<NexmarkConfiguration, NexmarkPerf> baseline, Instant start) {
+    if (javascriptFilename == null) {
+      return;
+    }
+
+    List<String> lines = new ArrayList<>();
+    lines.add(String.format(
+        "// Run started %s and ran for %s", start, new Duration(start, Instant.now())));
+    lines.add("var all = [");
+
+    for (NexmarkConfiguration configuration : configurations) {
+      lines.add("  {");
+      lines.add(String.format("    config: %s", configuration));
+      NexmarkPerf actualPerf = actual.get(configuration);
+      if (actualPerf != null) {
+        lines.add(String.format("    ,perf: %s", actualPerf));
+      }
+      NexmarkPerf baselinePerf = baseline == null ? null : baseline.get(configuration);
+      if (baselinePerf != null) {
+        lines.add(String.format("    ,baseline: %s", baselinePerf));
+      }
+      lines.add("  },");
+    }
+
+    lines.add("];");
+
+    try {
+      Files.write(Paths.get(javascriptFilename), lines, StandardCharsets.UTF_8,
+          StandardOpenOption.CREATE, StandardOpenOption.TRUNCATE_EXISTING);
+    } catch (IOException e) {
+      throw new RuntimeException("Unable to save javascript file: ", e);
+    }
+    NexmarkUtils.console("saved javascript to file %s.", javascriptFilename);
+  }
+
+  public static void main(String[] args) {
+    NexmarkOptions options = PipelineOptionsFactory.fromArgs(args)
+      .withValidation()
+      .as(NexmarkOptions.class);
+    NexmarkLauncher<NexmarkOptions> nexmarkLauncher = new NexmarkLauncher<>(options);
+    new Main<>().runAll(options, nexmarkLauncher);
+  }
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/683680b1/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkDriver.java
----------------------------------------------------------------------
diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkDriver.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkDriver.java
deleted file mode 100644
index a982a8d..0000000
--- a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkDriver.java
+++ /dev/null
@@ -1,304 +0,0 @@
-/*
- * 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.beam.integration.nexmark;
-
-import java.io.IOException;
-import java.nio.charset.StandardCharsets;
-import java.nio.file.Files;
-import java.nio.file.Paths;
-import java.nio.file.StandardOpenOption;
-import java.util.ArrayList;
-import java.util.LinkedHashMap;
-import java.util.List;
-import java.util.Map;
-import javax.annotation.Nullable;
-
-import org.apache.beam.integration.nexmark.model.Auction;
-import org.apache.beam.integration.nexmark.model.Bid;
-import org.apache.beam.integration.nexmark.model.Person;
-import org.apache.beam.sdk.options.PipelineOptionsFactory;
-import org.joda.time.Duration;
-import org.joda.time.Instant;
-
-/**
- * An implementation of the 'NEXMark queries' for Google Dataflow.
- * These are multiple queries over a three table schema representing an online auction system:
- * <ul>
- * <li>{@link Person} represents a person submitting an item for auction and/or making a bid
- * on an auction.
- * <li>{@link Auction} represents an item under auction.
- * <li>{@link Bid} represents a bid for an item under auction.
- * </ul>
- * The queries exercise many aspects of streaming dataflow.
- *
- * <p>We synthesize the creation of people, auctions and bids in real-time. The data is not
- * particularly sensible.
- *
- * <p>See <a href="http://datalab.cs.pdx.edu/niagaraST/NEXMark/">
- * http://datalab.cs.pdx.edu/niagaraST/NEXMark/</a>
- */
-public class NexmarkDriver<OptionT extends NexmarkOptions> {
-
-  /**
-   * Entry point.
-   */
-  void runAll(OptionT options, NexmarkRunner runner) {
-    Instant start = Instant.now();
-    Map<NexmarkConfiguration, NexmarkPerf> baseline = loadBaseline(options.getBaselineFilename());
-    Map<NexmarkConfiguration, NexmarkPerf> actual = new LinkedHashMap<>();
-    Iterable<NexmarkConfiguration> configurations = options.getSuite().getConfigurations(options);
-
-    boolean successful = true;
-    try {
-      // Run all the configurations.
-      for (NexmarkConfiguration configuration : configurations) {
-        NexmarkPerf perf = runner.run(configuration);
-        if (perf != null) {
-          if (perf.errors == null || perf.errors.size() > 0) {
-            successful = false;
-          }
-          appendPerf(options.getPerfFilename(), configuration, perf);
-          actual.put(configuration, perf);
-          // Summarize what we've run so far.
-          saveSummary(null, configurations, actual, baseline, start);
-        }
-      }
-    } finally {
-      if (options.getMonitorJobs()) {
-        // Report overall performance.
-        saveSummary(options.getSummaryFilename(), configurations, actual, baseline, start);
-        saveJavascript(options.getJavascriptFilename(), configurations, actual, baseline, start);
-      }
-    }
-
-    if (!successful) {
-      throw new RuntimeException("Execution was not successful");
-    }
-  }
-
-  /**
-   * Append the pair of {@code configuration} and {@code perf} to perf file.
-   */
-  private void appendPerf(
-      @Nullable String perfFilename, NexmarkConfiguration configuration,
-      NexmarkPerf perf) {
-    if (perfFilename == null) {
-      return;
-    }
-    List<String> lines = new ArrayList<>();
-    lines.add("");
-    lines.add(String.format("# %s", Instant.now()));
-    lines.add(String.format("# %s", configuration.toShortString()));
-    lines.add(configuration.toString());
-    lines.add(perf.toString());
-    try {
-      Files.write(Paths.get(perfFilename), lines, StandardCharsets.UTF_8, StandardOpenOption.CREATE,
-          StandardOpenOption.APPEND);
-    } catch (IOException e) {
-      throw new RuntimeException("Unable to write perf file: ", e);
-    }
-    NexmarkUtils.console("appended results to perf file %s.", perfFilename);
-  }
-
-  /**
-   * Load the baseline perf.
-   */
-  @Nullable
-  private static Map<NexmarkConfiguration, NexmarkPerf> loadBaseline(
-      @Nullable String baselineFilename) {
-    if (baselineFilename == null) {
-      return null;
-    }
-    Map<NexmarkConfiguration, NexmarkPerf> baseline = new LinkedHashMap<>();
-    List<String> lines;
-    try {
-      lines = Files.readAllLines(Paths.get(baselineFilename), StandardCharsets.UTF_8);
-    } catch (IOException e) {
-      throw new RuntimeException("Unable to read baseline perf file: ", e);
-    }
-    for (int i = 0; i < lines.size(); i++) {
-      if (lines.get(i).startsWith("#") || lines.get(i).trim().isEmpty()) {
-        continue;
-      }
-      NexmarkConfiguration configuration = NexmarkConfiguration.fromString(lines.get(i++));
-      NexmarkPerf perf = NexmarkPerf.fromString(lines.get(i));
-      baseline.put(configuration, perf);
-    }
-    NexmarkUtils.console("loaded %d entries from baseline file %s.", baseline.size(),
-        baselineFilename);
-    return baseline;
-  }
-
-  private static final String LINE =
-      "==========================================================================================";
-
-  /**
-   * Print summary  of {@code actual} vs (if non-null) {@code baseline}.
-   */
-  private static void saveSummary(
-      @Nullable String summaryFilename,
-      Iterable<NexmarkConfiguration> configurations, Map<NexmarkConfiguration, NexmarkPerf> actual,
-      @Nullable Map<NexmarkConfiguration, NexmarkPerf> baseline, Instant start) {
-    List<String> lines = new ArrayList<>();
-
-    lines.add("");
-    lines.add(LINE);
-
-    lines.add(
-        String.format("Run started %s and ran for %s", start, new Duration(start, Instant.now())));
-    lines.add("");
-
-    lines.add("Default configuration:");
-    lines.add(NexmarkConfiguration.DEFAULT.toString());
-    lines.add("");
-
-    lines.add("Configurations:");
-    lines.add("  Conf  Description");
-    int conf = 0;
-    for (NexmarkConfiguration configuration : configurations) {
-      lines.add(String.format("  %04d  %s", conf++, configuration.toShortString()));
-      NexmarkPerf actualPerf = actual.get(configuration);
-      if (actualPerf != null && actualPerf.jobId != null) {
-        lines.add(String.format("  %4s  [Ran as job %s]", "", actualPerf.jobId));
-      }
-    }
-
-    lines.add("");
-    lines.add("Performance:");
-    lines.add(String.format("  %4s  %12s  %12s  %12s  %12s  %12s  %12s", "Conf", "Runtime(sec)",
-        "(Baseline)", "Events(/sec)", "(Baseline)", "Results", "(Baseline)"));
-    conf = 0;
-    for (NexmarkConfiguration configuration : configurations) {
-      String line = String.format("  %04d  ", conf++);
-      NexmarkPerf actualPerf = actual.get(configuration);
-      if (actualPerf == null) {
-        line += "*** not run ***";
-      } else {
-        NexmarkPerf baselinePerf = baseline == null ? null : baseline.get(configuration);
-        double runtimeSec = actualPerf.runtimeSec;
-        line += String.format("%12.1f  ", runtimeSec);
-        if (baselinePerf == null) {
-          line += String.format("%12s  ", "");
-        } else {
-          double baselineRuntimeSec = baselinePerf.runtimeSec;
-          double diff = ((runtimeSec - baselineRuntimeSec) / baselineRuntimeSec) * 100.0;
-          line += String.format("%+11.2f%%  ", diff);
-        }
-
-        double eventsPerSec = actualPerf.eventsPerSec;
-        line += String.format("%12.1f  ", eventsPerSec);
-        if (baselinePerf == null) {
-          line += String.format("%12s  ", "");
-        } else {
-          double baselineEventsPerSec = baselinePerf.eventsPerSec;
-          double diff = ((eventsPerSec - baselineEventsPerSec) / baselineEventsPerSec) * 100.0;
-          line += String.format("%+11.2f%%  ", diff);
-        }
-
-        long numResults = actualPerf.numResults;
-        line += String.format("%12d  ", numResults);
-        if (baselinePerf == null) {
-          line += String.format("%12s", "");
-        } else {
-          long baselineNumResults = baselinePerf.numResults;
-          long diff = numResults - baselineNumResults;
-          line += String.format("%+12d", diff);
-        }
-      }
-      lines.add(line);
-
-      if (actualPerf != null) {
-        List<String> errors = actualPerf.errors;
-        if (errors == null) {
-          errors = new ArrayList<>();
-          errors.add("NexmarkGoogleRunner returned null errors list");
-        }
-        for (String error : errors) {
-          lines.add(String.format("  %4s  *** %s ***", "", error));
-        }
-      }
-    }
-
-    lines.add(LINE);
-    lines.add("");
-
-    for (String line : lines) {
-      System.out.println(line);
-    }
-
-    if (summaryFilename != null) {
-      try {
-        Files.write(Paths.get(summaryFilename), lines, StandardCharsets.UTF_8,
-            StandardOpenOption.CREATE, StandardOpenOption.APPEND);
-      } catch (IOException e) {
-        throw new RuntimeException("Unable to save summary file: ", e);
-      }
-      NexmarkUtils.console("appended summary to summary file %s.", summaryFilename);
-    }
-  }
-
-  /**
-   * Write all perf data and any baselines to a javascript file which can be used by
-   * graphing page etc.
-   */
-  private static void saveJavascript(
-      @Nullable String javascriptFilename,
-      Iterable<NexmarkConfiguration> configurations, Map<NexmarkConfiguration, NexmarkPerf> actual,
-      @Nullable Map<NexmarkConfiguration, NexmarkPerf> baseline, Instant start) {
-    if (javascriptFilename == null) {
-      return;
-    }
-
-    List<String> lines = new ArrayList<>();
-    lines.add(String.format(
-        "// Run started %s and ran for %s", start, new Duration(start, Instant.now())));
-    lines.add("var all = [");
-
-    for (NexmarkConfiguration configuration : configurations) {
-      lines.add("  {");
-      lines.add(String.format("    config: %s", configuration));
-      NexmarkPerf actualPerf = actual.get(configuration);
-      if (actualPerf != null) {
-        lines.add(String.format("    ,perf: %s", actualPerf));
-      }
-      NexmarkPerf baselinePerf = baseline == null ? null : baseline.get(configuration);
-      if (baselinePerf != null) {
-        lines.add(String.format("    ,baseline: %s", baselinePerf));
-      }
-      lines.add("  },");
-    }
-
-    lines.add("];");
-
-    try {
-      Files.write(Paths.get(javascriptFilename), lines, StandardCharsets.UTF_8,
-          StandardOpenOption.CREATE, StandardOpenOption.TRUNCATE_EXISTING);
-    } catch (IOException e) {
-      throw new RuntimeException("Unable to save javascript file: ", e);
-    }
-    NexmarkUtils.console("saved javascript to file %s.", javascriptFilename);
-  }
-
-  public static void main(String[] args) {
-    NexmarkOptions options = PipelineOptionsFactory.fromArgs(args)
-      .withValidation()
-      .as(NexmarkOptions.class);
-    NexmarkRunner<NexmarkOptions> runner = new NexmarkRunner<>(options);
-    new NexmarkDriver<>().runAll(options, runner);
-  }
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/683680b1/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkLauncher.java
----------------------------------------------------------------------
diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkLauncher.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkLauncher.java
new file mode 100644
index 0000000..ea4ff58
--- /dev/null
+++ b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkLauncher.java
@@ -0,0 +1,1172 @@
+/*
+ * 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.beam.integration.nexmark;
+
+import static com.google.common.base.Preconditions.checkState;
+
+import com.google.api.services.bigquery.model.TableFieldSchema;
+import com.google.api.services.bigquery.model.TableRow;
+import com.google.api.services.bigquery.model.TableSchema;
+import com.google.common.base.Strings;
+import com.google.common.collect.ImmutableList;
+import com.google.common.collect.Lists;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.NoSuchElementException;
+import java.util.concurrent.ThreadLocalRandom;
+import javax.annotation.Nullable;
+import org.apache.beam.integration.nexmark.model.Auction;
+import org.apache.beam.integration.nexmark.model.Bid;
+import org.apache.beam.integration.nexmark.model.Event;
+import org.apache.beam.integration.nexmark.model.KnownSize;
+import org.apache.beam.integration.nexmark.model.Person;
+import org.apache.beam.integration.nexmark.queries.NexmarkQuery;
+import org.apache.beam.integration.nexmark.queries.NexmarkQueryModel;
+import org.apache.beam.integration.nexmark.queries.Query0;
+import org.apache.beam.integration.nexmark.queries.Query0Model;
+import org.apache.beam.integration.nexmark.queries.Query1;
+import org.apache.beam.integration.nexmark.queries.Query10;
+import org.apache.beam.integration.nexmark.queries.Query11;
+import org.apache.beam.integration.nexmark.queries.Query12;
+import org.apache.beam.integration.nexmark.queries.Query1Model;
+import org.apache.beam.integration.nexmark.queries.Query2;
+import org.apache.beam.integration.nexmark.queries.Query2Model;
+import org.apache.beam.integration.nexmark.queries.Query3;
+import org.apache.beam.integration.nexmark.queries.Query3Model;
+import org.apache.beam.integration.nexmark.queries.Query4;
+import org.apache.beam.integration.nexmark.queries.Query4Model;
+import org.apache.beam.integration.nexmark.queries.Query5;
+import org.apache.beam.integration.nexmark.queries.Query5Model;
+import org.apache.beam.integration.nexmark.queries.Query6;
+import org.apache.beam.integration.nexmark.queries.Query6Model;
+import org.apache.beam.integration.nexmark.queries.Query7;
+import org.apache.beam.integration.nexmark.queries.Query7Model;
+import org.apache.beam.integration.nexmark.queries.Query8;
+import org.apache.beam.integration.nexmark.queries.Query8Model;
+import org.apache.beam.integration.nexmark.queries.Query9;
+import org.apache.beam.integration.nexmark.queries.Query9Model;
+import org.apache.beam.sdk.Pipeline;
+import org.apache.beam.sdk.PipelineResult;
+import org.apache.beam.sdk.coders.CoderException;
+import org.apache.beam.sdk.io.AvroIO;
+import org.apache.beam.sdk.io.TextIO;
+import org.apache.beam.sdk.io.gcp.bigquery.BigQueryIO;
+import org.apache.beam.sdk.io.gcp.pubsub.PubsubIO;
+import org.apache.beam.sdk.io.gcp.pubsub.PubsubMessage;
+import org.apache.beam.sdk.metrics.DistributionResult;
+import org.apache.beam.sdk.metrics.MetricNameFilter;
+import org.apache.beam.sdk.metrics.MetricQueryResults;
+import org.apache.beam.sdk.metrics.MetricResult;
+import org.apache.beam.sdk.metrics.MetricsFilter;
+import org.apache.beam.sdk.testing.PAssert;
+import org.apache.beam.sdk.transforms.DoFn;
+import org.apache.beam.sdk.transforms.ParDo;
+import org.apache.beam.sdk.util.CoderUtils;
+import org.apache.beam.sdk.values.PCollection;
+import org.apache.beam.sdk.values.PCollectionTuple;
+import org.apache.beam.sdk.values.TimestampedValue;
+import org.apache.beam.sdk.values.TupleTag;
+import org.apache.beam.sdk.values.TupleTagList;
+import org.joda.time.Duration;
+
+/**
+ * Run a single Nexmark query using a given configuration.
+ */
+public class NexmarkLauncher<OptionT extends NexmarkOptions> {
+  /**
+   * Minimum number of samples needed for 'stead-state' rate calculation.
+   */
+  private static final int MIN_SAMPLES = 9;
+  /**
+   * Minimum length of time over which to consider samples for 'steady-state' rate calculation.
+   */
+  private static final Duration MIN_WINDOW = Duration.standardMinutes(2);
+  /**
+   * Delay between perf samples.
+   */
+  private static final Duration PERF_DELAY = Duration.standardSeconds(15);
+  /**
+   * How long to let streaming pipeline run after all events have been generated and we've
+   * seen no activity.
+   */
+  private static final Duration DONE_DELAY = Duration.standardMinutes(1);
+  /**
+   * How long to allow no activity without warning.
+   */
+  private static final Duration STUCK_WARNING_DELAY = Duration.standardMinutes(10);
+  /**
+   * How long to let streaming pipeline run after we've
+   * seen no activity, even if all events have not been generated.
+   */
+  private static final Duration STUCK_TERMINATE_DELAY = Duration.standardDays(3);
+  /**
+   * NexmarkOptions shared by all runs.
+   */
+  private final OptionT options;
+
+  /**
+   * Which configuration we are running.
+   */
+  @Nullable
+  private NexmarkConfiguration configuration;
+
+  /**
+   * If in --pubsubMode=COMBINED, the event monitor for the publisher pipeline. Otherwise null.
+   */
+  @Nullable
+  private Monitor<Event> publisherMonitor;
+
+  /**
+   * If in --pubsubMode=COMBINED, the pipeline result for the publisher pipeline. Otherwise null.
+   */
+  @Nullable
+  private PipelineResult publisherResult;
+
+  /**
+   * Result for the main pipeline.
+   */
+  @Nullable
+  private PipelineResult mainResult;
+
+  /**
+   * Query name we are running.
+   */
+  @Nullable
+  private String queryName;
+
+  public NexmarkLauncher(OptionT options) {
+    this.options = options;
+  }
+
+
+  /**
+   * Is this query running in streaming mode?
+   */
+  private boolean isStreaming() {
+    return options.isStreaming();
+  }
+
+  /**
+   * Return number of cores per worker.
+   */
+  protected int coresPerWorker() {
+    return 4;
+  }
+
+  /**
+   * Return maximum number of workers.
+   */
+  private int maxNumWorkers() {
+    return 5;
+  }
+
+  /**
+   * Return the current value for a long counter, or a default value if can't be retrieved.
+   * Note this uses only attempted metrics because some runners don't support committed metrics.
+   */
+  private long getCounterMetric(PipelineResult result, String namespace, String name,
+    long defaultValue) {
+    //TODO Ismael calc this only once
+    MetricQueryResults metrics = result.metrics().queryMetrics(
+        MetricsFilter.builder().addNameFilter(MetricNameFilter.named(namespace, name)).build());
+    Iterable<MetricResult<Long>> counters = metrics.counters();
+    try {
+      MetricResult<Long> metricResult = counters.iterator().next();
+      return metricResult.attempted();
+    } catch (NoSuchElementException e) {
+      //TODO Ismael
+    }
+    return defaultValue;
+  }
+
+  /**
+   * Return the current value for a long counter, or a default value if can't be retrieved.
+   * Note this uses only attempted metrics because some runners don't support committed metrics.
+   */
+  private long getDistributionMetric(PipelineResult result, String namespace, String name,
+      DistributionType distType, long defaultValue) {
+    MetricQueryResults metrics = result.metrics().queryMetrics(
+        MetricsFilter.builder().addNameFilter(MetricNameFilter.named(namespace, name)).build());
+    Iterable<MetricResult<DistributionResult>> distributions = metrics.distributions();
+    try {
+      MetricResult<DistributionResult> distributionResult = distributions.iterator().next();
+      if (distType.equals(DistributionType.MIN)) {
+        return distributionResult.attempted().min();
+      } else if (distType.equals(DistributionType.MAX)) {
+        return distributionResult.attempted().max();
+      } else {
+        //TODO Ismael
+      }
+    } catch (NoSuchElementException e) {
+      //TODO Ismael
+    }
+    return defaultValue;
+  }
+
+  private enum DistributionType {MIN, MAX}
+
+  /**
+   * Return the current value for a time counter, or -1 if can't be retrieved.
+   */
+  private long getTimestampMetric(long now, long value) {
+    //TODO Ismael improve doc
+    if (Math.abs(value - now) > Duration.standardDays(10000).getMillis()) {
+      return -1;
+    }
+    return value;
+  }
+
+  /**
+   * Find a 'steady state' events/sec from {@code snapshots} and
+   * store it in {@code perf} if found.
+   */
+  private void captureSteadyState(NexmarkPerf perf, List<NexmarkPerf.ProgressSnapshot> snapshots) {
+    if (!options.isStreaming()) {
+      return;
+    }
+
+    // Find the first sample with actual event and result counts.
+    int dataStart = 0;
+    for (; dataStart < snapshots.size(); dataStart++) {
+      if (snapshots.get(dataStart).numEvents >= 0 && snapshots.get(dataStart).numResults >= 0) {
+        break;
+      }
+    }
+
+    // Find the last sample which demonstrated progress.
+    int dataEnd = snapshots.size() - 1;
+    for (; dataEnd > dataStart; dataEnd--) {
+      if (snapshots.get(dataEnd).anyActivity(snapshots.get(dataEnd - 1))) {
+        break;
+      }
+    }
+
+    int numSamples = dataEnd - dataStart + 1;
+    if (numSamples < MIN_SAMPLES) {
+      // Not enough samples.
+      NexmarkUtils.console("%d samples not enough to calculate steady-state event rate",
+          numSamples);
+      return;
+    }
+
+    // We'll look at only the middle third samples.
+    int sampleStart = dataStart + numSamples / 3;
+    int sampleEnd = dataEnd - numSamples / 3;
+
+    double sampleSec =
+        snapshots.get(sampleEnd).secSinceStart - snapshots.get(sampleStart).secSinceStart;
+    if (sampleSec < MIN_WINDOW.getStandardSeconds()) {
+      // Not sampled over enough time.
+      NexmarkUtils.console(
+          "sample of %.1f sec not long enough to calculate steady-state event rate",
+          sampleSec);
+      return;
+    }
+
+    // Find rate with least squares error.
+    double sumxx = 0.0;
+    double sumxy = 0.0;
+    long prevNumEvents = -1;
+    for (int i = sampleStart; i <= sampleEnd; i++) {
+      if (prevNumEvents == snapshots.get(i).numEvents) {
+        // Skip samples with no change in number of events since they contribute no data.
+        continue;
+      }
+      // Use the effective runtime instead of wallclock time so we can
+      // insulate ourselves from delays and stutters in the query manager.
+      double x = snapshots.get(i).runtimeSec;
+      prevNumEvents = snapshots.get(i).numEvents;
+      double y = prevNumEvents;
+      sumxx += x * x;
+      sumxy += x * y;
+    }
+    double eventsPerSec = sumxy / sumxx;
+    NexmarkUtils.console("revising events/sec from %.1f to %.1f", perf.eventsPerSec, eventsPerSec);
+    perf.eventsPerSec = eventsPerSec;
+  }
+
+  /**
+   * Return the current performance given {@code eventMonitor} and {@code resultMonitor}.
+   */
+  private NexmarkPerf currentPerf(
+      long startMsSinceEpoch, long now, PipelineResult result,
+      List<NexmarkPerf.ProgressSnapshot> snapshots, Monitor<?> eventMonitor,
+      Monitor<?> resultMonitor) {
+    NexmarkPerf perf = new NexmarkPerf();
+
+    long numEvents =
+      getCounterMetric(result, eventMonitor.name, eventMonitor.prefix + ".elements", -1);
+    long numEventBytes =
+      getCounterMetric(result, eventMonitor.name, eventMonitor.prefix + ".bytes", -1);
+    long eventStart =
+      getTimestampMetric(now,
+        getDistributionMetric(result, eventMonitor.name, eventMonitor.prefix + ".startTime",
+          DistributionType.MIN, -1));
+    long eventEnd =
+      getTimestampMetric(now,
+        getDistributionMetric(result, eventMonitor.name, eventMonitor.prefix + ".endTime",
+          DistributionType.MAX, -1));
+
+    long numResults =
+      getCounterMetric(result, resultMonitor.name, resultMonitor.prefix + ".elements", -1);
+    long numResultBytes =
+      getCounterMetric(result, resultMonitor.name, resultMonitor.prefix + ".bytes", -1);
+    long resultStart =
+      getTimestampMetric(now,
+        getDistributionMetric(result, resultMonitor.name, resultMonitor.prefix + ".startTime",
+          DistributionType.MIN, -1));
+    long resultEnd =
+      getTimestampMetric(now,
+        getDistributionMetric(result, resultMonitor.name, resultMonitor.prefix + ".endTime",
+          DistributionType.MAX, -1));
+    long timestampStart =
+      getTimestampMetric(now,
+        getDistributionMetric(result,
+          resultMonitor.name, resultMonitor.prefix + ".startTimestamp",
+          DistributionType.MIN, -1));
+    long timestampEnd =
+      getTimestampMetric(now,
+        getDistributionMetric(result,
+          resultMonitor.name, resultMonitor.prefix + ".endTimestamp",
+          DistributionType.MAX, -1));
+
+    long effectiveEnd = -1;
+    if (eventEnd >= 0 && resultEnd >= 0) {
+      // It is possible for events to be generated after the last result was emitted.
+      // (Eg Query 2, which only yields results for a small prefix of the event stream.)
+      // So use the max of last event and last result times.
+      effectiveEnd = Math.max(eventEnd, resultEnd);
+    } else if (resultEnd >= 0) {
+      effectiveEnd = resultEnd;
+    } else if (eventEnd >= 0) {
+      // During startup we may have no result yet, but we would still like to track how
+      // long the pipeline has been running.
+      effectiveEnd = eventEnd;
+    }
+
+    if (effectiveEnd >= 0 && eventStart >= 0 && effectiveEnd >= eventStart) {
+      perf.runtimeSec = (effectiveEnd - eventStart) / 1000.0;
+    }
+
+    if (numEvents >= 0) {
+      perf.numEvents = numEvents;
+    }
+
+    if (numEvents >= 0 && perf.runtimeSec > 0.0) {
+      // For streaming we may later replace this with a 'steady-state' value calculated
+      // from the progress snapshots.
+      perf.eventsPerSec = numEvents / perf.runtimeSec;
+    }
+
+    if (numEventBytes >= 0 && perf.runtimeSec > 0.0) {
+      perf.eventBytesPerSec = numEventBytes / perf.runtimeSec;
+    }
+
+    if (numResults >= 0) {
+      perf.numResults = numResults;
+    }
+
+    if (numResults >= 0 && perf.runtimeSec > 0.0) {
+      perf.resultsPerSec = numResults / perf.runtimeSec;
+    }
+
+    if (numResultBytes >= 0 && perf.runtimeSec > 0.0) {
+      perf.resultBytesPerSec = numResultBytes / perf.runtimeSec;
+    }
+
+    if (eventStart >= 0) {
+      perf.startupDelaySec = (eventStart - startMsSinceEpoch) / 1000.0;
+    }
+
+    if (resultStart >= 0 && eventStart >= 0 && resultStart >= eventStart) {
+      perf.processingDelaySec = (resultStart - eventStart) / 1000.0;
+    }
+
+    if (timestampStart >= 0 && timestampEnd >= 0 && perf.runtimeSec > 0.0) {
+      double eventRuntimeSec = (timestampEnd - timestampStart) / 1000.0;
+      perf.timeDilation = eventRuntimeSec / perf.runtimeSec;
+    }
+
+    if (resultEnd >= 0) {
+      // Fill in the shutdown delay assuming the job has now finished.
+      perf.shutdownDelaySec = (now - resultEnd) / 1000.0;
+    }
+
+    // As soon as available, try to capture cumulative cost at this point too.
+
+    NexmarkPerf.ProgressSnapshot snapshot = new NexmarkPerf.ProgressSnapshot();
+    snapshot.secSinceStart = (now - startMsSinceEpoch) / 1000.0;
+    snapshot.runtimeSec = perf.runtimeSec;
+    snapshot.numEvents = numEvents;
+    snapshot.numResults = numResults;
+    snapshots.add(snapshot);
+
+    captureSteadyState(perf, snapshots);
+
+    return perf;
+  }
+
+  /**
+   * Build and run a pipeline using specified options.
+   */
+  interface PipelineBuilder<OptionT extends NexmarkOptions> {
+    void build(OptionT publishOnlyOptions);
+  }
+
+  /**
+   * Invoke the builder with options suitable for running a publish-only child pipeline.
+   */
+  private void invokeBuilderForPublishOnlyPipeline(PipelineBuilder<NexmarkOptions> builder) {
+    builder.build(options);
+//    throw new UnsupportedOperationException(
+//        "Cannot use --pubSubMode=COMBINED with DirectRunner");
+  }
+
+  /**
+   * If monitoring, wait until the publisher pipeline has run long enough to establish
+   * a backlog on the Pubsub topic. Otherwise, return immediately.
+   */
+  private void waitForPublisherPreload() {
+    throw new UnsupportedOperationException();
+  }
+
+  /**
+   * Monitor the performance and progress of a running job. Return final performance if
+   * it was measured.
+   */
+  @Nullable
+  private NexmarkPerf monitor(NexmarkQuery query) {
+    if (!options.getMonitorJobs()) {
+      return null;
+    }
+
+    if (configuration.debug) {
+      NexmarkUtils.console("Waiting for main pipeline to 'finish'");
+    } else {
+      NexmarkUtils.console("--debug=false, so job will not self-cancel");
+    }
+
+    PipelineResult job = mainResult;
+    PipelineResult publisherJob = publisherResult;
+    List<NexmarkPerf.ProgressSnapshot> snapshots = new ArrayList<>();
+    long startMsSinceEpoch = System.currentTimeMillis();
+    long endMsSinceEpoch = -1;
+    if (options.getRunningTimeMinutes() != null) {
+      endMsSinceEpoch = startMsSinceEpoch
+                        + Duration.standardMinutes(options.getRunningTimeMinutes()).getMillis()
+                        - Duration.standardSeconds(configuration.preloadSeconds).getMillis();
+    }
+    long lastActivityMsSinceEpoch = -1;
+    NexmarkPerf perf = null;
+    boolean waitingForShutdown = false;
+    boolean publisherCancelled = false;
+    List<String> errors = new ArrayList<>();
+
+    while (true) {
+      long now = System.currentTimeMillis();
+      if (endMsSinceEpoch >= 0 && now > endMsSinceEpoch && !waitingForShutdown) {
+        NexmarkUtils.console("Reached end of test, cancelling job");
+        try {
+          job.cancel();
+        } catch (IOException e) {
+          throw new RuntimeException("Unable to cancel main job: ", e);
+        }
+        if (publisherResult != null) {
+          try {
+            publisherJob.cancel();
+          } catch (IOException e) {
+            throw new RuntimeException("Unable to cancel publisher job: ", e);
+          }
+          publisherCancelled = true;
+        }
+        waitingForShutdown = true;
+      }
+
+      PipelineResult.State state = job.getState();
+      NexmarkUtils.console("%s %s%s", state, queryName,
+          waitingForShutdown ? " (waiting for shutdown)" : "");
+
+      NexmarkPerf currPerf;
+      if (configuration.debug) {
+        currPerf = currentPerf(startMsSinceEpoch, now, job, snapshots,
+                               query.eventMonitor, query.resultMonitor);
+      } else {
+        currPerf = null;
+      }
+
+      if (perf == null || perf.anyActivity(currPerf)) {
+        lastActivityMsSinceEpoch = now;
+      }
+
+      if (options.isStreaming() && !waitingForShutdown) {
+        Duration quietFor = new Duration(lastActivityMsSinceEpoch, now);
+        long fatalCount = getCounterMetric(job, query.getName(), "fatal", 0);
+        if (fatalCount > 0) {
+          NexmarkUtils.console("job has fatal errors, cancelling.");
+          errors.add(String.format("Pipeline reported %s fatal errors", fatalCount));
+          waitingForShutdown = true;
+        } else if (configuration.debug && configuration.numEvents > 0
+                   && currPerf.numEvents == configuration.numEvents
+                   && currPerf.numResults >= 0 && quietFor.isLongerThan(DONE_DELAY)) {
+          NexmarkUtils.console("streaming query appears to have finished, cancelling job.");
+          waitingForShutdown = true;
+        } else if (quietFor.isLongerThan(STUCK_TERMINATE_DELAY)) {
+          NexmarkUtils.console("streaming query appears to have gotten stuck, cancelling job.");
+          errors.add("Streaming job was cancelled since appeared stuck");
+          waitingForShutdown = true;
+        } else if (quietFor.isLongerThan(STUCK_WARNING_DELAY)) {
+          NexmarkUtils.console("WARNING: streaming query appears to have been stuck for %d min.",
+              quietFor.getStandardMinutes());
+          errors.add(
+              String.format("Streaming query was stuck for %d min", quietFor.getStandardMinutes()));
+        }
+
+        if (waitingForShutdown) {
+          try {
+            job.cancel();
+          } catch (IOException e) {
+            throw new RuntimeException("Unable to cancel main job: ", e);
+          }
+        }
+      }
+
+      perf = currPerf;
+
+      boolean running = true;
+      switch (state) {
+        case UNKNOWN:
+        case STOPPED:
+        case RUNNING:
+          // Keep going.
+          break;
+        case DONE:
+          // All done.
+          running = false;
+          break;
+        case CANCELLED:
+          running = false;
+          if (!waitingForShutdown) {
+            errors.add("Job was unexpectedly cancelled");
+          }
+          break;
+        case FAILED:
+        case UPDATED:
+          // Abnormal termination.
+          running = false;
+          errors.add("Job was unexpectedly updated");
+          break;
+      }
+
+      if (!running) {
+        break;
+      }
+
+      if (lastActivityMsSinceEpoch == now) {
+        NexmarkUtils.console("new perf %s", perf);
+      } else {
+        NexmarkUtils.console("no activity");
+      }
+
+      try {
+        Thread.sleep(PERF_DELAY.getMillis());
+      } catch (InterruptedException e) {
+        Thread.interrupted();
+        NexmarkUtils.console("Interrupted: pipeline is still running");
+      }
+    }
+
+    perf.errors = errors;
+    perf.snapshots = snapshots;
+
+    if (publisherResult != null) {
+      NexmarkUtils.console("Shutting down publisher pipeline.");
+      try {
+        if (!publisherCancelled) {
+          publisherJob.cancel();
+        }
+        publisherJob.waitUntilFinish(Duration.standardMinutes(5));
+      } catch (IOException e) {
+        throw new RuntimeException("Unable to cancel publisher job: ", e);
+      } //TODO Ismael
+//      catch (InterruptedException e) {
+//        Thread.interrupted();
+//        throw new RuntimeException("Interrupted: publish job still running.", e);
+//      }
+    }
+
+    return perf;
+  }
+
+  // ================================================================================
+  // Basic sources and sinks
+  // ================================================================================
+
+  /**
+   * Return a topic name.
+   */
+  private String shortTopic(long now) {
+    String baseTopic = options.getPubsubTopic();
+    if (Strings.isNullOrEmpty(baseTopic)) {
+      throw new RuntimeException("Missing --pubsubTopic");
+    }
+    switch (options.getResourceNameMode()) {
+      case VERBATIM:
+        return baseTopic;
+      case QUERY:
+        return String.format("%s_%s_source", baseTopic, queryName);
+      case QUERY_AND_SALT:
+        return String.format("%s_%s_%d_source", baseTopic, queryName, now);
+    }
+    throw new RuntimeException("Unrecognized enum " + options.getResourceNameMode());
+  }
+
+  /**
+   * Return a subscription name.
+   */
+  private String shortSubscription(long now) {
+    String baseSubscription = options.getPubsubSubscription();
+    if (Strings.isNullOrEmpty(baseSubscription)) {
+      throw new RuntimeException("Missing --pubsubSubscription");
+    }
+    switch (options.getResourceNameMode()) {
+      case VERBATIM:
+        return baseSubscription;
+      case QUERY:
+        return String.format("%s_%s_source", baseSubscription, queryName);
+      case QUERY_AND_SALT:
+        return String.format("%s_%s_%d_source", baseSubscription, queryName, now);
+    }
+    throw new RuntimeException("Unrecognized enum " + options.getResourceNameMode());
+  }
+
+  /**
+   * Return a file name for plain text.
+   */
+  private String textFilename(long now) {
+    String baseFilename = options.getOutputPath();
+    if (Strings.isNullOrEmpty(baseFilename)) {
+      throw new RuntimeException("Missing --outputPath");
+    }
+    switch (options.getResourceNameMode()) {
+      case VERBATIM:
+        return baseFilename;
+      case QUERY:
+        return String.format("%s/nexmark_%s.txt", baseFilename, queryName);
+      case QUERY_AND_SALT:
+        return String.format("%s/nexmark_%s_%d.txt", baseFilename, queryName, now);
+    }
+    throw new RuntimeException("Unrecognized enum " + options.getResourceNameMode());
+  }
+
+  /**
+   * Return a BigQuery table spec.
+   */
+  private String tableSpec(long now, String version) {
+    String baseTableName = options.getBigQueryTable();
+    if (Strings.isNullOrEmpty(baseTableName)) {
+      throw new RuntimeException("Missing --bigQueryTable");
+    }
+    switch (options.getResourceNameMode()) {
+      case VERBATIM:
+        return String.format("%s:nexmark.%s_%s",
+                             options.getProject(), baseTableName, version);
+      case QUERY:
+        return String.format("%s:nexmark.%s_%s_%s",
+                             options.getProject(), baseTableName, queryName, version);
+      case QUERY_AND_SALT:
+        return String.format("%s:nexmark.%s_%s_%s_%d",
+                             options.getProject(), baseTableName, queryName, version, now);
+    }
+    throw new RuntimeException("Unrecognized enum " + options.getResourceNameMode());
+  }
+
+  /**
+   * Return a directory for logs.
+   */
+  private String logsDir(long now) {
+    String baseFilename = options.getOutputPath();
+    if (Strings.isNullOrEmpty(baseFilename)) {
+      throw new RuntimeException("Missing --outputPath");
+    }
+    switch (options.getResourceNameMode()) {
+      case VERBATIM:
+        return baseFilename;
+      case QUERY:
+        return String.format("%s/logs_%s", baseFilename, queryName);
+      case QUERY_AND_SALT:
+        return String.format("%s/logs_%s_%d", baseFilename, queryName, now);
+    }
+    throw new RuntimeException("Unrecognized enum " + options.getResourceNameMode());
+  }
+
+  /**
+   * Return a source of synthetic events.
+   */
+  private PCollection<Event> sourceEventsFromSynthetic(Pipeline p) {
+    if (isStreaming()) {
+      NexmarkUtils.console("Generating %d events in streaming mode", configuration.numEvents);
+      return p.apply(queryName + ".ReadUnbounded", NexmarkUtils.streamEventsSource(configuration));
+    } else {
+      NexmarkUtils.console("Generating %d events in batch mode", configuration.numEvents);
+      return p.apply(queryName + ".ReadBounded", NexmarkUtils.batchEventsSource(configuration));
+    }
+  }
+
+  /**
+   * Return source of events from Pubsub.
+   */
+  private PCollection<Event> sourceEventsFromPubsub(Pipeline p, long now) {
+    String shortSubscription = shortSubscription(now);
+    NexmarkUtils.console("Reading events from Pubsub %s", shortSubscription);
+
+    PubsubIO.Read<PubsubMessage> io =
+        PubsubIO.readPubsubMessagesWithAttributes().fromSubscription(shortSubscription)
+            .withIdAttribute(NexmarkUtils.PUBSUB_ID);
+    if (!configuration.usePubsubPublishTime) {
+      io = io.withTimestampAttribute(NexmarkUtils.PUBSUB_TIMESTAMP);
+    }
+
+    return p
+      .apply(queryName + ".ReadPubsubEvents", io)
+      .apply(queryName + ".PubsubMessageToEvent", ParDo.of(new DoFn<PubsubMessage, Event>() {
+        @ProcessElement
+        public void processElement(ProcessContext c) {
+          byte[] payload = c.element().getPayload();
+          try {
+            Event event = CoderUtils.decodeFromByteArray(Event.CODER, payload);
+            c.output(event);
+          } catch (CoderException e) {
+            // TODO Log decoding Event error
+          }
+        }
+      }));
+  }
+
+  /**
+   * Return Avro source of events from {@code options.getInputFilePrefix}.
+   */
+  private PCollection<Event> sourceEventsFromAvro(Pipeline p) {
+    String filename = options.getInputPath();
+    if (Strings.isNullOrEmpty(filename)) {
+      throw new RuntimeException("Missing --inputPath");
+    }
+    NexmarkUtils.console("Reading events from Avro files at %s", filename);
+    return p
+        .apply(queryName + ".ReadAvroEvents", AvroIO.read(Event.class)
+                          .from(filename + "*.avro"))
+        .apply("OutputWithTimestamp", NexmarkQuery.EVENT_TIMESTAMP_FROM_DATA);
+  }
+
+  /**
+   * Send {@code events} to Pubsub.
+   */
+  private void sinkEventsToPubsub(PCollection<Event> events, long now) {
+    String shortTopic = shortTopic(now);
+    NexmarkUtils.console("Writing events to Pubsub %s", shortTopic);
+
+    PubsubIO.Write<PubsubMessage> io =
+        PubsubIO.writePubsubMessages().to(shortTopic)
+            .withIdAttribute(NexmarkUtils.PUBSUB_ID);
+    if (!configuration.usePubsubPublishTime) {
+      io = io.withTimestampAttribute(NexmarkUtils.PUBSUB_TIMESTAMP);
+    }
+
+    events.apply(queryName + ".EventToPubsubMessage",
+            ParDo.of(new DoFn<Event, PubsubMessage>() {
+              @ProcessElement
+              public void processElement(ProcessContext c) {
+                try {
+                  byte[] payload = CoderUtils.encodeToByteArray(Event.CODER, c.element());
+                  c.output(new PubsubMessage(payload, new HashMap<String, String>()));
+                } catch (CoderException e1) {
+                  // TODO Log encoding Event error
+                }
+              }
+            })
+        )
+        .apply(queryName + ".WritePubsubEvents", io);
+  }
+
+  /**
+   * Send {@code formattedResults} to Pubsub.
+   */
+  private void sinkResultsToPubsub(PCollection<String> formattedResults, long now) {
+    String shortTopic = shortTopic(now);
+    NexmarkUtils.console("Writing results to Pubsub %s", shortTopic);
+    PubsubIO.Write<String> io =
+        PubsubIO.writeStrings().to(shortTopic)
+            .withIdAttribute(NexmarkUtils.PUBSUB_ID);
+    if (!configuration.usePubsubPublishTime) {
+      io = io.withTimestampAttribute(NexmarkUtils.PUBSUB_TIMESTAMP);
+    }
+    formattedResults.apply(queryName + ".WritePubsubResults", io);
+  }
+
+  /**
+   * Sink all raw Events in {@code source} to {@code options.getOutputPath}.
+   * This will configure the job to write the following files:
+   * <ul>
+   * <li>{@code $outputPath/event*.avro} All Event entities.
+   * <li>{@code $outputPath/auction*.avro} Auction entities.
+   * <li>{@code $outputPath/bid*.avro} Bid entities.
+   * <li>{@code $outputPath/person*.avro} Person entities.
+   * </ul>
+   *
+   * @param source A PCollection of events.
+   */
+  private void sinkEventsToAvro(PCollection<Event> source) {
+    String filename = options.getOutputPath();
+    if (Strings.isNullOrEmpty(filename)) {
+      throw new RuntimeException("Missing --outputPath");
+    }
+    NexmarkUtils.console("Writing events to Avro files at %s", filename);
+    source.apply(queryName + ".WriteAvroEvents",
+            AvroIO.write(Event.class).to(filename + "/event").withSuffix(".avro"));
+    source.apply(NexmarkQuery.JUST_BIDS)
+          .apply(queryName + ".WriteAvroBids",
+            AvroIO.write(Bid.class).to(filename + "/bid").withSuffix(".avro"));
+    source.apply(NexmarkQuery.JUST_NEW_AUCTIONS)
+          .apply(queryName + ".WriteAvroAuctions",
+            AvroIO.write(Auction.class).to(filename + "/auction").withSuffix(".avro"));
+    source.apply(NexmarkQuery.JUST_NEW_PERSONS)
+          .apply(queryName + ".WriteAvroPeople",
+            AvroIO.write(Person.class).to(filename + "/person").withSuffix(".avro"));
+  }
+
+  /**
+   * Send {@code formattedResults} to text files.
+   */
+  private void sinkResultsToText(PCollection<String> formattedResults, long now) {
+    String filename = textFilename(now);
+    NexmarkUtils.console("Writing results to text files at %s", filename);
+    formattedResults.apply(queryName + ".WriteTextResults",
+        TextIO.write().to(filename));
+  }
+
+  private static class StringToTableRow extends DoFn<String, TableRow> {
+    @ProcessElement
+    public void processElement(ProcessContext c) {
+      int n = ThreadLocalRandom.current().nextInt(10);
+      List<TableRow> records = new ArrayList<>(n);
+      for (int i = 0; i < n; i++) {
+        records.add(new TableRow().set("index", i).set("value", Integer.toString(i)));
+      }
+      c.output(new TableRow().set("result", c.element()).set("records", records));
+    }
+  }
+
+  /**
+   * Send {@code formattedResults} to BigQuery.
+   */
+  private void sinkResultsToBigQuery(
+      PCollection<String> formattedResults, long now,
+      String version) {
+    String tableSpec = tableSpec(now, version);
+    TableSchema tableSchema =
+        new TableSchema().setFields(ImmutableList.of(
+            new TableFieldSchema().setName("result").setType("STRING"),
+            new TableFieldSchema().setName("records").setMode("REPEATED").setType("RECORD")
+                                  .setFields(ImmutableList.of(
+                                      new TableFieldSchema().setName("index").setType("INTEGER"),
+                                      new TableFieldSchema().setName("value").setType("STRING")))));
+    NexmarkUtils.console("Writing results to BigQuery table %s", tableSpec);
+    BigQueryIO.Write io =
+        BigQueryIO.write().to(tableSpec)
+                        .withSchema(tableSchema)
+                        .withCreateDisposition(BigQueryIO.Write.CreateDisposition.CREATE_IF_NEEDED)
+                        .withWriteDisposition(BigQueryIO.Write.WriteDisposition.WRITE_APPEND);
+    formattedResults
+        .apply(queryName + ".StringToTableRow", ParDo.of(new StringToTableRow()))
+        .apply(queryName + ".WriteBigQueryResults", io);
+  }
+
+  // ================================================================================
+  // Construct overall pipeline
+  // ================================================================================
+
+  /**
+   * Return source of events for this run, or null if we are simply publishing events
+   * to Pubsub.
+   */
+  private PCollection<Event> createSource(Pipeline p, final long now) {
+    PCollection<Event> source = null;
+    switch (configuration.sourceType) {
+      case DIRECT:
+        source = sourceEventsFromSynthetic(p);
+        break;
+      case AVRO:
+        source = sourceEventsFromAvro(p);
+        break;
+      case PUBSUB:
+        // Setup the sink for the publisher.
+        switch (configuration.pubSubMode) {
+          case SUBSCRIBE_ONLY:
+            // Nothing to publish.
+            break;
+          case PUBLISH_ONLY:
+            // Send synthesized events to Pubsub in this job.
+            sinkEventsToPubsub(sourceEventsFromSynthetic(p).apply(queryName + ".Snoop",
+                    NexmarkUtils.snoop(queryName)), now);
+            break;
+          case COMBINED:
+            // Send synthesized events to Pubsub in separate publisher job.
+            // We won't start the main pipeline until the publisher has sent the pre-load events.
+            // We'll shutdown the publisher job when we notice the main job has finished.
+            invokeBuilderForPublishOnlyPipeline(new PipelineBuilder<NexmarkOptions>() {
+              @Override
+              public void build(NexmarkOptions publishOnlyOptions) {
+                Pipeline sp = Pipeline.create(options);
+                NexmarkUtils.setupPipeline(configuration.coderStrategy, sp);
+                publisherMonitor = new Monitor<>(queryName, "publisher");
+                sinkEventsToPubsub(
+                    sourceEventsFromSynthetic(sp)
+                            .apply(queryName + ".Monitor", publisherMonitor.getTransform()),
+                    now);
+                publisherResult = sp.run();
+              }
+            });
+            break;
+        }
+
+        // Setup the source for the consumer.
+        switch (configuration.pubSubMode) {
+          case PUBLISH_ONLY:
+            // Nothing to consume. Leave source null.
+            break;
+          case SUBSCRIBE_ONLY:
+          case COMBINED:
+            // Read events from pubsub.
+            source = sourceEventsFromPubsub(p, now);
+            break;
+        }
+        break;
+    }
+    return source;
+  }
+
+  private static final TupleTag<String> MAIN = new TupleTag<String>(){};
+  private static final TupleTag<String> SIDE = new TupleTag<String>(){};
+
+  private static class PartitionDoFn extends DoFn<String, String> {
+    @ProcessElement
+    public void processElement(ProcessContext c) {
+      if (c.element().hashCode() % 2 == 0) {
+        c.output(c.element());
+      } else {
+        c.output(SIDE, c.element());
+      }
+    }
+  }
+
+  /**
+   * Consume {@code results}.
+   */
+  private void sink(PCollection<TimestampedValue<KnownSize>> results, long now) {
+    if (configuration.sinkType == NexmarkUtils.SinkType.COUNT_ONLY) {
+      // Avoid the cost of formatting the results.
+      results.apply(queryName + ".DevNull", NexmarkUtils.devNull(queryName));
+      return;
+    }
+
+    PCollection<String> formattedResults =
+      results.apply(queryName + ".Format", NexmarkUtils.format(queryName));
+    if (options.getLogResults()) {
+      formattedResults = formattedResults.apply(queryName + ".Results.Log",
+              NexmarkUtils.<String>log(queryName + ".Results"));
+    }
+
+    switch (configuration.sinkType) {
+      case DEVNULL:
+        // Discard all results
+        formattedResults.apply(queryName + ".DevNull", NexmarkUtils.devNull(queryName));
+        break;
+      case PUBSUB:
+        sinkResultsToPubsub(formattedResults, now);
+        break;
+      case TEXT:
+        sinkResultsToText(formattedResults, now);
+        break;
+      case AVRO:
+        NexmarkUtils.console(
+            "WARNING: with --sinkType=AVRO, actual query results will be discarded.");
+        break;
+      case BIGQUERY:
+        // Multiple BigQuery backends to mimic what most customers do.
+        PCollectionTuple res = formattedResults.apply(queryName + ".Partition",
+            ParDo.of(new PartitionDoFn()).withOutputTags(MAIN, TupleTagList.of(SIDE)));
+        sinkResultsToBigQuery(res.get(MAIN), now, "main");
+        sinkResultsToBigQuery(res.get(SIDE), now, "side");
+        sinkResultsToBigQuery(formattedResults, now, "copy");
+        break;
+      case COUNT_ONLY:
+        // Short-circuited above.
+        throw new RuntimeException();
+    }
+  }
+
+  // ================================================================================
+  // Entry point
+  // ================================================================================
+
+  /**
+   * Calculate the distribution of the expected rate of results per minute (in event time, not
+   * wallclock time).
+   */
+  private void modelResultRates(NexmarkQueryModel model) {
+    List<Long> counts = Lists.newArrayList(model.simulator().resultsPerWindow());
+    Collections.sort(counts);
+    int n = counts.size();
+    if (n < 5) {
+      NexmarkUtils.console("Query%d: only %d samples", model.configuration.query, n);
+    } else {
+      NexmarkUtils.console("Query%d: N:%d; min:%d; 1st%%:%d; mean:%d; 3rd%%:%d; max:%d",
+                           model.configuration.query, n, counts.get(0), counts.get(n / 4),
+                           counts.get(n / 2),
+                           counts.get(n - 1 - n / 4), counts.get(n - 1));
+    }
+  }
+
+  /**
+   * Run {@code configuration} and return its performance if possible.
+   */
+  @Nullable
+  public NexmarkPerf run(NexmarkConfiguration runConfiguration) {
+    if (options.getManageResources() && !options.getMonitorJobs()) {
+      throw new RuntimeException("If using --manageResources then must also use --monitorJobs.");
+    }
+
+    //
+    // Setup per-run state.
+    //
+    checkState(configuration == null);
+    checkState(queryName == null);
+    configuration = runConfiguration;
+
+    try {
+      NexmarkUtils.console("Running %s", configuration.toShortString());
+
+      if (configuration.numEvents < 0) {
+        NexmarkUtils.console("skipping since configuration is disabled");
+        return null;
+      }
+
+      List<NexmarkQuery> queries = Arrays.asList(new Query0(configuration),
+                                                 new Query1(configuration),
+                                                 new Query2(configuration),
+                                                 new Query3(configuration),
+                                                 new Query4(configuration),
+                                                 new Query5(configuration),
+                                                 new Query6(configuration),
+                                                 new Query7(configuration),
+                                                 new Query8(configuration),
+                                                 new Query9(configuration),
+                                                 new Query10(configuration),
+                                                 new Query11(configuration),
+                                                 new Query12(configuration));
+      NexmarkQuery query = queries.get(configuration.query);
+      queryName = query.getName();
+
+      List<NexmarkQueryModel> models = Arrays.asList(
+          new Query0Model(configuration),
+          new Query1Model(configuration),
+          new Query2Model(configuration),
+          new Query3Model(configuration),
+          new Query4Model(configuration),
+          new Query5Model(configuration),
+          new Query6Model(configuration),
+          new Query7Model(configuration),
+          new Query8Model(configuration),
+          new Query9Model(configuration),
+          null,
+          null,
+          null);
+      NexmarkQueryModel model = models.get(configuration.query);
+
+      if (options.getJustModelResultRate()) {
+        if (model == null) {
+          throw new RuntimeException(String.format("No model for %s", queryName));
+        }
+        modelResultRates(model);
+        return null;
+      }
+
+      long now = System.currentTimeMillis();
+      Pipeline p = Pipeline.create(options);
+      NexmarkUtils.setupPipeline(configuration.coderStrategy, p);
+
+      // Generate events.
+      PCollection<Event> source = createSource(p, now);
+
+      if (options.getLogEvents()) {
+        source = source.apply(queryName + ".Events.Log",
+                NexmarkUtils.<Event>log(queryName + ".Events"));
+      }
+
+      // Source will be null if source type is PUBSUB and mode is PUBLISH_ONLY.
+      // In that case there's nothing more to add to pipeline.
+      if (source != null) {
+        // Optionally sink events in Avro format.
+        // (Query results are ignored).
+        if (configuration.sinkType == NexmarkUtils.SinkType.AVRO) {
+          sinkEventsToAvro(source);
+        }
+
+        // Special hacks for Query 10 (big logger).
+        if (configuration.query == 10) {
+          String path = null;
+          if (options.getOutputPath() != null && !options.getOutputPath().isEmpty()) {
+            path = logsDir(now);
+          }
+          ((Query10) query).setOutputPath(path);
+          ((Query10) query).setMaxNumWorkers(maxNumWorkers());
+        }
+
+        // Apply query.
+        PCollection<TimestampedValue<KnownSize>> results = source.apply(query);
+
+        if (options.getAssertCorrectness()) {
+          if (model == null) {
+            throw new RuntimeException(String.format("No model for %s", queryName));
+          }
+          // We know all our streams have a finite number of elements.
+          results.setIsBoundedInternal(PCollection.IsBounded.BOUNDED);
+          // If we have a finite number of events then assert our pipeline's
+          // results match those of a model using the same sequence of events.
+          PAssert.that(results).satisfies(model.assertionFor());
+        }
+
+        // Output results.
+        sink(results, now);
+      }
+
+      if (publisherResult != null) {
+        waitForPublisherPreload();
+      }
+      mainResult = p.run();
+      mainResult.waitUntilFinish(Duration.standardSeconds(configuration.streamTimeout));
+      return monitor(query);
+    } finally {
+      configuration = null;
+      queryName = null;
+    }
+  }
+}


[48/55] [abbrv] beam git commit: Move module beam-integration-java-nexmark to beam-sdks-java-nexmark

Posted by ie...@apache.org.
http://git-wip-us.apache.org/repos/asf/beam/blob/f4333df7/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query4.java
----------------------------------------------------------------------
diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query4.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query4.java
deleted file mode 100644
index 9c0fe6d..0000000
--- a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query4.java
+++ /dev/null
@@ -1,116 +0,0 @@
-/*
- * 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.beam.integration.nexmark.queries;
-
-import org.apache.beam.integration.nexmark.Monitor;
-import org.apache.beam.integration.nexmark.NexmarkConfiguration;
-import org.apache.beam.integration.nexmark.NexmarkUtils;
-import org.apache.beam.integration.nexmark.model.Auction;
-import org.apache.beam.integration.nexmark.model.AuctionBid;
-import org.apache.beam.integration.nexmark.model.Bid;
-import org.apache.beam.integration.nexmark.model.CategoryPrice;
-import org.apache.beam.integration.nexmark.model.Event;
-import org.apache.beam.integration.nexmark.model.KnownSize;
-import org.apache.beam.sdk.transforms.DoFn;
-import org.apache.beam.sdk.transforms.Mean;
-import org.apache.beam.sdk.transforms.ParDo;
-import org.apache.beam.sdk.transforms.windowing.SlidingWindows;
-import org.apache.beam.sdk.transforms.windowing.Window;
-import org.apache.beam.sdk.values.KV;
-import org.apache.beam.sdk.values.PCollection;
-import org.joda.time.Duration;
-
-/**
- * Query 4, 'Average Price for a Category'. Select the average of the wining bid prices for all
- * closed auctions in each category. In CQL syntax:
- *
- * <pre>{@code
- * SELECT Istream(AVG(Q.final))
- * FROM Category C, (SELECT Rstream(MAX(B.price) AS final, A.category)
- *                   FROM Auction A [ROWS UNBOUNDED], Bid B [ROWS UNBOUNDED]
- *                   WHERE A.id=B.auction AND B.datetime < A.expires AND A.expires < CURRENT_TIME
- *                   GROUP BY A.id, A.category) Q
- * WHERE Q.category = C.id
- * GROUP BY C.id;
- * }</pre>
- *
- * <p>For extra spiciness our implementation differs slightly from the above:
- * <ul>
- * <li>We select both the average winning price and the category.
- * <li>We don't bother joining with a static category table, since it's contents are never used.
- * <li>We only consider bids which are above the auction's reserve price.
- * <li>We accept the highest-price, earliest valid bid as the winner.
- * <li>We calculate the averages oven a sliding window of size {@code windowSizeSec} and
- * period {@code windowPeriodSec}.
- * </ul>
- */
-public class Query4 extends NexmarkQuery {
-  private final Monitor<AuctionBid> winningBidsMonitor;
-
-  public Query4(NexmarkConfiguration configuration) {
-    super(configuration, "Query4");
-    winningBidsMonitor = new Monitor<>(name + ".WinningBids", "winning");
-  }
-
-  private PCollection<CategoryPrice> applyTyped(PCollection<Event> events) {
-    PCollection<AuctionBid> winningBids =
-        events
-            // Find the winning bid for each closed auction.
-            .apply(new WinningBids(name + ".WinningBids", configuration));
-
-    // Monitor winning bids
-    winningBids = winningBids.apply(name + ".WinningBidsMonitor",
-            winningBidsMonitor.getTransform());
-
-    return winningBids
-        // Key the winning bid price by the auction category.
-        .apply(name + ".Rekey",
-            ParDo.of(new DoFn<AuctionBid, KV<Long, Long>>() {
-                  @ProcessElement
-                  public void processElement(ProcessContext c) {
-                    Auction auction = c.element().auction;
-                    Bid bid = c.element().bid;
-                    c.output(KV.of(auction.category, bid.price));
-                  }
-                }))
-
-        // Re-window so we can calculate a sliding average
-        .apply(Window.<KV<Long, Long>>into(
-            SlidingWindows.of(Duration.standardSeconds(configuration.windowSizeSec))
-                .every(Duration.standardSeconds(configuration.windowPeriodSec))))
-
-        // Find the average of the winning bids for each category.
-        // Make sure we share the work for each category between workers.
-        .apply(Mean.<Long, Long>perKey().withHotKeyFanout(configuration.fanout))
-
-        // For testing against Query4Model, capture which results are 'final'.
-        .apply(name + ".Project",
-            ParDo.of(new DoFn<KV<Long, Double>, CategoryPrice>() {
-                  @ProcessElement
-                  public void processElement(ProcessContext c) {
-                    c.output(new CategoryPrice(c.element().getKey(),
-                        Math.round(c.element().getValue()), c.pane().isLast()));
-                  }
-                }));
-  }
-
-  @Override
-  protected PCollection<KnownSize> applyPrim(PCollection<Event> events) {
-    return NexmarkUtils.castToKnownSize(name, applyTyped(events));
-  }
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/f4333df7/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query4Model.java
----------------------------------------------------------------------
diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query4Model.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query4Model.java
deleted file mode 100644
index 269e47a..0000000
--- a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query4Model.java
+++ /dev/null
@@ -1,186 +0,0 @@
-/*
- * 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.beam.integration.nexmark.queries;
-
-import java.io.Serializable;
-import java.util.ArrayList;
-import java.util.Collection;
-import java.util.Iterator;
-import java.util.List;
-import java.util.Map;
-import java.util.TreeMap;
-import org.apache.beam.integration.nexmark.NexmarkConfiguration;
-import org.apache.beam.integration.nexmark.NexmarkUtils;
-import org.apache.beam.integration.nexmark.model.Auction;
-import org.apache.beam.integration.nexmark.model.AuctionBid;
-import org.apache.beam.integration.nexmark.model.Bid;
-import org.apache.beam.integration.nexmark.model.CategoryPrice;
-import org.apache.beam.integration.nexmark.model.KnownSize;
-import org.apache.beam.sdk.transforms.windowing.BoundedWindow;
-import org.apache.beam.sdk.values.TimestampedValue;
-import org.joda.time.Duration;
-import org.joda.time.Instant;
-import org.junit.Assert;
-
-/**
- * A direct implementation of {@link Query4}.
- */
-public class Query4Model extends NexmarkQueryModel implements Serializable {
-  /**
-   * Simulator for query 4.
-   */
-  private class Simulator extends AbstractSimulator<AuctionBid, CategoryPrice> {
-    /** The prices and categories for all winning bids in the last window size. */
-    private final List<TimestampedValue<CategoryPrice>> winningPricesByCategory;
-
-    /** Timestamp of last result (ms since epoch). */
-    private Instant lastTimestamp;
-
-    /** When oldest active window starts. */
-    private Instant windowStart;
-
-    /** The last seen result for each category. */
-    private final Map<Long, TimestampedValue<CategoryPrice>> lastSeenResults;
-
-    public Simulator(NexmarkConfiguration configuration) {
-      super(new WinningBidsSimulator(configuration).results());
-      winningPricesByCategory = new ArrayList<>();
-      lastTimestamp = BoundedWindow.TIMESTAMP_MIN_VALUE;
-      windowStart = NexmarkUtils.BEGINNING_OF_TIME;
-      lastSeenResults = new TreeMap<>();
-    }
-
-    /**
-     * Calculate the average bid price for each category for all winning bids
-     * which are strictly before {@code end}.
-     */
-    private void averages(Instant end) {
-      Map<Long, Long> counts = new TreeMap<>();
-      Map<Long, Long> totals = new TreeMap<>();
-      for (TimestampedValue<CategoryPrice> value : winningPricesByCategory) {
-        if (!value.getTimestamp().isBefore(end)) {
-          continue;
-        }
-        long category = value.getValue().category;
-        long price = value.getValue().price;
-        Long count = counts.get(category);
-        if (count == null) {
-          count = 1L;
-        } else {
-          count += 1;
-        }
-        counts.put(category, count);
-        Long total = totals.get(category);
-        if (total == null) {
-          total = price;
-        } else {
-          total += price;
-        }
-        totals.put(category, total);
-      }
-      for (Map.Entry<Long, Long> entry : counts.entrySet()) {
-        long category = entry.getKey();
-        long count = entry.getValue();
-        long total = totals.get(category);
-        TimestampedValue<CategoryPrice> result = TimestampedValue.of(
-            new CategoryPrice(category, Math.round((double) total / count), true), lastTimestamp);
-        addIntermediateResult(result);
-        lastSeenResults.put(category, result);
-      }
-    }
-
-    /**
-     * Calculate averages for any windows which can now be retired. Also prune entries
-     * which can no longer contribute to any future window.
-     */
-    private void prune(Instant newWindowStart) {
-      while (!newWindowStart.equals(windowStart)) {
-        averages(windowStart.plus(Duration.standardSeconds(configuration.windowSizeSec)));
-        windowStart = windowStart.plus(Duration.standardSeconds(configuration.windowPeriodSec));
-        Iterator<TimestampedValue<CategoryPrice>> itr = winningPricesByCategory.iterator();
-        while (itr.hasNext()) {
-          if (itr.next().getTimestamp().isBefore(windowStart)) {
-            itr.remove();
-          }
-        }
-        if (winningPricesByCategory.isEmpty()) {
-          windowStart = newWindowStart;
-        }
-      }
-    }
-
-    /**
-     * Capture the winning bid.
-     */
-    private void captureWinningBid(Auction auction, Bid bid, Instant timestamp) {
-      winningPricesByCategory.add(
-          TimestampedValue.of(new CategoryPrice(auction.category, bid.price, false), timestamp));
-    }
-
-    @Override
-    protected void run() {
-      TimestampedValue<AuctionBid> timestampedWinningBid = nextInput();
-      if (timestampedWinningBid == null) {
-        prune(NexmarkUtils.END_OF_TIME);
-        for (TimestampedValue<CategoryPrice> result : lastSeenResults.values()) {
-          addResult(result);
-        }
-        allDone();
-        return;
-      }
-      lastTimestamp = timestampedWinningBid.getTimestamp();
-      Instant newWindowStart = windowStart(Duration.standardSeconds(configuration.windowSizeSec),
-          Duration.standardSeconds(configuration.windowPeriodSec), lastTimestamp);
-      prune(newWindowStart);
-      captureWinningBid(timestampedWinningBid.getValue().auction,
-          timestampedWinningBid.getValue().bid, lastTimestamp);
-    }
-  }
-
-  public Query4Model(NexmarkConfiguration configuration) {
-    super(configuration);
-  }
-
-  @Override
-  public AbstractSimulator<?, ?> simulator() {
-    return new Simulator(configuration);
-  }
-
-  @Override
-  protected Iterable<TimestampedValue<KnownSize>> relevantResults(
-      Iterable<TimestampedValue<KnownSize>> results) {
-    // Find the last (in processing time) reported average price for each category.
-    Map<Long, TimestampedValue<KnownSize>> finalAverages = new TreeMap<>();
-    for (TimestampedValue<KnownSize> obj : results) {
-      Assert.assertTrue("have CategoryPrice", obj.getValue() instanceof CategoryPrice);
-      CategoryPrice categoryPrice = (CategoryPrice) obj.getValue();
-      if (categoryPrice.isLast) {
-        finalAverages.put(
-            categoryPrice.category,
-            TimestampedValue.of((KnownSize) categoryPrice, obj.getTimestamp()));
-      }
-    }
-
-    return finalAverages.values();
-  }
-
-  @Override
-  protected <T> Collection<String> toCollection(Iterator<TimestampedValue<T>> itr) {
-    return toValue(itr);
-  }
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/f4333df7/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query5.java
----------------------------------------------------------------------
diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query5.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query5.java
deleted file mode 100644
index bdf3e5f..0000000
--- a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query5.java
+++ /dev/null
@@ -1,138 +0,0 @@
-/*
- * 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.beam.integration.nexmark.queries;
-
-import java.util.ArrayList;
-import java.util.Collections;
-import java.util.List;
-import org.apache.beam.integration.nexmark.NexmarkConfiguration;
-import org.apache.beam.integration.nexmark.NexmarkUtils;
-import org.apache.beam.integration.nexmark.model.AuctionCount;
-import org.apache.beam.integration.nexmark.model.Bid;
-import org.apache.beam.integration.nexmark.model.Event;
-import org.apache.beam.integration.nexmark.model.KnownSize;
-import org.apache.beam.sdk.transforms.Combine;
-import org.apache.beam.sdk.transforms.Count;
-import org.apache.beam.sdk.transforms.DoFn;
-import org.apache.beam.sdk.transforms.ParDo;
-import org.apache.beam.sdk.transforms.windowing.SlidingWindows;
-import org.apache.beam.sdk.transforms.windowing.Window;
-import org.apache.beam.sdk.values.KV;
-import org.apache.beam.sdk.values.PCollection;
-import org.joda.time.Duration;
-
-/**
- * Query 5, 'Hot Items'. Which auctions have seen the most bids in the last hour (updated every
- * minute). In CQL syntax:
- *
- * <pre>{@code
- * SELECT Rstream(auction)
- * FROM (SELECT B1.auction, count(*) AS num
- *       FROM Bid [RANGE 60 MINUTE SLIDE 1 MINUTE] B1
- *       GROUP BY B1.auction)
- * WHERE num >= ALL (SELECT count(*)
- *                   FROM Bid [RANGE 60 MINUTE SLIDE 1 MINUTE] B2
- *                   GROUP BY B2.auction);
- * }</pre>
- *
- * <p>To make things a bit more dynamic and easier to test we use much shorter windows, and
- * we'll also preserve the bid counts.
- */
-public class Query5 extends NexmarkQuery {
-  public Query5(NexmarkConfiguration configuration) {
-    super(configuration, "Query5");
-  }
-
-  private PCollection<AuctionCount> applyTyped(PCollection<Event> events) {
-    return events
-        // Only want the bid events.
-        .apply(JUST_BIDS)
-        // Window the bids into sliding windows.
-        .apply(
-            Window.<Bid>into(
-                SlidingWindows.of(Duration.standardSeconds(configuration.windowSizeSec))
-                    .every(Duration.standardSeconds(configuration.windowPeriodSec))))
-        // Project just the auction id.
-        .apply("BidToAuction", BID_TO_AUCTION)
-
-        // Count the number of bids per auction id.
-        .apply(Count.<Long>perElement())
-
-        // We'll want to keep all auctions with the maximal number of bids.
-        // Start by lifting each into a singleton list.
-        // need to do so because bellow combine returns a list of auctions in the key in case of
-        // equal number of bids. Combine needs to have same input type and return type.
-        .apply(
-            name + ".ToSingletons",
-            ParDo.of(
-                new DoFn<KV<Long, Long>, KV<List<Long>, Long>>() {
-                  @ProcessElement
-                  public void processElement(ProcessContext c) {
-                    c.output(
-                        KV.of(
-                            Collections.singletonList(c.element().getKey()),
-                            c.element().getValue()));
-                  }
-                }))
-
-        // Keep only the auction ids with the most bids.
-        .apply(
-            Combine.globally(
-                    new Combine.BinaryCombineFn<KV<List<Long>, Long>>() {
-                      @Override
-                      public KV<List<Long>, Long> apply(
-                          KV<List<Long>, Long> left, KV<List<Long>, Long> right) {
-                        List<Long> leftBestAuctions = left.getKey();
-                        long leftCount = left.getValue();
-                        List<Long> rightBestAuctions = right.getKey();
-                        long rightCount = right.getValue();
-                        if (leftCount > rightCount) {
-                          return left;
-                        } else if (leftCount < rightCount) {
-                          return right;
-                        } else {
-                          List<Long> newBestAuctions = new ArrayList<>();
-                          newBestAuctions.addAll(leftBestAuctions);
-                          newBestAuctions.addAll(rightBestAuctions);
-                          return KV.of(newBestAuctions, leftCount);
-                        }
-                      }
-                    })
-                .withoutDefaults()
-                .withFanout(configuration.fanout))
-
-        // Project into result.
-        .apply(
-            name + ".Select",
-            ParDo.of(
-                new DoFn<KV<List<Long>, Long>, AuctionCount>() {
-                  @ProcessElement
-                  public void processElement(ProcessContext c) {
-                    long count = c.element().getValue();
-                    for (long auction : c.element().getKey()) {
-                      c.output(new AuctionCount(auction, count));
-                    }
-                  }
-                }));
-  }
-
-  @Override
-  protected PCollection<KnownSize> applyPrim(PCollection<Event> events) {
-    return NexmarkUtils.castToKnownSize(name, applyTyped(events));
-  }
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/f4333df7/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query5Model.java
----------------------------------------------------------------------
diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query5Model.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query5Model.java
deleted file mode 100644
index 24d9a00..0000000
--- a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query5Model.java
+++ /dev/null
@@ -1,176 +0,0 @@
-/*
- * 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.beam.integration.nexmark.queries;
-
-import java.io.Serializable;
-import java.util.ArrayList;
-import java.util.Collection;
-import java.util.Iterator;
-import java.util.List;
-import java.util.Map;
-import java.util.TreeMap;
-import org.apache.beam.integration.nexmark.NexmarkConfiguration;
-import org.apache.beam.integration.nexmark.NexmarkUtils;
-import org.apache.beam.integration.nexmark.model.AuctionCount;
-import org.apache.beam.integration.nexmark.model.Bid;
-import org.apache.beam.integration.nexmark.model.Event;
-import org.apache.beam.sdk.values.TimestampedValue;
-import org.joda.time.Duration;
-import org.joda.time.Instant;
-
-/**
- * A direct implementation of {@link Query5}.
- */
-public class Query5Model extends NexmarkQueryModel implements Serializable {
-  /**
-   * Simulator for query 5.
-   */
-  private class Simulator extends AbstractSimulator<Event, AuctionCount> {
-    /** Time of bids still contributing to open windows, indexed by their auction id. */
-    private final Map<Long, List<Instant>> bids;
-
-    /** When oldest active window starts. */
-    private Instant windowStart;
-
-    public Simulator(NexmarkConfiguration configuration) {
-      super(NexmarkUtils.standardEventIterator(configuration));
-      bids = new TreeMap<>();
-      windowStart = NexmarkUtils.BEGINNING_OF_TIME;
-    }
-
-    /**
-     * Count bids per auction id for bids strictly before {@code end}. Add the auction ids with
-     * the maximum number of bids to results.
-     */
-    private void countBids(Instant end) {
-      Map<Long, Long> counts = new TreeMap<>();
-      long maxCount = 0L;
-      for (Map.Entry<Long, List<Instant>> entry : bids.entrySet()) {
-        long count = 0L;
-        long auction = entry.getKey();
-        for (Instant bid : entry.getValue()) {
-          if (bid.isBefore(end)) {
-            count++;
-          }
-        }
-        if (count > 0) {
-          counts.put(auction, count);
-          maxCount = Math.max(maxCount, count);
-        }
-      }
-      for (Map.Entry<Long, Long> entry : counts.entrySet()) {
-        long auction = entry.getKey();
-        long count = entry.getValue();
-        if (count == maxCount) {
-          AuctionCount result = new AuctionCount(auction, count);
-          addResult(TimestampedValue.of(result, end));
-        }
-      }
-    }
-
-    /**
-     * Retire bids which are strictly before {@code cutoff}. Return true if there are any bids
-     * remaining.
-     */
-    private boolean retireBids(Instant cutoff) {
-      boolean anyRemain = false;
-      for (Map.Entry<Long, List<Instant>> entry : bids.entrySet()) {
-        long auction = entry.getKey();
-        Iterator<Instant> itr = entry.getValue().iterator();
-        while (itr.hasNext()) {
-          Instant bid = itr.next();
-          if (bid.isBefore(cutoff)) {
-            NexmarkUtils.info("retire: %s for %s", bid, auction);
-            itr.remove();
-          } else {
-            anyRemain = true;
-          }
-        }
-      }
-      return anyRemain;
-    }
-
-    /**
-     * Retire active windows until we've reached {@code newWindowStart}.
-     */
-    private void retireWindows(Instant newWindowStart) {
-      while (!newWindowStart.equals(windowStart)) {
-        NexmarkUtils.info("retiring window %s, aiming for %s", windowStart, newWindowStart);
-        // Count bids in the window (windowStart, windowStart + size].
-        countBids(windowStart.plus(Duration.standardSeconds(configuration.windowSizeSec)));
-        // Advance the window.
-        windowStart = windowStart.plus(Duration.standardSeconds(configuration.windowPeriodSec));
-        // Retire bids which will never contribute to a future window.
-        if (!retireBids(windowStart)) {
-          // Can fast forward to latest window since no more outstanding bids.
-          windowStart = newWindowStart;
-        }
-      }
-    }
-
-    /**
-     * Add bid to state.
-     */
-    private void captureBid(Bid bid, Instant timestamp) {
-      List<Instant> existing = bids.get(bid.auction);
-      if (existing == null) {
-        existing = new ArrayList<>();
-        bids.put(bid.auction, existing);
-      }
-      existing.add(timestamp);
-    }
-
-    @Override
-    public void run() {
-      TimestampedValue<Event> timestampedEvent = nextInput();
-      if (timestampedEvent == null) {
-        // Drain the remaining windows.
-        retireWindows(NexmarkUtils.END_OF_TIME);
-        allDone();
-        return;
-      }
-
-      Event event = timestampedEvent.getValue();
-      if (event.bid == null) {
-        // Ignore non-bid events.
-        return;
-      }
-      Instant timestamp = timestampedEvent.getTimestamp();
-      Instant newWindowStart = windowStart(Duration.standardSeconds(configuration.windowSizeSec),
-          Duration.standardSeconds(configuration.windowPeriodSec), timestamp);
-      // Capture results from any windows we can now retire.
-      retireWindows(newWindowStart);
-      // Capture current bid.
-      captureBid(event.bid, timestamp);
-    }
-  }
-
-  public Query5Model(NexmarkConfiguration configuration) {
-    super(configuration);
-  }
-
-  @Override
-  public AbstractSimulator<?, ?> simulator() {
-    return new Simulator(configuration);
-  }
-
-  @Override
-  protected <T> Collection<String> toCollection(Iterator<TimestampedValue<T>> itr) {
-    return toValue(itr);
-  }
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/f4333df7/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query6.java
----------------------------------------------------------------------
diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query6.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query6.java
deleted file mode 100644
index ea39ede..0000000
--- a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query6.java
+++ /dev/null
@@ -1,155 +0,0 @@
-/*
- * 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.beam.integration.nexmark.queries;
-
-import com.google.common.collect.Lists;
-import java.util.ArrayList;
-import java.util.Collections;
-import java.util.List;
-import org.apache.beam.integration.nexmark.NexmarkConfiguration;
-import org.apache.beam.integration.nexmark.NexmarkUtils;
-import org.apache.beam.integration.nexmark.model.Auction;
-import org.apache.beam.integration.nexmark.model.AuctionBid;
-import org.apache.beam.integration.nexmark.model.Bid;
-import org.apache.beam.integration.nexmark.model.Event;
-import org.apache.beam.integration.nexmark.model.KnownSize;
-import org.apache.beam.integration.nexmark.model.SellerPrice;
-import org.apache.beam.sdk.transforms.Combine;
-import org.apache.beam.sdk.transforms.DoFn;
-import org.apache.beam.sdk.transforms.ParDo;
-import org.apache.beam.sdk.transforms.windowing.AfterPane;
-import org.apache.beam.sdk.transforms.windowing.GlobalWindows;
-import org.apache.beam.sdk.transforms.windowing.Repeatedly;
-import org.apache.beam.sdk.transforms.windowing.Window;
-import org.apache.beam.sdk.values.KV;
-import org.apache.beam.sdk.values.PCollection;
-import org.joda.time.Duration;
-
-/**
- * Query 6, 'Average Selling Price by Seller'. Select the average selling price over the
- * last 10 closed auctions by the same seller. In CQL syntax:
- *
- * <pre>{@code
- * SELECT Istream(AVG(Q.final), Q.seller)
- * FROM (SELECT Rstream(MAX(B.price) AS final, A.seller)
- *       FROM Auction A [ROWS UNBOUNDED], Bid B [ROWS UNBOUNDED]
- *       WHERE A.id=B.auction AND B.datetime < A.expires AND A.expires < CURRENT_TIME
- *       GROUP BY A.id, A.seller) [PARTITION BY A.seller ROWS 10] Q
- * GROUP BY Q.seller;
- * }</pre>
- *
- * <p>We are a little more exact with selecting winning bids: see {@link WinningBids}.
- */
-public class Query6 extends NexmarkQuery {
-  /**
-   * Combiner to keep track of up to {@code maxNumBids} of the most recent wining bids and calculate
-   * their average selling price.
-   */
-  private static class MovingMeanSellingPrice extends Combine.CombineFn<Bid, List<Bid>, Long> {
-    private final int maxNumBids;
-
-    public MovingMeanSellingPrice(int maxNumBids) {
-      this.maxNumBids = maxNumBids;
-    }
-
-    @Override
-    public List<Bid> createAccumulator() {
-      return new ArrayList<>();
-    }
-
-    @Override
-    public List<Bid> addInput(List<Bid> accumulator, Bid input) {
-      accumulator.add(input);
-      Collections.sort(accumulator, Bid.ASCENDING_TIME_THEN_PRICE);
-      if (accumulator.size() > maxNumBids) {
-        accumulator.remove(0);
-      }
-      return accumulator;
-    }
-
-    @Override
-    public List<Bid> mergeAccumulators(Iterable<List<Bid>> accumulators) {
-      List<Bid> result = new ArrayList<>();
-      for (List<Bid> accumulator : accumulators) {
-        result.addAll(accumulator);
-      }
-      Collections.sort(result, Bid.ASCENDING_TIME_THEN_PRICE);
-      if (result.size() > maxNumBids) {
-        result = Lists.newArrayList(result.listIterator(result.size() - maxNumBids));
-      }
-      return result;
-    }
-
-    @Override
-    public Long extractOutput(List<Bid> accumulator) {
-      if (accumulator.isEmpty()) {
-        return 0L;
-      }
-      long sumOfPrice = 0;
-      for (Bid bid : accumulator) {
-        sumOfPrice += bid.price;
-      }
-      return Math.round((double) sumOfPrice / accumulator.size());
-    }
-  }
-
-  public Query6(NexmarkConfiguration configuration) {
-    super(configuration, "Query6");
-  }
-
-  private PCollection<SellerPrice> applyTyped(PCollection<Event> events) {
-    return events
-        // Find the winning bid for each closed auction.
-        .apply(new WinningBids(name + ".WinningBids", configuration))
-
-        // Key the winning bid by the seller id.
-        .apply(name + ".Rekey",
-            ParDo.of(new DoFn<AuctionBid, KV<Long, Bid>>() {
-                  @ProcessElement
-                  public void processElement(ProcessContext c) {
-                    Auction auction = c.element().auction;
-                    Bid bid = c.element().bid;
-                    c.output(KV.of(auction.seller, bid));
-                  }
-                }))
-
-        // Re-window to update on every wining bid.
-        .apply(
-            Window.<KV<Long, Bid>>into(new GlobalWindows())
-                .triggering(Repeatedly.forever(AfterPane.elementCountAtLeast(1)))
-                .accumulatingFiredPanes()
-                .withAllowedLateness(Duration.ZERO))
-
-        // Find the average of last 10 winning bids for each seller.
-        .apply(Combine.<Long, Bid, Long>perKey(new MovingMeanSellingPrice(10)))
-
-        // Project into our datatype.
-        .apply(name + ".Select",
-            ParDo.of(new DoFn<KV<Long, Long>, SellerPrice>() {
-                  @ProcessElement
-                  public void processElement(ProcessContext c) {
-                    c.output(new SellerPrice(c.element().getKey(), c.element().getValue()));
-                  }
-                }));
-  }
-
-  @Override
-  protected PCollection<KnownSize> applyPrim(PCollection<Event> events) {
-    return NexmarkUtils.castToKnownSize(name, applyTyped(events));
-  }
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/f4333df7/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query6Model.java
----------------------------------------------------------------------
diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query6Model.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query6Model.java
deleted file mode 100644
index 9cb8b3d..0000000
--- a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query6Model.java
+++ /dev/null
@@ -1,133 +0,0 @@
-/*
- * 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.beam.integration.nexmark.queries;
-
-import java.io.Serializable;
-import java.util.Collection;
-import java.util.Iterator;
-import java.util.Map;
-import java.util.TreeMap;
-import org.apache.beam.integration.nexmark.NexmarkConfiguration;
-import org.apache.beam.integration.nexmark.NexmarkUtils;
-import org.apache.beam.integration.nexmark.model.Auction;
-import org.apache.beam.integration.nexmark.model.AuctionBid;
-import org.apache.beam.integration.nexmark.model.Bid;
-import org.apache.beam.integration.nexmark.model.KnownSize;
-import org.apache.beam.integration.nexmark.model.SellerPrice;
-import org.apache.beam.sdk.transforms.windowing.BoundedWindow;
-import org.apache.beam.sdk.values.TimestampedValue;
-import org.joda.time.Instant;
-import org.junit.Assert;
-
-/**
- * A direct implementation of {@link Query6}.
- */
-public class Query6Model extends NexmarkQueryModel implements Serializable {
-  /**
-   * Simulator for query 6.
-   */
-  private static class Simulator extends AbstractSimulator<AuctionBid, SellerPrice> {
-    /** The cumulative count of winning bids, indexed by seller id. */
-    private final Map<Long, Long> numWinningBidsPerSeller;
-
-    /** The cumulative total of winning bid prices, indexed by seller id. */
-    private final Map<Long, Long> totalWinningBidPricesPerSeller;
-
-    private Instant lastTimestamp;
-
-    public Simulator(NexmarkConfiguration configuration) {
-      super(new WinningBidsSimulator(configuration).results());
-      numWinningBidsPerSeller = new TreeMap<>();
-      totalWinningBidPricesPerSeller = new TreeMap<>();
-      lastTimestamp = BoundedWindow.TIMESTAMP_MIN_VALUE;
-    }
-
-    /**
-     * Update the per-seller running counts/sums.
-     */
-    private void captureWinningBid(Auction auction, Bid bid, Instant timestamp) {
-      NexmarkUtils.info("winning auction, bid: %s, %s", auction, bid);
-      Long count = numWinningBidsPerSeller.get(auction.seller);
-      if (count == null) {
-        count = 1L;
-      } else {
-        count += 1;
-      }
-      numWinningBidsPerSeller.put(auction.seller, count);
-      Long total = totalWinningBidPricesPerSeller.get(auction.seller);
-      if (total == null) {
-        total = bid.price;
-      } else {
-        total += bid.price;
-      }
-      totalWinningBidPricesPerSeller.put(auction.seller, total);
-      TimestampedValue<SellerPrice> intermediateResult = TimestampedValue.of(
-          new SellerPrice(auction.seller, Math.round((double) total / count)), timestamp);
-      addIntermediateResult(intermediateResult);
-    }
-
-
-    @Override
-    protected void run() {
-      TimestampedValue<AuctionBid> timestampedWinningBid = nextInput();
-      if (timestampedWinningBid == null) {
-        for (Map.Entry<Long, Long> entry : numWinningBidsPerSeller.entrySet()) {
-          long seller = entry.getKey();
-          long count = entry.getValue();
-          long total = totalWinningBidPricesPerSeller.get(seller);
-          addResult(TimestampedValue.of(
-              new SellerPrice(seller, Math.round((double) total / count)), lastTimestamp));
-        }
-        allDone();
-        return;
-      }
-
-      lastTimestamp = timestampedWinningBid.getTimestamp();
-      captureWinningBid(timestampedWinningBid.getValue().auction,
-          timestampedWinningBid.getValue().bid, lastTimestamp);
-    }
-  }
-
-  public Query6Model(NexmarkConfiguration configuration) {
-    super(configuration);
-  }
-
-  @Override
-  public AbstractSimulator<?, ?> simulator() {
-    return new Simulator(configuration);
-  }
-
-  @Override
-  protected Iterable<TimestampedValue<KnownSize>> relevantResults(
-      Iterable<TimestampedValue<KnownSize>> results) {
-    // Find the last (in processing time) reported average price for each seller.
-    Map<Long, TimestampedValue<KnownSize>> finalAverages = new TreeMap<>();
-    for (TimestampedValue<KnownSize> obj : results) {
-      Assert.assertTrue("have SellerPrice", obj.getValue() instanceof SellerPrice);
-      SellerPrice sellerPrice = (SellerPrice) obj.getValue();
-      finalAverages.put(
-          sellerPrice.seller, TimestampedValue.of((KnownSize) sellerPrice, obj.getTimestamp()));
-    }
-    return finalAverages.values();
-  }
-
-  @Override
-  protected <T> Collection<String> toCollection(Iterator<TimestampedValue<T>> itr) {
-    return toValue(itr);
-  }
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/f4333df7/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query7.java
----------------------------------------------------------------------
diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query7.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query7.java
deleted file mode 100644
index 217d0d4..0000000
--- a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query7.java
+++ /dev/null
@@ -1,90 +0,0 @@
-/*
- * 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.beam.integration.nexmark.queries;
-
-import org.apache.beam.integration.nexmark.NexmarkConfiguration;
-import org.apache.beam.integration.nexmark.NexmarkUtils;
-import org.apache.beam.integration.nexmark.model.Bid;
-import org.apache.beam.integration.nexmark.model.Event;
-import org.apache.beam.integration.nexmark.model.KnownSize;
-import org.apache.beam.sdk.transforms.DoFn;
-import org.apache.beam.sdk.transforms.Max;
-import org.apache.beam.sdk.transforms.ParDo;
-import org.apache.beam.sdk.transforms.windowing.FixedWindows;
-import org.apache.beam.sdk.transforms.windowing.Window;
-import org.apache.beam.sdk.values.PCollection;
-import org.apache.beam.sdk.values.PCollectionView;
-
-import org.joda.time.Duration;
-
-/**
- * Query 7, 'Highest Bid'. Select the bids with the highest bid
- * price in the last minute. In CQL syntax:
- *
- * <pre>
- * SELECT Rstream(B.auction, B.price, B.bidder)
- * FROM Bid [RANGE 1 MINUTE SLIDE 1 MINUTE] B
- * WHERE B.price = (SELECT MAX(B1.price)
- *                  FROM BID [RANGE 1 MINUTE SLIDE 1 MINUTE] B1);
- * </pre>
- *
- * <p>We will use a shorter window to help make testing easier. We'll also implement this using
- * a side-input in order to exercise that functionality. (A combiner, as used in Query 5, is
- * a more efficient approach.).
- */
-public class Query7 extends NexmarkQuery {
-  public Query7(NexmarkConfiguration configuration) {
-    super(configuration, "Query7");
-  }
-
-  private PCollection<Bid> applyTyped(PCollection<Event> events) {
-    // Window the bids.
-    PCollection<Bid> slidingBids = events.apply(JUST_BIDS).apply(
-        Window.<Bid>into(FixedWindows.of(Duration.standardSeconds(configuration.windowSizeSec))));
-
-    // Find the largest price in all bids.
-    // NOTE: It would be more efficient to write this query much as we did for Query5, using
-    // a binary combiner to accumulate the bids with maximal price. As written this query
-    // requires an additional scan per window, with the associated cost of snapshotted state and
-    // its I/O. We'll keep this implementation since it illustrates the use of side inputs.
-    final PCollectionView<Long> maxPriceView =
-        slidingBids
-            .apply("BidToPrice", BID_TO_PRICE)
-            .apply(Max.longsGlobally().withFanout(configuration.fanout).asSingletonView());
-
-    return slidingBids
-        // Select all bids which have that maximum price (there may be more than one).
-        .apply(name + ".Select", ParDo
-          .of(new DoFn<Bid, Bid>() {
-                @ProcessElement
-                public void processElement(ProcessContext c) {
-                  long maxPrice = c.sideInput(maxPriceView);
-                  Bid bid = c.element();
-                  if (bid.price == maxPrice) {
-                    c.output(bid);
-                  }
-                }
-              })
-          .withSideInputs(maxPriceView));
-  }
-
-  @Override
-  protected PCollection<KnownSize> applyPrim(PCollection<Event> events) {
-    return NexmarkUtils.castToKnownSize(name, applyTyped(events));
-  }
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/f4333df7/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query7Model.java
----------------------------------------------------------------------
diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query7Model.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query7Model.java
deleted file mode 100644
index 0ada5e8..0000000
--- a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query7Model.java
+++ /dev/null
@@ -1,130 +0,0 @@
-/*
- * 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.beam.integration.nexmark.queries;
-
-import java.io.Serializable;
-import java.util.ArrayList;
-import java.util.Collection;
-import java.util.Iterator;
-import java.util.List;
-
-import org.apache.beam.integration.nexmark.NexmarkConfiguration;
-import org.apache.beam.integration.nexmark.NexmarkUtils;
-import org.apache.beam.integration.nexmark.model.Bid;
-import org.apache.beam.integration.nexmark.model.Event;
-import org.apache.beam.sdk.transforms.windowing.BoundedWindow;
-import org.apache.beam.sdk.values.TimestampedValue;
-import org.joda.time.Duration;
-import org.joda.time.Instant;
-
-/**
- * A direct implementation of {@link Query7}.
- */
-public class Query7Model extends NexmarkQueryModel implements Serializable {
-  /**
-   * Simulator for query 7.
-   */
-  private class Simulator extends AbstractSimulator<Event, Bid> {
-    /** Bids with highest bid price seen in the current window. */
-    private final List<Bid> highestBids;
-
-    /** When current window started. */
-    private Instant windowStart;
-
-    private Instant lastTimestamp;
-
-    public Simulator(NexmarkConfiguration configuration) {
-      super(NexmarkUtils.standardEventIterator(configuration));
-      highestBids = new ArrayList<>();
-      windowStart = NexmarkUtils.BEGINNING_OF_TIME;
-      lastTimestamp = BoundedWindow.TIMESTAMP_MIN_VALUE;
-    }
-
-    /**
-     * Transfer the currently winning bids into results and retire them.
-     */
-    private void retireWindow(Instant timestamp) {
-      for (Bid bid : highestBids) {
-        addResult(TimestampedValue.of(bid, timestamp));
-      }
-      highestBids.clear();
-    }
-
-    /**
-     * Keep just the highest price bid.
-     */
-    private void captureBid(Bid bid) {
-      Iterator<Bid> itr = highestBids.iterator();
-      boolean isWinning = true;
-      while (itr.hasNext()) {
-        Bid existingBid = itr.next();
-        if (existingBid.price > bid.price) {
-          isWinning = false;
-          break;
-        }
-        NexmarkUtils.info("smaller price: %s", existingBid);
-        itr.remove();
-      }
-      if (isWinning) {
-        NexmarkUtils.info("larger price: %s", bid);
-        highestBids.add(bid);
-      }
-    }
-
-    @Override
-    protected void run() {
-      TimestampedValue<Event> timestampedEvent = nextInput();
-      if (timestampedEvent == null) {
-        // Capture all remaining bids in results.
-        retireWindow(lastTimestamp);
-        allDone();
-        return;
-      }
-
-      Event event = timestampedEvent.getValue();
-      if (event.bid == null) {
-        // Ignore non-bid events.
-        return;
-      }
-      lastTimestamp = timestampedEvent.getTimestamp();
-      Instant newWindowStart = windowStart(Duration.standardSeconds(configuration.windowSizeSec),
-          Duration.standardSeconds(configuration.windowSizeSec), lastTimestamp);
-      if (!newWindowStart.equals(windowStart)) {
-        // Capture highest priced bids in current window and retire it.
-        retireWindow(lastTimestamp);
-        windowStart = newWindowStart;
-      }
-      // Keep only the highest bids.
-      captureBid(event.bid);
-    }
-  }
-
-  public Query7Model(NexmarkConfiguration configuration) {
-    super(configuration);
-  }
-
-  @Override
-  public AbstractSimulator<?, ?> simulator() {
-    return new Simulator(configuration);
-  }
-
-  @Override
-  protected <T> Collection<String> toCollection(Iterator<TimestampedValue<T>> itr) {
-    return toValueOrder(itr);
-  }
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/f4333df7/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query8.java
----------------------------------------------------------------------
diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query8.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query8.java
deleted file mode 100644
index 603841b..0000000
--- a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query8.java
+++ /dev/null
@@ -1,97 +0,0 @@
-/*
- * 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.beam.integration.nexmark.queries;
-
-import org.apache.beam.integration.nexmark.NexmarkConfiguration;
-import org.apache.beam.integration.nexmark.NexmarkUtils;
-import org.apache.beam.integration.nexmark.model.Auction;
-import org.apache.beam.integration.nexmark.model.Event;
-import org.apache.beam.integration.nexmark.model.IdNameReserve;
-import org.apache.beam.integration.nexmark.model.KnownSize;
-import org.apache.beam.integration.nexmark.model.Person;
-import org.apache.beam.sdk.transforms.DoFn;
-import org.apache.beam.sdk.transforms.ParDo;
-import org.apache.beam.sdk.transforms.join.CoGbkResult;
-import org.apache.beam.sdk.transforms.join.CoGroupByKey;
-import org.apache.beam.sdk.transforms.join.KeyedPCollectionTuple;
-import org.apache.beam.sdk.transforms.windowing.FixedWindows;
-import org.apache.beam.sdk.transforms.windowing.Window;
-import org.apache.beam.sdk.values.KV;
-import org.apache.beam.sdk.values.PCollection;
-import org.joda.time.Duration;
-
-/**
- * Query 8, 'Monitor New Users'. Select people who have entered the system and created auctions
- * in the last 12 hours, updated every 12 hours. In CQL syntax:
- *
- * <pre>
- * SELECT Rstream(P.id, P.name, A.reserve)
- * FROM Person [RANGE 12 HOUR] P, Auction [RANGE 12 HOUR] A
- * WHERE P.id = A.seller;
- * </pre>
- *
- * <p>To make things a bit more dynamic and easier to test we'll use a much shorter window.
- */
-public class Query8 extends NexmarkQuery {
-  public Query8(NexmarkConfiguration configuration) {
-    super(configuration, "Query8");
-  }
-
-  private PCollection<IdNameReserve> applyTyped(PCollection<Event> events) {
-    // Window and key new people by their id.
-    PCollection<KV<Long, Person>> personsById =
-        events
-          .apply(JUST_NEW_PERSONS)
-          .apply("Query8.WindowPersons",
-            Window.<Person>into(
-              FixedWindows.of(Duration.standardSeconds(configuration.windowSizeSec))))
-            .apply("PersonById", PERSON_BY_ID);
-
-    // Window and key new auctions by their id.
-    PCollection<KV<Long, Auction>> auctionsBySeller =
-        events.apply(JUST_NEW_AUCTIONS)
-          .apply("Query8.WindowAuctions",
-            Window.<Auction>into(
-              FixedWindows.of(Duration.standardSeconds(configuration.windowSizeSec))))
-            .apply("AuctionBySeller", AUCTION_BY_SELLER);
-
-    // Join people and auctions and project the person id, name and auction reserve price.
-    return KeyedPCollectionTuple.of(PERSON_TAG, personsById)
-        .and(AUCTION_TAG, auctionsBySeller)
-        .apply(CoGroupByKey.<Long>create())
-        .apply(name + ".Select",
-            ParDo.of(new DoFn<KV<Long, CoGbkResult>, IdNameReserve>() {
-                  @ProcessElement
-                  public void processElement(ProcessContext c) {
-                    Person person = c.element().getValue().getOnly(PERSON_TAG, null);
-                    if (person == null) {
-                      // Person was not created in last window period.
-                      return;
-                    }
-                    for (Auction auction : c.element().getValue().getAll(AUCTION_TAG)) {
-                      c.output(new IdNameReserve(person.id, person.name, auction.reserve));
-                    }
-                  }
-                }));
-  }
-
-  @Override
-  protected PCollection<KnownSize> applyPrim(PCollection<Event> events) {
-    return NexmarkUtils.castToKnownSize(name, applyTyped(events));
-  }
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/f4333df7/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query8Model.java
----------------------------------------------------------------------
diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query8Model.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query8Model.java
deleted file mode 100644
index 8c76bc6..0000000
--- a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query8Model.java
+++ /dev/null
@@ -1,148 +0,0 @@
-/*
- * 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.beam.integration.nexmark.queries;
-
-import com.google.common.collect.ArrayListMultimap;
-import com.google.common.collect.Multimap;
-import java.io.Serializable;
-import java.util.Collection;
-import java.util.HashMap;
-import java.util.Iterator;
-import java.util.Map;
-import org.apache.beam.integration.nexmark.NexmarkConfiguration;
-import org.apache.beam.integration.nexmark.NexmarkUtils;
-import org.apache.beam.integration.nexmark.model.Auction;
-import org.apache.beam.integration.nexmark.model.Event;
-import org.apache.beam.integration.nexmark.model.IdNameReserve;
-import org.apache.beam.integration.nexmark.model.Person;
-import org.apache.beam.sdk.values.TimestampedValue;
-import org.joda.time.Duration;
-import org.joda.time.Instant;
-
-/**
- * A direct implementation of {@link Query8}.
- */
-public class Query8Model extends NexmarkQueryModel implements Serializable {
-  /**
-   * Simulator for query 8.
-   */
-  private class Simulator extends AbstractSimulator<Event, IdNameReserve> {
-    /** New persons seen in the current window, indexed by id. */
-    private final Map<Long, Person> newPersons;
-
-    /** New auctions seen in the current window, indexed by seller id. */
-    private final Multimap<Long, Auction> newAuctions;
-
-    /** When did the current window start. */
-    private Instant windowStart;
-
-    public Simulator(NexmarkConfiguration configuration) {
-      super(NexmarkUtils.standardEventIterator(configuration));
-      newPersons = new HashMap<>();
-      newAuctions = ArrayListMultimap.create();
-      windowStart = NexmarkUtils.BEGINNING_OF_TIME;
-    }
-
-    /**
-     * Retire all persons added in last window.
-     */
-    private void retirePersons() {
-      for (Map.Entry<Long, Person> entry : newPersons.entrySet()) {
-        NexmarkUtils.info("retire: %s", entry.getValue());
-      }
-      newPersons.clear();
-    }
-
-    /**
-     * Retire all auctions added in last window.
-     */
-    private void retireAuctions() {
-      for (Map.Entry<Long, Auction> entry : newAuctions.entries()) {
-        NexmarkUtils.info("retire: %s", entry.getValue());
-      }
-      newAuctions.clear();
-    }
-
-    /**
-     * Capture new result.
-     */
-    private void addResult(Auction auction, Person person, Instant timestamp) {
-      addResult(TimestampedValue.of(
-          new IdNameReserve(person.id, person.name, auction.reserve), timestamp));
-    }
-
-    @Override
-    public void run() {
-      TimestampedValue<Event> timestampedEvent = nextInput();
-      if (timestampedEvent == null) {
-        allDone();
-        return;
-      }
-
-      Event event = timestampedEvent.getValue();
-      if (event.bid != null) {
-        // Ignore bid events.
-        // Keep looking for next events.
-        return;
-      }
-      Instant timestamp = timestampedEvent.getTimestamp();
-      Instant newWindowStart = windowStart(Duration.standardSeconds(configuration.windowSizeSec),
-          Duration.standardSeconds(configuration.windowSizeSec), timestamp);
-      if (!newWindowStart.equals(windowStart)) {
-        // Retire this window.
-        retirePersons();
-        retireAuctions();
-        windowStart = newWindowStart;
-      }
-
-      if (event.newAuction != null) {
-        // Join new auction with existing person, if any.
-        Person person = newPersons.get(event.newAuction.seller);
-        if (person != null) {
-          addResult(event.newAuction, person, timestamp);
-        } else {
-          // Remember auction for future new people.
-          newAuctions.put(event.newAuction.seller, event.newAuction);
-        }
-      } else { // event is not an auction, nor a bid, so it is a person
-        // Join new person with existing auctions.
-        for (Auction auction : newAuctions.get(event.newPerson.id)) {
-          addResult(auction, event.newPerson, timestamp);
-        }
-        // We'll never need these auctions again.
-        newAuctions.removeAll(event.newPerson.id);
-        // Remember person for future auctions.
-        newPersons.put(event.newPerson.id, event.newPerson);
-      }
-    }
-  }
-
-  public Query8Model(NexmarkConfiguration configuration) {
-    super(configuration);
-  }
-
-  @Override
-  public AbstractSimulator<?, ?> simulator() {
-    return new Simulator(configuration);
-  }
-
-  @Override
-  protected <T> Collection<String> toCollection(Iterator<TimestampedValue<T>> itr) {
-    return toValue(itr);
-  }
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/f4333df7/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query9.java
----------------------------------------------------------------------
diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query9.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query9.java
deleted file mode 100644
index 6dd189d..0000000
--- a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query9.java
+++ /dev/null
@@ -1,44 +0,0 @@
-/*
- * 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.beam.integration.nexmark.queries;
-
-import org.apache.beam.integration.nexmark.NexmarkConfiguration;
-import org.apache.beam.integration.nexmark.NexmarkUtils;
-import org.apache.beam.integration.nexmark.model.AuctionBid;
-import org.apache.beam.integration.nexmark.model.Event;
-import org.apache.beam.integration.nexmark.model.KnownSize;
-import org.apache.beam.sdk.values.PCollection;
-
-/**
- * Query "9", 'Winning bids'. Select just the winning bids. Not in original NEXMark suite, but
- * handy for testing. See {@link WinningBids} for the details.
- */
-public class Query9 extends NexmarkQuery {
-  public Query9(NexmarkConfiguration configuration) {
-    super(configuration, "Query9");
-  }
-
-  private PCollection<AuctionBid> applyTyped(PCollection<Event> events) {
-    return events.apply(new WinningBids(name, configuration));
-  }
-
-  @Override
-  protected PCollection<KnownSize> applyPrim(PCollection<Event> events) {
-    return NexmarkUtils.castToKnownSize(name, applyTyped(events));
-  }
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/f4333df7/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query9Model.java
----------------------------------------------------------------------
diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query9Model.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query9Model.java
deleted file mode 100644
index d117e2d..0000000
--- a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query9Model.java
+++ /dev/null
@@ -1,44 +0,0 @@
-/*
- * 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.beam.integration.nexmark.queries;
-
-import java.io.Serializable;
-import java.util.Collection;
-import java.util.Iterator;
-
-import org.apache.beam.integration.nexmark.NexmarkConfiguration;
-import org.apache.beam.sdk.values.TimestampedValue;
-
-/**
- * A direct implementation of {@link Query9}.
- */
-public class Query9Model extends NexmarkQueryModel implements Serializable {
-  public Query9Model(NexmarkConfiguration configuration) {
-    super(configuration);
-  }
-
-  @Override
-  public AbstractSimulator<?, ?> simulator() {
-    return new WinningBidsSimulator(configuration);
-  }
-
-  @Override
-  protected <T> Collection<String> toCollection(Iterator<TimestampedValue<T>> itr) {
-    return toValue(itr);
-  }
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/f4333df7/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/WinningBids.java
----------------------------------------------------------------------
diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/WinningBids.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/WinningBids.java
deleted file mode 100644
index d4ca177..0000000
--- a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/WinningBids.java
+++ /dev/null
@@ -1,412 +0,0 @@
-/*
- * 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.beam.integration.nexmark.queries;
-
-import static com.google.common.base.Preconditions.checkState;
-
-import com.fasterxml.jackson.annotation.JsonCreator;
-import java.io.IOException;
-import java.io.InputStream;
-import java.io.OutputStream;
-import java.util.ArrayList;
-import java.util.Collection;
-import java.util.Collections;
-import java.util.List;
-import java.util.Map;
-import java.util.Objects;
-import java.util.TreeMap;
-import org.apache.beam.integration.nexmark.NexmarkConfiguration;
-import org.apache.beam.integration.nexmark.NexmarkUtils;
-import org.apache.beam.integration.nexmark.model.Auction;
-import org.apache.beam.integration.nexmark.model.AuctionBid;
-import org.apache.beam.integration.nexmark.model.Bid;
-import org.apache.beam.integration.nexmark.model.Event;
-import org.apache.beam.integration.nexmark.sources.GeneratorConfig;
-import org.apache.beam.sdk.coders.Coder;
-import org.apache.beam.sdk.coders.CoderException;
-import org.apache.beam.sdk.coders.CustomCoder;
-import org.apache.beam.sdk.coders.VarIntCoder;
-import org.apache.beam.sdk.coders.VarLongCoder;
-import org.apache.beam.sdk.metrics.Counter;
-import org.apache.beam.sdk.metrics.Metrics;
-import org.apache.beam.sdk.transforms.DoFn;
-import org.apache.beam.sdk.transforms.PTransform;
-import org.apache.beam.sdk.transforms.ParDo;
-import org.apache.beam.sdk.transforms.join.CoGbkResult;
-import org.apache.beam.sdk.transforms.join.CoGroupByKey;
-import org.apache.beam.sdk.transforms.join.KeyedPCollectionTuple;
-import org.apache.beam.sdk.transforms.windowing.IntervalWindow;
-import org.apache.beam.sdk.transforms.windowing.Window;
-import org.apache.beam.sdk.transforms.windowing.WindowFn;
-import org.apache.beam.sdk.transforms.windowing.WindowMappingFn;
-import org.apache.beam.sdk.values.KV;
-import org.apache.beam.sdk.values.PCollection;
-import org.joda.time.Instant;
-
-/**
- * A transform to find the winning bid for each closed auction. In pseudo CQL syntax:
- *
- * <pre>{@code
- * SELECT Rstream(A.*, B.auction, B.bidder, MAX(B.price), B.dateTime)
- * FROM Auction A [ROWS UNBOUNDED], Bid B [ROWS UNBOUNDED]
- * WHERE A.id = B.auction AND B.datetime < A.expires AND A.expires < CURRENT_TIME
- * GROUP BY A.id
- * }</pre>
- *
- * <p>We will also check that the winning bid is above the auction reserve. Note that
- * we ignore the auction opening bid value since it has no impact on which bid eventually wins,
- * if any.
- *
- * <p>Our implementation will use a custom windowing function in order to bring bids and
- * auctions together without requiring global state.
- */
-public class WinningBids extends PTransform<PCollection<Event>, PCollection<AuctionBid>> {
-  /** Windows for open auctions and bids. */
-  private static class AuctionOrBidWindow extends IntervalWindow {
-    /** Id of auction this window is for. */
-    public final long auction;
-
-    /**
-     * True if this window represents an actual auction, and thus has a start/end
-     * time matching that of the auction. False if this window represents a bid, and
-     * thus has an unbounded start/end time.
-     */
-    public final boolean isAuctionWindow;
-
-    /** For avro only. */
-    private AuctionOrBidWindow() {
-      super(TIMESTAMP_MIN_VALUE, TIMESTAMP_MAX_VALUE);
-      auction = 0;
-      isAuctionWindow = false;
-    }
-
-    private AuctionOrBidWindow(
-        Instant start, Instant end, long auctionId, boolean isAuctionWindow) {
-      super(start, end);
-      this.auction = auctionId;
-      this.isAuctionWindow = isAuctionWindow;
-    }
-
-    /** Return an auction window for {@code auction}. */
-    public static AuctionOrBidWindow forAuction(Instant timestamp, Auction auction) {
-      return new AuctionOrBidWindow(timestamp, new Instant(auction.expires), auction.id, true);
-    }
-
-    /**
-     * Return a bid window for {@code bid}. It should later be merged into
-     * the corresponding auction window. However, it is possible this bid is for an already
-     * expired auction, or for an auction which the system has not yet seen. So we
-     * give the bid a bit of wiggle room in its interval.
-     */
-    public static AuctionOrBidWindow forBid(
-        long expectedAuctionDurationMs, Instant timestamp, Bid bid) {
-      // At this point we don't know which auctions are still valid, and the bid may
-      // be for an auction which won't start until some unknown time in the future
-      // (due to Generator.AUCTION_ID_LEAD in Generator.nextBid).
-      // A real system would atomically reconcile bids and auctions by a separate mechanism.
-      // If we give bids an unbounded window it is possible a bid for an auction which
-      // has already expired would cause the system watermark to stall, since that window
-      // would never be retired.
-      // Instead, we will just give the bid a finite window which expires at
-      // the upper bound of auctions assuming the auction starts at the same time as the bid,
-      // and assuming the system is running at its lowest event rate (as per interEventDelayUs).
-      return new AuctionOrBidWindow(
-          timestamp, timestamp.plus(expectedAuctionDurationMs * 2), bid.auction, false);
-    }
-
-    /** Is this an auction window? */
-    public boolean isAuctionWindow() {
-      return isAuctionWindow;
-    }
-
-    @Override
-    public String toString() {
-      return String.format("AuctionOrBidWindow{start:%s; end:%s; auction:%d; isAuctionWindow:%s}",
-          start(), end(), auction, isAuctionWindow);
-    }
-
-    @Override public boolean equals(Object o) {
-      if (this == o) {
-        return true;
-      }
-      if (o == null || getClass() != o.getClass()) {
-        return false;
-      }
-      if (!super.equals(o)) {
-        return false;
-      }
-      AuctionOrBidWindow that = (AuctionOrBidWindow) o;
-      return (isAuctionWindow == that.isAuctionWindow) && (auction == that.auction);
-    }
-
-    @Override public int hashCode() {
-      return Objects.hash(isAuctionWindow, auction);
-    }
-  }
-
-  /**
-   * Encodes an {@link AuctionOrBidWindow} as an {@link IntervalWindow} and an auction id long.
-   */
-  private static class AuctionOrBidWindowCoder extends CustomCoder<AuctionOrBidWindow> {
-    private static final AuctionOrBidWindowCoder INSTANCE = new AuctionOrBidWindowCoder();
-    private static final Coder<IntervalWindow> SUPER_CODER = IntervalWindow.getCoder();
-    private static final Coder<Long> ID_CODER = VarLongCoder.of();
-    private static final Coder<Integer> INT_CODER = VarIntCoder.of();
-
-    @JsonCreator
-    public static AuctionOrBidWindowCoder of() {
-      return INSTANCE;
-    }
-
-    @Override
-    public void encode(AuctionOrBidWindow window, OutputStream outStream)
-        throws IOException, CoderException {
-      SUPER_CODER.encode(window, outStream);
-      ID_CODER.encode(window.auction, outStream);
-      INT_CODER.encode(window.isAuctionWindow ? 1 : 0, outStream);
-    }
-
-    @Override
-    public AuctionOrBidWindow decode(InputStream inStream)
-        throws IOException, CoderException {
-      IntervalWindow superWindow = SUPER_CODER.decode(inStream);
-      long auction = ID_CODER.decode(inStream);
-      boolean isAuctionWindow = INT_CODER.decode(inStream) != 0;
-      return new AuctionOrBidWindow(
-          superWindow.start(), superWindow.end(), auction, isAuctionWindow);
-    }
-
-    @Override public void verifyDeterministic() throws NonDeterministicException {}
-  }
-
-  /** Assign events to auction windows and merges them intelligently. */
-  private static class AuctionOrBidWindowFn extends WindowFn<Event, AuctionOrBidWindow> {
-    /** Expected duration of auctions in ms. */
-    private final long expectedAuctionDurationMs;
-
-    public AuctionOrBidWindowFn(long expectedAuctionDurationMs) {
-      this.expectedAuctionDurationMs = expectedAuctionDurationMs;
-    }
-
-    @Override
-    public Collection<AuctionOrBidWindow> assignWindows(AssignContext c) {
-      Event event = c.element();
-      if (event.newAuction != null) {
-        // Assign auctions to an auction window which expires at the auction's close.
-        return Collections
-            .singletonList(AuctionOrBidWindow.forAuction(c.timestamp(), event.newAuction));
-      } else if (event.bid != null) {
-        // Assign bids to a temporary bid window which will later be merged into the appropriate
-        // auction window.
-        return Collections.singletonList(
-            AuctionOrBidWindow.forBid(expectedAuctionDurationMs, c.timestamp(), event.bid));
-      } else {
-        // Don't assign people to any window. They will thus be dropped.
-        return Collections.emptyList();
-      }
-    }
-
-    @Override
-    public void mergeWindows(MergeContext c) throws Exception {
-      // Split and index the auction and bid windows by auction id.
-      Map<Long, AuctionOrBidWindow> idToTrueAuctionWindow = new TreeMap<>();
-      Map<Long, List<AuctionOrBidWindow>> idToBidAuctionWindows = new TreeMap<>();
-      for (AuctionOrBidWindow window : c.windows()) {
-        if (window.isAuctionWindow()) {
-          idToTrueAuctionWindow.put(window.auction, window);
-        } else {
-          List<AuctionOrBidWindow> bidWindows = idToBidAuctionWindows.get(window.auction);
-          if (bidWindows == null) {
-            bidWindows = new ArrayList<>();
-            idToBidAuctionWindows.put(window.auction, bidWindows);
-          }
-          bidWindows.add(window);
-        }
-      }
-
-      // Merge all 'bid' windows into their corresponding 'auction' window, provided the
-      // auction has not expired.
-      for (Map.Entry<Long, AuctionOrBidWindow> entry : idToTrueAuctionWindow.entrySet()) {
-        long auction = entry.getKey();
-        AuctionOrBidWindow auctionWindow = entry.getValue();
-        List<AuctionOrBidWindow> bidWindows = idToBidAuctionWindows.get(auction);
-        if (bidWindows != null) {
-          List<AuctionOrBidWindow> toBeMerged = new ArrayList<>();
-          for (AuctionOrBidWindow bidWindow : bidWindows) {
-            if (bidWindow.start().isBefore(auctionWindow.end())) {
-              toBeMerged.add(bidWindow);
-            }
-            // else: This bid window will remain until its expire time, at which point it
-            // will expire without ever contributing to an output.
-          }
-          if (!toBeMerged.isEmpty()) {
-            toBeMerged.add(auctionWindow);
-            c.merge(toBeMerged, auctionWindow);
-          }
-        }
-      }
-    }
-
-    @Override
-    public boolean isCompatible(WindowFn<?, ?> other) {
-      return other instanceof AuctionOrBidWindowFn;
-    }
-
-    @Override
-    public Coder<AuctionOrBidWindow> windowCoder() {
-      return AuctionOrBidWindowCoder.of();
-    }
-
-    @Override
-    public WindowMappingFn<AuctionOrBidWindow> getDefaultWindowMappingFn() {
-      throw new UnsupportedOperationException("AuctionWindowFn not supported for side inputs");
-    }
-
-    /**
-     * Below we will GBK auctions and bids on their auction ids. Then we will reduce those
-     * per id to emit {@code (auction, winning bid)} pairs for auctions which have expired with at
-     * least one valid bid. We would like those output pairs to have a timestamp of the auction's
-     * expiry (since that's the earliest we know for sure we have the correct winner). We would
-     * also like to make that winning results are available to following stages at the auction's
-     * expiry.
-     *
-     * <p>Each result of the GBK will have a timestamp of the min of the result of this object's
-     * assignOutputTime over all records which end up in one of its iterables. Thus we get the
-     * desired behavior if we ignore each record's timestamp and always return the auction window's
-     * 'maxTimestamp', which will correspond to the auction's expiry.
-     *
-     * <p>In contrast, if this object's assignOutputTime were to return 'inputTimestamp'
-     * (the usual implementation), then each GBK record will take as its timestamp the minimum of
-     * the timestamps of all bids and auctions within it, which will always be the auction's
-     * timestamp. An auction which expires well into the future would thus hold up the watermark
-     * of the GBK results until that auction expired. That in turn would hold up all winning pairs.
-     */
-    @Override
-    public Instant getOutputTime(
-        Instant inputTimestamp, AuctionOrBidWindow window) {
-      return window.maxTimestamp();
-    }
-  }
-
-  private final AuctionOrBidWindowFn auctionOrBidWindowFn;
-
-  public WinningBids(String name, NexmarkConfiguration configuration) {
-    super(name);
-    // What's the expected auction time (when the system is running at the lowest event rate).
-    long[] interEventDelayUs = configuration.rateShape.interEventDelayUs(
-        configuration.firstEventRate, configuration.nextEventRate,
-        configuration.rateUnit, configuration.numEventGenerators);
-    long longestDelayUs = 0;
-    for (long interEventDelayU : interEventDelayUs) {
-      longestDelayUs = Math.max(longestDelayUs, interEventDelayU);
-    }
-    // Adjust for proportion of auction events amongst all events.
-    longestDelayUs =
-        (longestDelayUs * GeneratorConfig.PROPORTION_DENOMINATOR)
-        / GeneratorConfig.AUCTION_PROPORTION;
-    // Adjust for number of in-flight auctions.
-    longestDelayUs = longestDelayUs * configuration.numInFlightAuctions;
-    long expectedAuctionDurationMs = (longestDelayUs + 999) / 1000;
-    NexmarkUtils.console("Expected auction duration is %d ms", expectedAuctionDurationMs);
-    auctionOrBidWindowFn = new AuctionOrBidWindowFn(expectedAuctionDurationMs);
-  }
-
-  @Override
-  public PCollection<AuctionBid> expand(PCollection<Event> events) {
-    // Window auctions and bids into custom auction windows. New people events will be discarded.
-    // This will allow us to bring bids and auctions together irrespective of how long
-    // each auction is open for.
-    events = events.apply("Window", Window.into(auctionOrBidWindowFn));
-
-    // Key auctions by their id.
-    PCollection<KV<Long, Auction>> auctionsById =
-        events.apply(NexmarkQuery.JUST_NEW_AUCTIONS)
-              .apply("AuctionById:", NexmarkQuery.AUCTION_BY_ID);
-
-    // Key bids by their auction id.
-    PCollection<KV<Long, Bid>> bidsByAuctionId =
-        events.apply(NexmarkQuery.JUST_BIDS).apply("BidByAuction", NexmarkQuery.BID_BY_AUCTION);
-
-    // Find the highest price valid bid for each closed auction.
-    return
-      // Join auctions and bids.
-      KeyedPCollectionTuple.of(NexmarkQuery.AUCTION_TAG, auctionsById)
-        .and(NexmarkQuery.BID_TAG, bidsByAuctionId)
-        .apply(CoGroupByKey.<Long>create())
-        // Filter and select.
-        .apply(name + ".Join",
-          ParDo.of(new DoFn<KV<Long, CoGbkResult>, AuctionBid>() {
-            private final Counter noAuctionCounter = Metrics.counter(name, "noAuction");
-            private final Counter underReserveCounter = Metrics.counter(name, "underReserve");
-            private final Counter noValidBidsCounter = Metrics.counter(name, "noValidBids");
-
-            @ProcessElement
-            public void processElement(ProcessContext c) {
-              Auction auction =
-                  c.element().getValue().getOnly(NexmarkQuery.AUCTION_TAG, null);
-              if (auction == null) {
-                // We have bids without a matching auction. Give up.
-                noAuctionCounter.inc();
-                return;
-              }
-              // Find the current winning bid for auction.
-              // The earliest bid with the maximum price above the reserve wins.
-              Bid bestBid = null;
-              for (Bid bid : c.element().getValue().getAll(NexmarkQuery.BID_TAG)) {
-                // Bids too late for their auction will have been
-                // filtered out by the window merge function.
-                checkState(bid.dateTime < auction.expires);
-                if (bid.price < auction.reserve) {
-                  // Bid price is below auction reserve.
-                  underReserveCounter.inc();
-                  continue;
-                }
-
-                if (bestBid == null
-                    || Bid.PRICE_THEN_DESCENDING_TIME.compare(bid, bestBid) > 0) {
-                  bestBid = bid;
-                }
-              }
-              if (bestBid == null) {
-                // We don't have any valid bids for auction.
-                noValidBidsCounter.inc();
-                return;
-              }
-              c.output(new AuctionBid(auction, bestBid));
-            }
-          }
-        ));
-  }
-
-  @Override
-  public int hashCode() {
-    return Objects.hash(auctionOrBidWindowFn);
-  }
-
-  @Override public boolean equals(Object o) {
-    if (this == o) {
-      return true;
-    }
-    if (o == null || getClass() != o.getClass()) {
-      return false;
-    }
-
-    WinningBids that = (WinningBids) o;
-    return auctionOrBidWindowFn.equals(that.auctionOrBidWindowFn);
-  }
-}


[53/55] [abbrv] beam git commit: Clean, fix findbugs, fix checkstyle

Posted by ie...@apache.org.
Clean, fix findbugs, fix checkstyle


Project: http://git-wip-us.apache.org/repos/asf/beam/repo
Commit: http://git-wip-us.apache.org/repos/asf/beam/commit/2f9b4948
Tree: http://git-wip-us.apache.org/repos/asf/beam/tree/2f9b4948
Diff: http://git-wip-us.apache.org/repos/asf/beam/diff/2f9b4948

Branch: refs/heads/master
Commit: 2f9b4948fd60a749ada832d003acf0bd84875fcb
Parents: 6c11670
Author: Etienne Chauchot <ec...@gmail.com>
Authored: Tue May 30 18:00:00 2017 +0100
Committer: Ismaël Mejía <ie...@gmail.com>
Committed: Wed Aug 23 19:07:29 2017 +0200

----------------------------------------------------------------------
 .../nexmark/NexmarkConfiguration.java           |  9 +-
 .../integration/nexmark/NexmarkLauncher.java    | 62 +++++-------
 .../integration/nexmark/NexmarkOptions.java     |  3 +-
 .../beam/integration/nexmark/NexmarkSuite.java  |  4 +-
 .../beam/integration/nexmark/model/Event.java   | 99 ++++++++++----------
 .../nexmark/queries/Query0Model.java            |  1 -
 .../nexmark/queries/Query1Model.java            |  1 -
 .../integration/nexmark/queries/Query3.java     |  8 +-
 .../integration/nexmark/queries/Query5.java     | 68 ++++++++------
 .../integration/nexmark/queries/Query7.java     |  2 +-
 .../nexmark/queries/Query7Model.java            |  1 -
 .../nexmark/queries/WinningBids.java            | 37 +++++++-
 .../nexmark/queries/WinningBidsSimulator.java   |  1 -
 .../integration/nexmark/sources/Generator.java  | 36 +++++--
 .../nexmark/sources/GeneratorConfig.java        | 29 +++---
 .../integration/nexmark/queries/QueryTest.java  |  6 +-
 .../sources/UnboundedEventSourceTest.java       |  6 +-
 17 files changed, 211 insertions(+), 162 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/beam/blob/2f9b4948/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkConfiguration.java
----------------------------------------------------------------------
diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkConfiguration.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkConfiguration.java
index 5a8cb71..2faf3f5 100644
--- a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkConfiguration.java
+++ b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkConfiguration.java
@@ -19,7 +19,6 @@ package org.apache.beam.integration.nexmark;
 
 import com.fasterxml.jackson.annotation.JsonProperty;
 import com.fasterxml.jackson.core.JsonProcessingException;
-
 import java.io.IOException;
 import java.io.Serializable;
 import java.util.Objects;
@@ -359,11 +358,11 @@ public class NexmarkConfiguration implements Serializable {
   }
 
   /**
-   * Return clone of configuration with given label.
+   * Return copy of configuration with given label.
    */
-  @Override
-  public NexmarkConfiguration clone() {
-    NexmarkConfiguration result = new NexmarkConfiguration();
+  public NexmarkConfiguration copy() {
+    NexmarkConfiguration result;
+    result = new NexmarkConfiguration();
     result.debug = debug;
     result.query = query;
     result.sourceType = sourceType;

http://git-wip-us.apache.org/repos/asf/beam/blob/2f9b4948/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkLauncher.java
----------------------------------------------------------------------
diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkLauncher.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkLauncher.java
index db53191..a609975 100644
--- a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkLauncher.java
+++ b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkLauncher.java
@@ -87,11 +87,13 @@ import org.apache.beam.sdk.values.TimestampedValue;
 import org.apache.beam.sdk.values.TupleTag;
 import org.apache.beam.sdk.values.TupleTagList;
 import org.joda.time.Duration;
+import org.slf4j.LoggerFactory;
 
 /**
  * Run a single Nexmark query using a given configuration.
  */
 public class NexmarkLauncher<OptionT extends NexmarkOptions> {
+  private static final org.slf4j.Logger LOG = LoggerFactory.getLogger(NexmarkLauncher.class);
   /**
    * Minimum number of samples needed for 'stead-state' rate calculation.
    */
@@ -166,13 +168,6 @@ public class NexmarkLauncher<OptionT extends NexmarkOptions> {
   }
 
   /**
-   * Return number of cores per worker.
-   */
-  protected int coresPerWorker() {
-    return 4;
-  }
-
-  /**
    * Return maximum number of workers.
    */
   private int maxNumWorkers() {
@@ -185,7 +180,6 @@ public class NexmarkLauncher<OptionT extends NexmarkOptions> {
    */
   private long getCounterMetric(PipelineResult result, String namespace, String name,
     long defaultValue) {
-    //TODO Ismael calc this only once
     MetricQueryResults metrics = result.metrics().queryMetrics(
         MetricsFilter.builder().addNameFilter(MetricNameFilter.named(namespace, name)).build());
     Iterable<MetricResult<Long>> counters = metrics.counters();
@@ -193,7 +187,7 @@ public class NexmarkLauncher<OptionT extends NexmarkOptions> {
       MetricResult<Long> metricResult = counters.iterator().next();
       return metricResult.attempted();
     } catch (NoSuchElementException e) {
-      //TODO Ismael
+      LOG.error("Failed to get metric {}, from namespace {}", name, namespace);
     }
     return defaultValue;
   }
@@ -209,15 +203,20 @@ public class NexmarkLauncher<OptionT extends NexmarkOptions> {
     Iterable<MetricResult<DistributionResult>> distributions = metrics.distributions();
     try {
       MetricResult<DistributionResult> distributionResult = distributions.iterator().next();
-      if (distType.equals(DistributionType.MIN)) {
-        return distributionResult.attempted().min();
-      } else if (distType.equals(DistributionType.MAX)) {
-        return distributionResult.attempted().max();
-      } else {
-        //TODO Ismael
+      switch (distType)
+      {
+        case MIN:
+          return distributionResult.attempted().min();
+        case MAX:
+          return distributionResult.attempted().max();
+        default:
+          return defaultValue;
       }
     } catch (NoSuchElementException e) {
-      //TODO Ismael
+      LOG.error(
+          "Failed to get distribution metric {} for namespace {}",
+          name,
+          namespace);
     }
     return defaultValue;
   }
@@ -228,7 +227,9 @@ public class NexmarkLauncher<OptionT extends NexmarkOptions> {
    * Return the current value for a time counter, or -1 if can't be retrieved.
    */
   private long getTimestampMetric(long now, long value) {
-    //TODO Ismael improve doc
+    // timestamp metrics are used to monitor time of execution of transforms.
+    // If result timestamp metric is too far from now, consider that metric is erroneous
+
     if (Math.abs(value - now) > Duration.standardDays(10000).getMillis()) {
       return -1;
     }
@@ -437,16 +438,6 @@ public class NexmarkLauncher<OptionT extends NexmarkOptions> {
    */
   private void invokeBuilderForPublishOnlyPipeline(PipelineBuilder<NexmarkOptions> builder) {
     builder.build(options);
-//    throw new UnsupportedOperationException(
-//        "Cannot use --pubSubMode=COMBINED with DirectRunner");
-  }
-
-  /**
-   * If monitoring, wait until the publisher pipeline has run long enough to establish
-   * a backlog on the Pubsub topic. Otherwise, return immediately.
-   */
-  private void waitForPublisherPreload() {
-    throw new UnsupportedOperationException();
   }
 
   /**
@@ -606,11 +597,7 @@ public class NexmarkLauncher<OptionT extends NexmarkOptions> {
         publisherJob.waitUntilFinish(Duration.standardMinutes(5));
       } catch (IOException e) {
         throw new RuntimeException("Unable to cancel publisher job: ", e);
-      } //TODO Ismael
-//      catch (InterruptedException e) {
-//        Thread.interrupted();
-//        throw new RuntimeException("Interrupted: publish job still running.", e);
-//      }
+      }
     }
 
     return perf;
@@ -755,7 +742,7 @@ public class NexmarkLauncher<OptionT extends NexmarkOptions> {
             Event event = CoderUtils.decodeFromByteArray(Event.CODER, payload);
             c.output(event);
           } catch (CoderException e) {
-            // TODO Log decoding Event error
+            LOG.error("Error while decoding Event from pusbSub message: serialization error");
           }
         }
       }));
@@ -798,7 +785,8 @@ public class NexmarkLauncher<OptionT extends NexmarkOptions> {
                   byte[] payload = CoderUtils.encodeToByteArray(Event.CODER, c.element());
                   c.output(new PubsubMessage(payload, new HashMap<String, String>()));
                 } catch (CoderException e1) {
-                  // TODO Log encoding Event error
+                  LOG.error("Error while sending Event {} to pusbSub: serialization error",
+                      c.element().toString());
                 }
               }
             })
@@ -1130,7 +1118,8 @@ public class NexmarkLauncher<OptionT extends NexmarkOptions> {
           sinkEventsToAvro(source);
         }
 
-        // Special hacks for Query 10 (big logger).
+        // Query 10 logs all events to Google Cloud storage files. It could generate a lot of logs,
+        // so, set parallelism. Also set the output path where to write log files.
         if (configuration.query == 10) {
           String path = null;
           if (options.getOutputPath() != null && !options.getOutputPath().isEmpty()) {
@@ -1158,9 +1147,6 @@ public class NexmarkLauncher<OptionT extends NexmarkOptions> {
         sink(results, now);
       }
 
-      if (publisherResult != null) {
-        waitForPublisherPreload();
-      }
       mainResult = p.run();
       mainResult.waitUntilFinish(Duration.standardSeconds(configuration.streamTimeout));
       return monitor(query);

http://git-wip-us.apache.org/repos/asf/beam/blob/2f9b4948/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkOptions.java
----------------------------------------------------------------------
diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkOptions.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkOptions.java
index 9afffaa..fbd3e74 100644
--- a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkOptions.java
+++ b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkOptions.java
@@ -19,7 +19,6 @@ package org.apache.beam.integration.nexmark;
 
 import javax.annotation.Nullable;
 import org.apache.beam.sdk.extensions.gcp.options.GcpOptions;
-import org.apache.beam.sdk.io.gcp.pubsub.PubsubOptions;
 import org.apache.beam.sdk.options.ApplicationNameOptions;
 import org.apache.beam.sdk.options.Default;
 import org.apache.beam.sdk.options.Description;
@@ -119,7 +118,7 @@ public interface NexmarkOptions
   @Nullable
   Integer getStreamTimeout();
 
-  void setStreamTimeout(Integer preloadSeconds);
+  void setStreamTimeout(Integer streamTimeout);
 
   @Description("Number of unbounded sources to create events.")
   @Nullable

http://git-wip-us.apache.org/repos/asf/beam/blob/2f9b4948/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkSuite.java
----------------------------------------------------------------------
diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkSuite.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkSuite.java
index be7d7b8..0d98a5d 100644
--- a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkSuite.java
+++ b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkSuite.java
@@ -57,7 +57,7 @@ public enum NexmarkSuite {
   private static List<NexmarkConfiguration> smoke() {
     List<NexmarkConfiguration> configurations = new ArrayList<>();
     for (int query = 0; query <= 12; query++) {
-      NexmarkConfiguration configuration = NexmarkConfiguration.DEFAULT.clone();
+      NexmarkConfiguration configuration = NexmarkConfiguration.DEFAULT.copy();
       configuration.query = query;
       configuration.numEvents = 100_000;
       if (query == 4 || query == 6 || query == 9) {
@@ -103,7 +103,7 @@ public enum NexmarkSuite {
   public Iterable<NexmarkConfiguration> getConfigurations(NexmarkOptions options) {
     Set<NexmarkConfiguration> results = new LinkedHashSet<>();
     for (NexmarkConfiguration configuration : configurations) {
-      NexmarkConfiguration result = configuration.clone();
+      NexmarkConfiguration result = configuration.copy();
       result.overrideFromOptions(options);
       results.add(result);
     }

http://git-wip-us.apache.org/repos/asf/beam/blob/2f9b4948/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/model/Event.java
----------------------------------------------------------------------
diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/model/Event.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/model/Event.java
index d813833..0e1672e 100644
--- a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/model/Event.java
+++ b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/model/Event.java
@@ -23,55 +23,65 @@ import java.io.OutputStream;
 import java.io.Serializable;
 import javax.annotation.Nullable;
 import org.apache.beam.sdk.coders.Coder;
-import org.apache.beam.sdk.coders.CoderException;
 import org.apache.beam.sdk.coders.CustomCoder;
 import org.apache.beam.sdk.coders.VarIntCoder;
 
 /**
- * An event in the auction system, either a (new) {@link Person}, a (new) {@link Auction},
- * or a {@link Bid}.
+ * An event in the auction system, either a (new) {@link Person}, a (new) {@link Auction}, or a
+ * {@link Bid}.
  */
 public class Event implements KnownSize, Serializable {
-  private static final Coder<Integer> INT_CODER = VarIntCoder.of();
+  private enum Tag {
+    PERSON(0),
+    AUCTION(1),
+    BID(2);
 
-  public static final Coder<Event> CODER = new CustomCoder<Event>() {
-    @Override
-    public void encode(Event value, OutputStream outStream)
-        throws CoderException, IOException {
-      if (value.newPerson != null) {
-        INT_CODER.encode(0, outStream);
-        Person.CODER.encode(value.newPerson, outStream);
-      } else if (value.newAuction != null) {
-        INT_CODER.encode(1, outStream);
-        Auction.CODER.encode(value.newAuction, outStream);
-      } else if (value.bid != null) {
-        INT_CODER.encode(2, outStream);
-        Bid.CODER.encode(value.bid, outStream);
-      } else {
-        throw new RuntimeException("invalid event");
-      }
-    }
+    private int value = -1;
 
-    @Override
-    public Event decode(
-        InputStream inStream)
-        throws CoderException, IOException {
-      int tag = INT_CODER.decode(inStream);
-      if (tag == 0) {
-        Person person = Person.CODER.decode(inStream);
-        return new Event(person);
-      } else if (tag == 1) {
-        Auction auction = Auction.CODER.decode(inStream);
-        return new Event(auction);
-      } else if (tag == 2) {
-        Bid bid = Bid.CODER.decode(inStream);
-        return new Event(bid);
-      } else {
-        throw new RuntimeException("invalid event encoding");
-      }
+    Tag(int value){
+      this.value = value;
     }
-    @Override public void verifyDeterministic() throws NonDeterministicException {}
-  };
+  }
+  private static final Coder<Integer> INT_CODER = VarIntCoder.of();
+
+  public static final Coder<Event> CODER =
+      new CustomCoder<Event>() {
+        @Override
+        public void encode(Event value, OutputStream outStream) throws IOException {
+          if (value.newPerson != null) {
+            INT_CODER.encode(Tag.PERSON.value, outStream);
+            Person.CODER.encode(value.newPerson, outStream);
+          } else if (value.newAuction != null) {
+            INT_CODER.encode(Tag.AUCTION.value, outStream);
+            Auction.CODER.encode(value.newAuction, outStream);
+          } else if (value.bid != null) {
+            INT_CODER.encode(Tag.BID.value, outStream);
+            Bid.CODER.encode(value.bid, outStream);
+          } else {
+            throw new RuntimeException("invalid event");
+          }
+        }
+
+        @Override
+        public Event decode(InputStream inStream) throws IOException {
+          int tag = INT_CODER.decode(inStream);
+          if (tag == Tag.PERSON.value) {
+            Person person = Person.CODER.decode(inStream);
+            return new Event(person);
+          } else if (tag == Tag.AUCTION.value) {
+            Auction auction = Auction.CODER.decode(inStream);
+            return new Event(auction);
+          } else if (tag == Tag.BID.value) {
+            Bid bid = Bid.CODER.decode(inStream);
+            return new Event(bid);
+          } else {
+            throw new RuntimeException("invalid event encoding");
+          }
+        }
+
+        @Override
+        public void verifyDeterministic() throws NonDeterministicException {}
+      };
 
   @Nullable
   @org.apache.avro.reflect.Nullable
@@ -111,10 +121,7 @@ public class Event implements KnownSize, Serializable {
     this.bid = bid;
   }
 
-  /**
-   * Return a copy of event which captures {@code annotation}.
-   * (Used for debugging).
-   */
+  /** Return a copy of event which captures {@code annotation}. (Used for debugging). */
   public Event withAnnotation(String annotation) {
     if (newPerson != null) {
       return new Event(newPerson.withAnnotation(annotation));
@@ -125,9 +132,7 @@ public class Event implements KnownSize, Serializable {
     }
   }
 
-  /**
-   * Does event have {@code annotation}? (Used for debugging.)
-   */
+  /** Does event have {@code annotation}? (Used for debugging.) */
   public boolean hasAnnotation(String annotation) {
     if (newPerson != null) {
       return newPerson.hasAnnotation(annotation);

http://git-wip-us.apache.org/repos/asf/beam/blob/2f9b4948/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query0Model.java
----------------------------------------------------------------------
diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query0Model.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query0Model.java
index 8e65591..e2522b8 100644
--- a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query0Model.java
+++ b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query0Model.java
@@ -45,7 +45,6 @@ public class Query0Model extends NexmarkQueryModel {
         return;
       }
       addResult(timestampedEvent);
-      //TODO test fails because offset of some hundreds of ms beween expect and actual
     }
   }
 

http://git-wip-us.apache.org/repos/asf/beam/blob/2f9b4948/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query1Model.java
----------------------------------------------------------------------
diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query1Model.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query1Model.java
index 5d4de45..f07db80 100644
--- a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query1Model.java
+++ b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query1Model.java
@@ -57,7 +57,6 @@ public class Query1Model extends NexmarkQueryModel implements Serializable {
       TimestampedValue<Bid> result =
           TimestampedValue.of(resultBid, timestampedEvent.getTimestamp());
       addResult(result);
-      //TODO test fails because offset of some hundreds of ms beween expect and actual
     }
   }
 

http://git-wip-us.apache.org/repos/asf/beam/blob/2f9b4948/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query3.java
----------------------------------------------------------------------
diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query3.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query3.java
index f74b78d..f2b66d7 100644
--- a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query3.java
+++ b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query3.java
@@ -29,13 +29,13 @@ import org.apache.beam.integration.nexmark.model.Person;
 import org.apache.beam.sdk.coders.ListCoder;
 import org.apache.beam.sdk.metrics.Counter;
 import org.apache.beam.sdk.metrics.Metrics;
+import org.apache.beam.sdk.state.StateSpec;
+import org.apache.beam.sdk.state.StateSpecs;
 import org.apache.beam.sdk.state.TimeDomain;
 import org.apache.beam.sdk.state.Timer;
 import org.apache.beam.sdk.state.TimerSpec;
 import org.apache.beam.sdk.state.TimerSpecs;
 import org.apache.beam.sdk.state.ValueState;
-import org.apache.beam.sdk.state.StateSpec;
-import org.apache.beam.sdk.state.StateSpecs;
 import org.apache.beam.sdk.transforms.DoFn;
 import org.apache.beam.sdk.transforms.Filter;
 import org.apache.beam.sdk.transforms.ParDo;
@@ -243,9 +243,9 @@ public class Query3 extends NexmarkQuery {
           theNewPerson = newPerson;
         } else {
           if (theNewPerson.equals(newPerson)) {
-            LOG.error("**** duplicate person {} ****", theNewPerson);
+            LOG.error("Duplicate person {}", theNewPerson);
           } else {
-            LOG.error("**** conflicting persons {} and {} ****", theNewPerson, newPerson);
+            LOG.error("Conflicting persons {} and {}", theNewPerson, newPerson);
           }
           fatalCounter.inc();
           continue;

http://git-wip-us.apache.org/repos/asf/beam/blob/2f9b4948/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query5.java
----------------------------------------------------------------------
diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query5.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query5.java
index 1944330..bdf3e5f 100644
--- a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query5.java
+++ b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query5.java
@@ -63,56 +63,64 @@ public class Query5 extends NexmarkQuery {
         // Only want the bid events.
         .apply(JUST_BIDS)
         // Window the bids into sliding windows.
-        .apply(Window.<Bid>into(
-            SlidingWindows.of(Duration.standardSeconds(configuration.windowSizeSec))
-                .every(Duration.standardSeconds(configuration.windowPeriodSec))))
+        .apply(
+            Window.<Bid>into(
+                SlidingWindows.of(Duration.standardSeconds(configuration.windowSizeSec))
+                    .every(Duration.standardSeconds(configuration.windowPeriodSec))))
         // Project just the auction id.
         .apply("BidToAuction", BID_TO_AUCTION)
 
         // Count the number of bids per auction id.
         .apply(Count.<Long>perElement())
 
-      // We'll want to keep all auctions with the maximal number of bids.
+        // We'll want to keep all auctions with the maximal number of bids.
         // Start by lifting each into a singleton list.
         // need to do so because bellow combine returns a list of auctions in the key in case of
         // equal number of bids. Combine needs to have same input type and return type.
-        .apply(name + ".ToSingletons",
-            ParDo.of(new DoFn<KV<Long, Long>, KV<List<Long>, Long>>() {
+        .apply(
+            name + ".ToSingletons",
+            ParDo.of(
+                new DoFn<KV<Long, Long>, KV<List<Long>, Long>>() {
                   @ProcessElement
                   public void processElement(ProcessContext c) {
-                    c.output(KV.of(Collections.singletonList(c.element().getKey()), c.element().getValue()));
+                    c.output(
+                        KV.of(
+                            Collections.singletonList(c.element().getKey()),
+                            c.element().getValue()));
                   }
                 }))
 
         // Keep only the auction ids with the most bids.
         .apply(
-            Combine
-                .globally(new Combine.BinaryCombineFn<KV<List<Long>, Long>>() {
-                  @Override
-                  public KV<List<Long>, Long> apply(
-                      KV<List<Long>, Long> left, KV<List<Long>, Long> right) {
-                    List<Long> leftBestAuctions = left.getKey();
-                    long leftCount = left.getValue();
-                    List<Long> rightBestAuctions = right.getKey();
-                    long rightCount = right.getValue();
-                    if (leftCount > rightCount) {
-                      return left;
-                    } else if (leftCount < rightCount) {
-                      return right;
-                    } else {
-                      List<Long> newBestAuctions = new ArrayList<>();
-                      newBestAuctions.addAll(leftBestAuctions);
-                      newBestAuctions.addAll(rightBestAuctions);
-                      return KV.of(newBestAuctions, leftCount);
-                    }
-                  }
-                })
+            Combine.globally(
+                    new Combine.BinaryCombineFn<KV<List<Long>, Long>>() {
+                      @Override
+                      public KV<List<Long>, Long> apply(
+                          KV<List<Long>, Long> left, KV<List<Long>, Long> right) {
+                        List<Long> leftBestAuctions = left.getKey();
+                        long leftCount = left.getValue();
+                        List<Long> rightBestAuctions = right.getKey();
+                        long rightCount = right.getValue();
+                        if (leftCount > rightCount) {
+                          return left;
+                        } else if (leftCount < rightCount) {
+                          return right;
+                        } else {
+                          List<Long> newBestAuctions = new ArrayList<>();
+                          newBestAuctions.addAll(leftBestAuctions);
+                          newBestAuctions.addAll(rightBestAuctions);
+                          return KV.of(newBestAuctions, leftCount);
+                        }
+                      }
+                    })
                 .withoutDefaults()
                 .withFanout(configuration.fanout))
 
         // Project into result.
-        .apply(name + ".Select",
-            ParDo.of(new DoFn<KV<List<Long>, Long>, AuctionCount>() {
+        .apply(
+            name + ".Select",
+            ParDo.of(
+                new DoFn<KV<List<Long>, Long>, AuctionCount>() {
                   @ProcessElement
                   public void processElement(ProcessContext c) {
                     long count = c.element().getValue();

http://git-wip-us.apache.org/repos/asf/beam/blob/2f9b4948/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query7.java
----------------------------------------------------------------------
diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query7.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query7.java
index 2a94ca9..217d0d4 100644
--- a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query7.java
+++ b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query7.java
@@ -63,7 +63,7 @@ public class Query7 extends NexmarkQuery {
     // requires an additional scan per window, with the associated cost of snapshotted state and
     // its I/O. We'll keep this implementation since it illustrates the use of side inputs.
     final PCollectionView<Long> maxPriceView =
-        slidingBids //
+        slidingBids
             .apply("BidToPrice", BID_TO_PRICE)
             .apply(Max.longsGlobally().withFanout(configuration.fanout).asSingletonView());
 

http://git-wip-us.apache.org/repos/asf/beam/blob/2f9b4948/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query7Model.java
----------------------------------------------------------------------
diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query7Model.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query7Model.java
index 5c039f9..0ada5e8 100644
--- a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query7Model.java
+++ b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query7Model.java
@@ -111,7 +111,6 @@ public class Query7Model extends NexmarkQueryModel implements Serializable {
       }
       // Keep only the highest bids.
       captureBid(event.bid);
-      //TODO test fails because offset of some hundreds of ms between expect and actual
     }
   }
 

http://git-wip-us.apache.org/repos/asf/beam/blob/2f9b4948/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/WinningBids.java
----------------------------------------------------------------------
diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/WinningBids.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/WinningBids.java
index bd6c2ed..d4ca177 100644
--- a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/WinningBids.java
+++ b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/WinningBids.java
@@ -23,12 +23,12 @@ import com.fasterxml.jackson.annotation.JsonCreator;
 import java.io.IOException;
 import java.io.InputStream;
 import java.io.OutputStream;
-import java.io.Serializable;
 import java.util.ArrayList;
 import java.util.Collection;
 import java.util.Collections;
 import java.util.List;
 import java.util.Map;
+import java.util.Objects;
 import java.util.TreeMap;
 import org.apache.beam.integration.nexmark.NexmarkConfiguration;
 import org.apache.beam.integration.nexmark.NexmarkUtils;
@@ -139,6 +139,24 @@ public class WinningBids extends PTransform<PCollection<Event>, PCollection<Auct
       return String.format("AuctionOrBidWindow{start:%s; end:%s; auction:%d; isAuctionWindow:%s}",
           start(), end(), auction, isAuctionWindow);
     }
+
+    @Override public boolean equals(Object o) {
+      if (this == o) {
+        return true;
+      }
+      if (o == null || getClass() != o.getClass()) {
+        return false;
+      }
+      if (!super.equals(o)) {
+        return false;
+      }
+      AuctionOrBidWindow that = (AuctionOrBidWindow) o;
+      return (isAuctionWindow == that.isAuctionWindow) && (auction == that.auction);
+    }
+
+    @Override public int hashCode() {
+      return Objects.hash(isAuctionWindow, auction);
+    }
   }
 
   /**
@@ -374,4 +392,21 @@ public class WinningBids extends PTransform<PCollection<Event>, PCollection<Auct
           }
         ));
   }
+
+  @Override
+  public int hashCode() {
+    return Objects.hash(auctionOrBidWindowFn);
+  }
+
+  @Override public boolean equals(Object o) {
+    if (this == o) {
+      return true;
+    }
+    if (o == null || getClass() != o.getClass()) {
+      return false;
+    }
+
+    WinningBids that = (WinningBids) o;
+    return auctionOrBidWindowFn.equals(that.auctionOrBidWindowFn);
+  }
 }

http://git-wip-us.apache.org/repos/asf/beam/blob/2f9b4948/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/WinningBidsSimulator.java
----------------------------------------------------------------------
diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/WinningBidsSimulator.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/WinningBidsSimulator.java
index 7d74f8f..9624a9d 100644
--- a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/WinningBidsSimulator.java
+++ b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/WinningBidsSimulator.java
@@ -181,7 +181,6 @@ public class WinningBidsSimulator extends AbstractSimulator<Event, AuctionBid> {
         return;
       }
       addResult(result);
-      //TODO test fails because offset of some hundreds of ms beween expect and actual
       return;
     }
 

http://git-wip-us.apache.org/repos/asf/beam/blob/2f9b4948/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/sources/Generator.java
----------------------------------------------------------------------
diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/sources/Generator.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/sources/Generator.java
index 4f548cd..f6deceb 100644
--- a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/sources/Generator.java
+++ b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/sources/Generator.java
@@ -26,6 +26,7 @@ import java.io.Serializable;
 import java.util.Arrays;
 import java.util.Iterator;
 import java.util.List;
+import java.util.Objects;
 import java.util.Random;
 import org.apache.beam.integration.nexmark.model.Auction;
 import org.apache.beam.integration.nexmark.model.Bid;
@@ -167,7 +168,7 @@ public class Generator implements Iterator<TimestampedValue<Event>>, Serializabl
     }
 
     /**
-     * Return a deep clone of next event with delay added to wallclock timestamp and
+     * Return a deep copy of next event with delay added to wallclock timestamp and
      * event annotate as 'LATE'.
      */
     public NextEvent withDelay(long delayMs) {
@@ -175,6 +176,26 @@ public class Generator implements Iterator<TimestampedValue<Event>>, Serializabl
           wallclockTimestamp + delayMs, eventTimestamp, event.withAnnotation("LATE"), watermark);
     }
 
+    @Override public boolean equals(Object o) {
+      if (this == o) {
+        return true;
+      }
+      if (o == null || getClass() != o.getClass()) {
+        return false;
+      }
+
+      NextEvent nextEvent = (NextEvent) o;
+
+      return (wallclockTimestamp == nextEvent.wallclockTimestamp
+          && eventTimestamp == nextEvent.eventTimestamp
+          && watermark == nextEvent.watermark
+          && event.equals(nextEvent.event));
+    }
+
+    @Override public int hashCode() {
+      return Objects.hash(wallclockTimestamp, eventTimestamp, watermark, event);
+    }
+
     @Override
     public int compareTo(NextEvent other) {
       int i = Long.compare(wallclockTimestamp, other.wallclockTimestamp);
@@ -221,11 +242,12 @@ public class Generator implements Iterator<TimestampedValue<Event>>, Serializabl
   }
 
   /**
-   * Return a deep clone of this generator.
+   * Return a deep copy of this generator.
    */
-  @Override
-  public Generator clone() {
-    return new Generator(config.clone(), numEvents, wallclockBaseTime);
+  public Generator copy() {
+    checkNotNull(config);
+    Generator result = new Generator(config, numEvents, wallclockBaseTime);
+    return result;
   }
 
   /**
@@ -243,9 +265,9 @@ public class Generator implements Iterator<TimestampedValue<Event>>, Serializabl
    */
   public GeneratorConfig splitAtEventId(long eventId) {
     long newMaxEvents = eventId - (config.firstEventId + config.firstEventNumber);
-    GeneratorConfig remainConfig = config.cloneWith(config.firstEventId,
+    GeneratorConfig remainConfig = config.copyWith(config.firstEventId,
         config.maxEvents - newMaxEvents, config.firstEventNumber + newMaxEvents);
-    config = config.cloneWith(config.firstEventId, newMaxEvents, config.firstEventNumber);
+    config = config.copyWith(config.firstEventId, newMaxEvents, config.firstEventNumber);
     return remainConfig;
   }
 

http://git-wip-us.apache.org/repos/asf/beam/blob/2f9b4948/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/sources/GeneratorConfig.java
----------------------------------------------------------------------
diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/sources/GeneratorConfig.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/sources/GeneratorConfig.java
index 5799bb2..95c276b 100644
--- a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/sources/GeneratorConfig.java
+++ b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/sources/GeneratorConfig.java
@@ -17,6 +17,8 @@
  */
 package org.apache.beam.integration.nexmark.sources;
 
+import static com.google.common.base.Preconditions.checkNotNull;
+
 import java.io.Serializable;
 import java.util.ArrayList;
 import java.util.List;
@@ -29,6 +31,7 @@ import org.apache.beam.sdk.values.KV;
  * Parameters controlling how {@link Generator} synthesizes {@link Event} elements.
  */
 public class GeneratorConfig implements Serializable {
+
   /**
    * We start the ids at specific values to help ensure the queries find a match even on
    * small synthesized dataset sizes.
@@ -132,18 +135,13 @@ public class GeneratorConfig implements Serializable {
   }
 
   /**
-   * Return a clone of this config.
-   */
-  @Override
-  public GeneratorConfig clone() {
-    return new GeneratorConfig(configuration, baseTime, firstEventId, maxEvents, firstEventNumber);
-  }
-
-  /**
-   * Return clone of this config except with given parameters.
+   * Return a copy of this config.
    */
-  public GeneratorConfig cloneWith(long firstEventId, long maxEvents, long firstEventNumber) {
-    return new GeneratorConfig(configuration, baseTime, firstEventId, maxEvents, firstEventNumber);
+  public GeneratorConfig copy() {
+    GeneratorConfig result;
+      result = new GeneratorConfig(configuration, baseTime, firstEventId,
+          maxEvents, firstEventNumber);
+    return result;
   }
 
   /**
@@ -164,7 +162,7 @@ public class GeneratorConfig implements Serializable {
           // Don't loose any events to round-down.
           subMaxEvents = maxEvents - subMaxEvents * (n - 1);
         }
-        results.add(cloneWith(subFirstEventId, subMaxEvents, firstEventNumber));
+        results.add(copyWith(subFirstEventId, subMaxEvents, firstEventNumber));
         subFirstEventId += subMaxEvents;
       }
     }
@@ -172,6 +170,13 @@ public class GeneratorConfig implements Serializable {
   }
 
   /**
+   * Return copy of this config except with given parameters.
+   */
+  public GeneratorConfig copyWith(long firstEventId, long maxEvents, long firstEventNumber) {
+    return new GeneratorConfig(configuration, baseTime, firstEventId, maxEvents, firstEventNumber);
+  }
+
+  /**
    * Return an estimate of the bytes needed by {@code numEvents}.
    */
   public long estimatedBytesForEvents(long numEvents) {

http://git-wip-us.apache.org/repos/asf/beam/blob/2f9b4948/integration/java/nexmark/src/test/java/org/apache/beam/integration/nexmark/queries/QueryTest.java
----------------------------------------------------------------------
diff --git a/integration/java/nexmark/src/test/java/org/apache/beam/integration/nexmark/queries/QueryTest.java b/integration/java/nexmark/src/test/java/org/apache/beam/integration/nexmark/queries/QueryTest.java
index b005d65..64a8e4f 100644
--- a/integration/java/nexmark/src/test/java/org/apache/beam/integration/nexmark/queries/QueryTest.java
+++ b/integration/java/nexmark/src/test/java/org/apache/beam/integration/nexmark/queries/QueryTest.java
@@ -37,7 +37,7 @@ import org.junit.runners.JUnit4;
 /** Test the various NEXMark queries yield results coherent with their models. */
 @RunWith(JUnit4.class)
 public class QueryTest {
-  private static final NexmarkConfiguration CONFIG = NexmarkConfiguration.DEFAULT.clone();
+  private static final NexmarkConfiguration CONFIG = NexmarkConfiguration.DEFAULT.copy();
 
   static {
     // careful, results of tests are linked to numEventGenerators because of timestamp generation
@@ -55,12 +55,8 @@ public class QueryTest {
     if (streamingMode) {
       results =
           p.apply(name + ".ReadUnBounded", NexmarkUtils.streamEventsSource(CONFIG)).apply(query);
-      //TODO Ismael this should not be called explicitly
-      results.setIsBoundedInternal(PCollection.IsBounded.UNBOUNDED);
     } else {
       results = p.apply(name + ".ReadBounded", NexmarkUtils.batchEventsSource(CONFIG)).apply(query);
-      //TODO Ismael this should not be called explicitly
-      results.setIsBoundedInternal(PCollection.IsBounded.BOUNDED);
     }
     PAssert.that(results).satisfies(model.assertionFor());
     PipelineResult result = p.run();

http://git-wip-us.apache.org/repos/asf/beam/blob/2f9b4948/integration/java/nexmark/src/test/java/org/apache/beam/integration/nexmark/sources/UnboundedEventSourceTest.java
----------------------------------------------------------------------
diff --git a/integration/java/nexmark/src/test/java/org/apache/beam/integration/nexmark/sources/UnboundedEventSourceTest.java b/integration/java/nexmark/src/test/java/org/apache/beam/integration/nexmark/sources/UnboundedEventSourceTest.java
index 1d04e2a..1ecc33e 100644
--- a/integration/java/nexmark/src/test/java/org/apache/beam/integration/nexmark/sources/UnboundedEventSourceTest.java
+++ b/integration/java/nexmark/src/test/java/org/apache/beam/integration/nexmark/sources/UnboundedEventSourceTest.java
@@ -28,7 +28,6 @@ import java.util.Set;
 
 import org.apache.beam.integration.nexmark.NexmarkConfiguration;
 import org.apache.beam.integration.nexmark.model.Event;
-import org.apache.beam.sdk.Pipeline;
 import org.apache.beam.sdk.io.UnboundedSource.CheckpointMark;
 import org.apache.beam.sdk.io.UnboundedSource.UnboundedReader;
 import org.apache.beam.sdk.options.PipelineOptions;
@@ -95,12 +94,11 @@ public class UnboundedEventSourceTest {
 
     while (n > 0) {
       int m = Math.min(459 + random.nextInt(455), n);
-      System.out.printf("reading %d...\n", m);
+      System.out.printf("reading %d...%n", m);
       checker.add(m, reader, modelGenerator);
       n -= m;
-      System.out.printf("splitting with %d remaining...\n", n);
+      System.out.printf("splitting with %d remaining...%n", n);
       CheckpointMark checkpointMark = reader.getCheckpointMark();
-      assertTrue(checkpointMark instanceof Generator.Checkpoint);
       reader = source.createReader(options, (Generator.Checkpoint) checkpointMark);
     }
 


[06/55] [abbrv] beam git commit: NexMark

Posted by ie...@apache.org.
NexMark

Port unit tests, cleanup pom and add license to readme


Project: http://git-wip-us.apache.org/repos/asf/beam/repo
Commit: http://git-wip-us.apache.org/repos/asf/beam/commit/1f08970a
Tree: http://git-wip-us.apache.org/repos/asf/beam/tree/1f08970a
Diff: http://git-wip-us.apache.org/repos/asf/beam/diff/1f08970a

Branch: refs/heads/master
Commit: 1f08970a8fdc9c5e5613227031125d9d929ca841
Parents: f0ce31b
Author: Mark Shields <ma...@google.com>
Authored: Mon Mar 28 16:25:29 2016 -0700
Committer: Ismaël Mejía <ie...@gmail.com>
Committed: Wed Aug 23 19:07:26 2017 +0200

----------------------------------------------------------------------
 integration/java/pom.xml                        | 288 +++++++
 .../integration/nexmark/AbstractSimulator.java  | 212 ++++++
 .../beam/integration/nexmark/Auction.java       | 190 +++++
 .../beam/integration/nexmark/AuctionBid.java    |  87 +++
 .../beam/integration/nexmark/AuctionCount.java  |  90 +++
 .../beam/integration/nexmark/AuctionPrice.java  |  91 +++
 .../integration/nexmark/BEAM_ON_FLINK_ON_GCP.md | 282 +++++++
 .../apache/beam/integration/nexmark/Bid.java    | 178 +++++
 .../integration/nexmark/BidsPerSession.java     |  89 +++
 .../integration/nexmark/BoundedEventSource.java | 197 +++++
 .../beam/integration/nexmark/CategoryPrice.java | 100 +++
 .../apache/beam/integration/nexmark/Done.java   |  83 +++
 .../apache/beam/integration/nexmark/Event.java  | 181 +++++
 .../beam/integration/nexmark/Generator.java     | 590 +++++++++++++++
 .../integration/nexmark/GeneratorConfig.java    | 295 ++++++++
 .../beam/integration/nexmark/IdNameReserve.java | 100 +++
 .../beam/integration/nexmark/KnownSize.java     |  27 +
 .../beam/integration/nexmark/Monitor.java       | 102 +++
 .../integration/nexmark/NameCityStateId.java    | 106 +++
 .../nexmark/NexmarkConfiguration.java           | 662 ++++++++++++++++
 .../beam/integration/nexmark/NexmarkDriver.java | 297 ++++++++
 .../integration/nexmark/NexmarkFlinkDriver.java |  49 ++
 .../integration/nexmark/NexmarkFlinkRunner.java |  67 ++
 .../nexmark/NexmarkGoogleDriver.java            |  90 +++
 .../nexmark/NexmarkGoogleRunner.java            | 660 ++++++++++++++++
 .../nexmark/NexmarkInProcessDriver.java         |  48 ++
 .../nexmark/NexmarkInProcessRunner.java         |  77 ++
 .../beam/integration/nexmark/NexmarkPerf.java   | 212 ++++++
 .../beam/integration/nexmark/NexmarkQuery.java  | 276 +++++++
 .../integration/nexmark/NexmarkQueryModel.java  | 123 +++
 .../beam/integration/nexmark/NexmarkRunner.java | 746 +++++++++++++++++++
 .../beam/integration/nexmark/NexmarkSuite.java  | 112 +++
 .../beam/integration/nexmark/NexmarkUtils.java  | 681 +++++++++++++++++
 .../beam/integration/nexmark/Options.java       | 360 +++++++++
 .../apache/beam/integration/nexmark/Person.java | 166 +++++
 .../beam/integration/nexmark/PubsubHelper.java  | 217 ++++++
 .../apache/beam/integration/nexmark/Query0.java |  72 ++
 .../beam/integration/nexmark/Query0Model.java   |  62 ++
 .../apache/beam/integration/nexmark/Query1.java |  64 ++
 .../beam/integration/nexmark/Query10.java       | 378 ++++++++++
 .../beam/integration/nexmark/Query11.java       |  76 ++
 .../beam/integration/nexmark/Query12.java       |  79 ++
 .../beam/integration/nexmark/Query1Model.java   |  73 ++
 .../apache/beam/integration/nexmark/Query2.java |  75 ++
 .../beam/integration/nexmark/Query2Model.java   |  76 ++
 .../apache/beam/integration/nexmark/Query3.java | 248 ++++++
 .../beam/integration/nexmark/Query3Model.java   | 119 +++
 .../apache/beam/integration/nexmark/Query4.java | 110 +++
 .../beam/integration/nexmark/Query4Model.java   | 181 +++++
 .../apache/beam/integration/nexmark/Query5.java | 127 ++++
 .../beam/integration/nexmark/Query5Model.java   | 174 +++++
 .../apache/beam/integration/nexmark/Query6.java | 154 ++++
 .../beam/integration/nexmark/Query6Model.java   | 128 ++++
 .../apache/beam/integration/nexmark/Query7.java |  87 +++
 .../beam/integration/nexmark/Query7Model.java   | 128 ++++
 .../apache/beam/integration/nexmark/Query8.java |  92 +++
 .../beam/integration/nexmark/Query8Model.java   | 145 ++++
 .../apache/beam/integration/nexmark/Query9.java |  40 +
 .../beam/integration/nexmark/Query9Model.java   |  44 ++
 .../apache/beam/integration/nexmark/README.md   | 166 +++++
 .../beam/integration/nexmark/SellerPrice.java   |  91 +++
 .../nexmark/UnboundedEventSource.java           | 329 ++++++++
 .../beam/integration/nexmark/WinningBids.java   | 378 ++++++++++
 .../nexmark/WinningBidsSimulator.java           | 203 +++++
 .../nexmark/BoundedEventSourceTest.java         |  71 ++
 .../beam/integration/nexmark/GeneratorTest.java | 111 +++
 .../beam/integration/nexmark/QueryTest.java     | 103 +++
 .../nexmark/UnboundedEventSourceTest.java       | 109 +++
 68 files changed, 12424 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/beam/blob/1f08970a/integration/java/pom.xml
----------------------------------------------------------------------
diff --git a/integration/java/pom.xml b/integration/java/pom.xml
new file mode 100644
index 0000000..b160b56
--- /dev/null
+++ b/integration/java/pom.xml
@@ -0,0 +1,288 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<!--
+    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.
+-->
+<project xmlns="http://maven.apache.org/POM/4.0.0"
+         xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
+         xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
+  <modelVersion>4.0.0</modelVersion>
+
+  <parent>
+    <groupId>org.apache.beam</groupId>
+    <artifactId>parent</artifactId>
+    <version>0.2.0-incubating-SNAPSHOT</version>
+    <relativePath>../../pom.xml</relativePath>
+  </parent>
+
+  <artifactId>java-integration-all</artifactId>
+
+  <name>Apache Beam :: Integration Tests :: Java All</name>
+
+  <packaging>jar</packaging>
+
+  <properties>
+    <skipITs>true</skipITs>
+  </properties>
+
+  <build>
+    <extensions>
+      <!-- Use os-maven-plugin to initialize the "os.detected" properties -->
+      <extension>
+        <groupId>kr.motd.maven</groupId>
+        <artifactId>os-maven-plugin</artifactId>
+        <version>1.4.0.Final</version>
+      </extension>
+    </extensions>
+
+    <plugins>
+      <plugin>
+        <groupId>org.apache.maven.plugins</groupId>
+        <artifactId>maven-compiler-plugin</artifactId>
+      </plugin>
+
+      <plugin>
+        <groupId>org.apache.maven.plugins</groupId>
+        <artifactId>maven-surefire-plugin</artifactId>
+        <configuration>
+          <systemPropertyVariables>
+            <beamTestPipelineOptions>
+            </beamTestPipelineOptions>
+          </systemPropertyVariables>
+        </configuration>
+      </plugin>
+
+      <plugin>
+        <groupId>org.apache.maven.plugins</groupId>
+        <artifactId>maven-dependency-plugin</artifactId>
+      </plugin>
+
+      <plugin>
+        <groupId>org.apache.maven.plugins</groupId>
+        <artifactId>maven-checkstyle-plugin</artifactId>
+      </plugin>
+
+
+      <!-- Source plugin for generating source and test-source JARs. -->
+      <plugin>
+        <groupId>org.apache.maven.plugins</groupId>
+        <artifactId>maven-source-plugin</artifactId>
+      </plugin>
+
+
+      <plugin>
+        <groupId>org.apache.maven.plugins</groupId>
+        <artifactId>maven-shade-plugin</artifactId>
+        <executions>
+          <execution>
+            <phase>package</phase>
+            <goals>
+              <goal>shade</goal>
+            </goals>
+            <configuration>
+              <finalName>${project.artifactId}-bundled-${project.version}</finalName>
+              <artifactSet>
+                <includes>
+                  <include>*:*</include>
+                </includes>
+              </artifactSet>
+              <filters>
+                <filter>
+                  <artifact>*:*</artifact>
+                  <excludes>
+                    <exclude>META-INF/*.SF</exclude>
+                    <exclude>META-INF/*.DSA</exclude>
+                    <exclude>META-INF/*.RSA</exclude>
+                  </excludes>
+                </filter>
+              </filters>
+            </configuration>
+          </execution>
+        </executions>
+      </plugin>
+
+      <plugin>
+        <groupId>org.apache.maven.plugins</groupId>
+        <artifactId>maven-jar-plugin</artifactId>
+      </plugin>
+
+      <!-- Avro plugin for automatic code generation -->
+      <plugin>
+        <groupId>org.apache.avro</groupId>
+        <artifactId>avro-maven-plugin</artifactId>
+        <version>${avro.version}</version>
+        <executions>
+          <execution>
+            <id>schemas</id>
+            <phase>generate-sources</phase>
+            <goals>
+              <goal>schema</goal>
+            </goals>
+            <configuration>
+              <sourceDirectory>${project.basedir}/src/main/</sourceDirectory>
+              <outputDirectory>${project.build.directory}/generated-sources/java</outputDirectory>
+            </configuration>
+          </execution>
+        </executions>
+      </plugin>
+
+      <!-- Coverage analysis for unit tests. -->
+      <plugin>
+        <groupId>org.jacoco</groupId>
+        <artifactId>jacoco-maven-plugin</artifactId>
+      </plugin>
+    </plugins>
+  </build>
+
+  <dependencies>
+    <!-- Java SDK -->
+    <dependency>
+      <groupId>org.apache.beam</groupId>
+      <artifactId>java-sdk-all</artifactId>
+    </dependency>
+
+    <!-- Java runner for Google Cloud Dataflow -->
+    <dependency>
+      <groupId>org.apache.beam.runners</groupId>
+      <artifactId>google-cloud-dataflow-java</artifactId>
+      <version>${project.version}</version>
+    </dependency>
+
+    <!-- Direct runner -->
+    <dependency>
+      <groupId>org.apache.beam.runners</groupId>
+      <artifactId>direct</artifactId>
+      <version>${project.version}</version>
+    </dependency>
+
+    <!-- Java runner for Flink -->
+    <dependency>
+      <groupId>org.apache.beam.runners</groupId>
+      <artifactId>flink_2.10</artifactId>
+      <version>${project.version}</version>
+    </dependency>
+
+    <dependency>
+      <groupId>org.apache.flink</groupId>
+      <artifactId>flink-shaded-hadoop2</artifactId>
+      <version>1.0.3</version>
+      <scope>provided</scope>
+    </dependency>
+
+    <!-- Extra libraries -->
+    <dependency>
+      <groupId>com.google.apis</groupId>
+      <artifactId>google-api-services-dataflow</artifactId>
+      <version>${dataflow.version}</version>
+    </dependency>
+
+    <dependency>
+      <groupId>joda-time</groupId>
+      <artifactId>joda-time</artifactId>
+    </dependency>
+
+    <dependency>
+      <groupId>com.fasterxml.jackson.core</groupId>
+      <artifactId>jackson-core</artifactId>
+      <version>${jackson.version}</version>
+    </dependency>
+
+    <dependency>
+      <groupId>com.google.apis</groupId>
+      <artifactId>google-api-services-bigquery</artifactId>
+      <version>${bigquery.version}</version>
+    </dependency>
+
+    <dependency>
+      <groupId>com.google.cloud.bigdataoss</groupId>
+      <artifactId>gcsio</artifactId>
+      <version>${google-cloud-bigdataoss.version}</version>
+    </dependency>
+
+    <dependency>
+      <groupId>com.fasterxml.jackson.core</groupId>
+      <artifactId>jackson-annotations</artifactId>
+      <version>${jackson.version}</version>
+    </dependency>
+
+    <dependency>
+      <groupId>com.fasterxml.jackson.core</groupId>
+      <artifactId>jackson-databind</artifactId>
+      <version>${jackson.version}</version>
+    </dependency>
+
+    <dependency>
+      <groupId>org.apache.avro</groupId>
+      <artifactId>avro</artifactId>
+      <version>${avro.version}</version>
+    </dependency>
+
+    <dependency>
+      <groupId>com.google.guava</groupId>
+      <artifactId>guava</artifactId>
+      <version>${guava.version}</version>
+    </dependency>
+
+    <dependency>
+      <groupId>org.hamcrest</groupId>
+      <artifactId>hamcrest-all</artifactId>
+      <version>${hamcrest.version}</version>
+    </dependency>
+
+    <dependency>
+      <groupId>org.slf4j</groupId>
+      <artifactId>slf4j-api</artifactId>
+      <version>${slf4j.version}</version>
+      <scope>compile</scope>
+    </dependency>
+
+    <dependency>
+      <groupId>org.slf4j</groupId>
+      <artifactId>slf4j-jdk14</artifactId>
+      <version>${slf4j.version}</version>
+      <!-- When loaded at runtime this will wire up slf4j to the JUL backend -->
+      <scope>runtime</scope>
+    </dependency>
+
+    <dependency>
+      <groupId>com.google.code.findbugs</groupId>
+      <artifactId>jsr305</artifactId>
+      <version>${jsr305.version}</version>
+      <scope>runtime</scope>
+    </dependency>
+
+    <dependency>
+      <groupId>log4j</groupId>
+      <artifactId>log4j</artifactId>
+      <version>1.2.17</version>
+      <scope>runtime</scope>
+    </dependency>
+
+    <dependency>
+      <groupId>junit</groupId>
+      <artifactId>junit</artifactId>
+      <version>${junit.version}</version>
+      <scope>compile</scope>
+    </dependency>
+
+    <dependency>
+      <groupId>io.netty</groupId>
+      <artifactId>netty-tcnative-boringssl-static</artifactId>
+      <version>1.1.33.Fork13</version>
+      <classifier>${os.detected.classifier}</classifier>
+      <scope>runtime</scope>
+    </dependency>
+  </dependencies>
+</project>

http://git-wip-us.apache.org/repos/asf/beam/blob/1f08970a/integration/java/src/main/java/org/apache/beam/integration/nexmark/AbstractSimulator.java
----------------------------------------------------------------------
diff --git a/integration/java/src/main/java/org/apache/beam/integration/nexmark/AbstractSimulator.java b/integration/java/src/main/java/org/apache/beam/integration/nexmark/AbstractSimulator.java
new file mode 100644
index 0000000..6473c35
--- /dev/null
+++ b/integration/java/src/main/java/org/apache/beam/integration/nexmark/AbstractSimulator.java
@@ -0,0 +1,212 @@
+/*
+ * 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.beam.integration.nexmark;
+
+import org.apache.beam.sdk.transforms.windowing.BoundedWindow;
+import org.apache.beam.sdk.values.TimestampedValue;
+
+import org.joda.time.Duration;
+import org.joda.time.Instant;
+
+import java.util.ArrayList;
+import java.util.Iterator;
+import java.util.List;
+
+import javax.annotation.Nullable;
+
+/**
+ * Abstract base class for simulator of a query.
+ *
+ * @param <InputT> Type of input elements.
+ * @param <OutputT> Type of output elements.
+ */
+abstract class AbstractSimulator<InputT, OutputT> {
+  /** Window size for action bucket sampling. */
+  public static final Duration WINDOW_SIZE = Duration.standardMinutes(1);
+
+  /** Input event stream we should draw from. */
+  private final Iterator<TimestampedValue<InputT>> input;
+
+  /** Set to true when no more results. */
+  private boolean isDone;
+
+  /**
+   * Results which have not yet been returned by the {@link #results} iterator.
+   */
+  private final List<TimestampedValue<OutputT>> pendingResults;
+
+  /**
+   * Current window timestamp (ms since epoch).
+   */
+  private long currentWindow;
+
+  /**
+   * Number of (possibly intermediate) results for the current window.
+   */
+  private long currentCount;
+
+  /**
+   * Result counts per window which have not yet been returned by the {@link #resultsPerWindow}
+   * iterator.
+   */
+  private final List<Long> pendingCounts;
+
+  public AbstractSimulator(Iterator<TimestampedValue<InputT>> input) {
+    this.input = input;
+    isDone = false;
+    pendingResults = new ArrayList<>();
+    currentWindow = BoundedWindow.TIMESTAMP_MIN_VALUE.getMillis();
+    currentCount = 0;
+    pendingCounts = new ArrayList<>();
+  }
+
+  /** Called by implementors of {@link #run}: Fetch the next input element. */
+  @Nullable
+  protected TimestampedValue<InputT> nextInput() {
+    if (!input.hasNext()) {
+      return null;
+    }
+    TimestampedValue<InputT> timestampedInput = input.next();
+    NexmarkUtils.info("input: %s", timestampedInput);
+    return timestampedInput;
+  }
+
+  /**
+   * Called by implementors of {@link #run}:  Capture an intermediate result, for the purpose of
+   * recording the expected activity of the query over time.
+   */
+  protected void addIntermediateResult(TimestampedValue<OutputT> result) {
+    NexmarkUtils.info("intermediate result: %s", result);
+    updateCounts(result.getTimestamp());
+  }
+
+  /**
+   * Called by implementors of {@link #run}: Capture a final result, for the purpose of checking
+   * semantic correctness.
+   */
+  protected void addResult(TimestampedValue<OutputT> result) {
+    NexmarkUtils.info("result: %s", result);
+    pendingResults.add(result);
+    updateCounts(result.getTimestamp());
+  }
+
+  /**
+   * Update window and counts.
+   */
+  private void updateCounts(Instant timestamp) {
+    long window = timestamp.getMillis() - timestamp.getMillis() % WINDOW_SIZE.getMillis();
+    if (window > currentWindow) {
+      if (currentWindow > BoundedWindow.TIMESTAMP_MIN_VALUE.getMillis()) {
+        pendingCounts.add(currentCount);
+      }
+      currentCount = 0;
+      currentWindow = window;
+    }
+    currentCount++;
+  }
+
+  /** Called by implementors of {@link #run}: Record that no more results will be emitted. */
+  protected void allDone() {
+    isDone = true;
+  }
+
+  /**
+   * Overridden by derived classes to do the next increment of work. Each call should
+   * call one or more of {@link #nextInput}, {@link #addIntermediateResult}, {@link #addResult}
+   * or {@link #allDone}. It is ok for a single call to emit more than one result via
+   * {@link #addResult}. It is ok for a single call to run the entire simulation, though
+   * this will prevent the {@link #results} and {@link #resultsPerWindow} iterators to
+   * stall.
+   */
+  protected abstract void run();
+
+  /**
+   * Return iterator over all expected timestamped results. The underlying simulator state is
+   * changed. Only one of {@link #results} or {@link #resultsPerWindow} can be called.
+   */
+  public Iterator<TimestampedValue<OutputT>> results() {
+    return new Iterator<TimestampedValue<OutputT>>() {
+      @Override
+      public boolean hasNext() {
+        while (true) {
+          if (!pendingResults.isEmpty()) {
+            return true;
+          }
+          if (isDone) {
+            return false;
+          }
+          run();
+        }
+      }
+
+      @Override
+      public TimestampedValue<OutputT> next() {
+        TimestampedValue<OutputT> result = pendingResults.get(0);
+        pendingResults.remove(0);
+        return result;
+      }
+
+      @Override
+      public void remove() {
+        throw new UnsupportedOperationException();
+      }
+    };
+  }
+
+  /**
+   * Return an iterator over the number of results per {@link #WINDOW_SIZE} period. The underlying
+   * simulator state is changed.  Only one of {@link #results} or {@link #resultsPerWindow} can be
+   * called.
+   */
+  public Iterator<Long> resultsPerWindow() {
+    return new Iterator<Long>() {
+      @Override
+      public boolean hasNext() {
+        while (true) {
+          if (!pendingCounts.isEmpty()) {
+            return true;
+          }
+          if (isDone) {
+            if (currentCount > 0) {
+              pendingCounts.add(currentCount);
+              currentCount = 0;
+              currentWindow = BoundedWindow.TIMESTAMP_MAX_VALUE.getMillis();
+              return true;
+            } else {
+              return false;
+            }
+          }
+          run();
+        }
+      }
+
+      @Override
+      public Long next() {
+        Long result = pendingCounts.get(0);
+        pendingCounts.remove(0);
+        return result;
+      }
+
+      @Override
+      public void remove() {
+        throw new UnsupportedOperationException();
+      }
+    };
+  }
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/1f08970a/integration/java/src/main/java/org/apache/beam/integration/nexmark/Auction.java
----------------------------------------------------------------------
diff --git a/integration/java/src/main/java/org/apache/beam/integration/nexmark/Auction.java b/integration/java/src/main/java/org/apache/beam/integration/nexmark/Auction.java
new file mode 100644
index 0000000..94f2647
--- /dev/null
+++ b/integration/java/src/main/java/org/apache/beam/integration/nexmark/Auction.java
@@ -0,0 +1,190 @@
+/*
+ * 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.beam.integration.nexmark;
+
+import org.apache.beam.sdk.coders.AtomicCoder;
+import org.apache.beam.sdk.coders.Coder;
+import org.apache.beam.sdk.coders.CoderException;
+import org.apache.beam.sdk.coders.StringUtf8Coder;
+import org.apache.beam.sdk.coders.VarLongCoder;
+
+import com.fasterxml.jackson.annotation.JsonProperty;
+import com.fasterxml.jackson.core.JsonProcessingException;
+
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
+import java.io.Serializable;
+
+/**
+ * An auction submitted by a person.
+ */
+public class Auction implements KnownSize, Serializable {
+  private static final Coder<Long> LONG_CODER = VarLongCoder.of();
+  private static final Coder<String> STRING_CODER = StringUtf8Coder.of();
+
+  public static final Coder<Auction> CODER = new AtomicCoder<Auction>() {
+    @Override
+    public void encode(Auction value, OutputStream outStream,
+        Coder.Context context)
+        throws CoderException, IOException {
+      LONG_CODER.encode(value.id, outStream, Context.NESTED);
+      STRING_CODER.encode(value.itemName, outStream, Context.NESTED);
+      STRING_CODER.encode(value.description, outStream, Context.NESTED);
+      LONG_CODER.encode(value.initialBid, outStream, Context.NESTED);
+      LONG_CODER.encode(value.reserve, outStream, Context.NESTED);
+      LONG_CODER.encode(value.dateTime, outStream, Context.NESTED);
+      LONG_CODER.encode(value.expires, outStream, Context.NESTED);
+      LONG_CODER.encode(value.seller, outStream, Context.NESTED);
+      LONG_CODER.encode(value.category, outStream, Context.NESTED);
+      STRING_CODER.encode(value.extra, outStream, Context.NESTED);
+    }
+
+    @Override
+    public Auction decode(
+        InputStream inStream, Coder.Context context)
+        throws CoderException, IOException {
+      long id = LONG_CODER.decode(inStream, Context.NESTED);
+      String itemName = STRING_CODER.decode(inStream, Context.NESTED);
+      String description = STRING_CODER.decode(inStream, Context.NESTED);
+      long initialBid = LONG_CODER.decode(inStream, Context.NESTED);
+      long reserve = LONG_CODER.decode(inStream, Context.NESTED);
+      long dateTime = LONG_CODER.decode(inStream, Context.NESTED);
+      long expires = LONG_CODER.decode(inStream, Context.NESTED);
+      long seller = LONG_CODER.decode(inStream, Context.NESTED);
+      long category = LONG_CODER.decode(inStream, Context.NESTED);
+      String extra = STRING_CODER.decode(inStream, Context.NESTED);
+      return new Auction(
+          id, itemName, description, initialBid, reserve, dateTime, expires, seller, category,
+          extra);
+    }
+  };
+
+
+  /** Id of auction. */
+  @JsonProperty
+  public final long id; // primary key
+
+  /** Extra auction properties. */
+  @JsonProperty
+  public final String itemName;
+
+  @JsonProperty
+  public final String description;
+
+  /** Initial bid price, in cents. */
+  @JsonProperty
+  public final long initialBid;
+
+  /** Reserve price, in cents. */
+  @JsonProperty
+  public final long reserve;
+
+  @JsonProperty
+  public final long dateTime;
+
+  /** When does auction expire? (ms since epoch). Bids at or after this time are ignored. */
+  @JsonProperty
+  public final long expires;
+
+  /** Id of person who instigated auction. */
+  @JsonProperty
+  public final long seller; // foreign key: Person.id
+
+  /** Id of category auction is listed under. */
+  @JsonProperty
+  public final long category; // foreign key: Category.id
+
+  /** Additional arbitrary payload for performance testing. */
+  @JsonProperty
+  public final String extra;
+
+
+  // For Avro only.
+  @SuppressWarnings("unused")
+  private Auction() {
+    id = 0;
+    itemName = null;
+    description = null;
+    initialBid = 0;
+    reserve = 0;
+    dateTime = 0;
+    expires = 0;
+    seller = 0;
+    category = 0;
+    extra = null;
+  }
+
+  public Auction(long id, String itemName, String description, long initialBid, long reserve,
+      long dateTime, long expires, long seller, long category, String extra) {
+    this.id = id;
+    this.itemName = itemName;
+    this.description = description;
+    this.initialBid = initialBid;
+    this.reserve = reserve;
+    this.dateTime = dateTime;
+    this.expires = expires;
+    this.seller = seller;
+    this.category = category;
+    this.extra = extra;
+  }
+
+  /**
+   * Return a copy of auction which capture the given annotation.
+   * (Used for debugging).
+   */
+  public Auction withAnnotation(String annotation) {
+    return new Auction(id, itemName, description, initialBid, reserve, dateTime, expires, seller,
+        category, annotation + ": " + extra);
+  }
+
+  /**
+   * Does auction have {@code annotation}? (Used for debugging.)
+   */
+  public boolean hasAnnotation(String annotation) {
+    return extra.startsWith(annotation + ": ");
+  }
+
+  /**
+   * Remove {@code annotation} from auction. (Used for debugging.)
+   */
+  public Auction withoutAnnotation(String annotation) {
+    if (hasAnnotation(annotation)) {
+      return new Auction(id, itemName, description, initialBid, reserve, dateTime, expires, seller,
+          category, extra.substring(annotation.length() + 2));
+    } else {
+      return this;
+    }
+  }
+
+  @Override
+  public long sizeInBytes() {
+    return 8 + itemName.length() + 1 + description.length() + 1 + 8 + 8 + 8 + 8 + 8 + 8
+        + extra.length() + 1;
+  }
+
+  @Override
+  public String toString() {
+    try {
+      return NexmarkUtils.MAPPER.writeValueAsString(this);
+    } catch (JsonProcessingException e) {
+      throw new RuntimeException(e);
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/1f08970a/integration/java/src/main/java/org/apache/beam/integration/nexmark/AuctionBid.java
----------------------------------------------------------------------
diff --git a/integration/java/src/main/java/org/apache/beam/integration/nexmark/AuctionBid.java b/integration/java/src/main/java/org/apache/beam/integration/nexmark/AuctionBid.java
new file mode 100644
index 0000000..8c3697a
--- /dev/null
+++ b/integration/java/src/main/java/org/apache/beam/integration/nexmark/AuctionBid.java
@@ -0,0 +1,87 @@
+/*
+ * 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.beam.integration.nexmark;
+
+import org.apache.beam.sdk.coders.AtomicCoder;
+import org.apache.beam.sdk.coders.Coder;
+import org.apache.beam.sdk.coders.CoderException;
+
+import com.fasterxml.jackson.annotation.JsonProperty;
+import com.fasterxml.jackson.core.JsonProcessingException;
+
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
+import java.io.Serializable;
+
+/**
+ * Result of {@link WinningBids} transform.
+ */
+public class AuctionBid implements KnownSize, Serializable {
+  public static final Coder<AuctionBid> CODER = new AtomicCoder<AuctionBid>() {
+    @Override
+    public void encode(AuctionBid value, OutputStream outStream,
+        Coder.Context context)
+        throws CoderException, IOException {
+      Auction.CODER.encode(value.auction, outStream, Context.NESTED);
+      Bid.CODER.encode(value.bid, outStream, Context.NESTED);
+    }
+
+    @Override
+    public AuctionBid decode(
+        InputStream inStream, Coder.Context context)
+        throws CoderException, IOException {
+      Auction auction = Auction.CODER.decode(inStream, Context.NESTED);
+      Bid bid = Bid.CODER.decode(inStream, Context.NESTED);
+      return new AuctionBid(auction, bid);
+    }
+  };
+
+  @JsonProperty
+  public final Auction auction;
+
+  @JsonProperty
+  public final Bid bid;
+
+  // For Avro only.
+  @SuppressWarnings("unused")
+  private AuctionBid() {
+    auction = null;
+    bid = null;
+  }
+
+  public AuctionBid(Auction auction, Bid bid) {
+    this.auction = auction;
+    this.bid = bid;
+  }
+
+  @Override
+  public long sizeInBytes() {
+    return auction.sizeInBytes() + bid.sizeInBytes();
+  }
+
+  @Override
+  public String toString() {
+    try {
+      return NexmarkUtils.MAPPER.writeValueAsString(this);
+    } catch (JsonProcessingException e) {
+      throw new RuntimeException(e);
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/1f08970a/integration/java/src/main/java/org/apache/beam/integration/nexmark/AuctionCount.java
----------------------------------------------------------------------
diff --git a/integration/java/src/main/java/org/apache/beam/integration/nexmark/AuctionCount.java b/integration/java/src/main/java/org/apache/beam/integration/nexmark/AuctionCount.java
new file mode 100644
index 0000000..a0fbebc
--- /dev/null
+++ b/integration/java/src/main/java/org/apache/beam/integration/nexmark/AuctionCount.java
@@ -0,0 +1,90 @@
+/*
+ * 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.beam.integration.nexmark;
+
+import org.apache.beam.sdk.coders.AtomicCoder;
+import org.apache.beam.sdk.coders.Coder;
+import org.apache.beam.sdk.coders.CoderException;
+import org.apache.beam.sdk.coders.VarLongCoder;
+
+import com.fasterxml.jackson.annotation.JsonProperty;
+import com.fasterxml.jackson.core.JsonProcessingException;
+
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
+import java.io.Serializable;
+
+/**
+ * Result of {@link Query5}.
+ */
+public class AuctionCount implements KnownSize, Serializable {
+  private static final Coder<Long> LONG_CODER = VarLongCoder.of();
+
+  public static final Coder<AuctionCount> CODER = new AtomicCoder<AuctionCount>() {
+    @Override
+    public void encode(AuctionCount value, OutputStream outStream,
+        Coder.Context context)
+        throws CoderException, IOException {
+      LONG_CODER.encode(value.auction, outStream, Context.NESTED);
+      LONG_CODER.encode(value.count, outStream, Context.NESTED);
+    }
+
+    @Override
+    public AuctionCount decode(
+        InputStream inStream, Coder.Context context)
+        throws CoderException, IOException {
+      long auction = LONG_CODER.decode(inStream, Context.NESTED);
+      long count = LONG_CODER.decode(inStream, Context.NESTED);
+      return new AuctionCount(auction, count);
+    }
+  };
+
+  @JsonProperty
+  public final long auction;
+
+  @JsonProperty
+  public final long count;
+
+  // For Avro only.
+  @SuppressWarnings("unused")
+  private AuctionCount() {
+    auction = 0;
+    count = 0;
+  }
+
+  public AuctionCount(long auction, long count) {
+    this.auction = auction;
+    this.count = count;
+  }
+
+  @Override
+  public long sizeInBytes() {
+    return 8 + 8;
+  }
+
+  @Override
+  public String toString() {
+    try {
+      return NexmarkUtils.MAPPER.writeValueAsString(this);
+    } catch (JsonProcessingException e) {
+      throw new RuntimeException(e);
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/1f08970a/integration/java/src/main/java/org/apache/beam/integration/nexmark/AuctionPrice.java
----------------------------------------------------------------------
diff --git a/integration/java/src/main/java/org/apache/beam/integration/nexmark/AuctionPrice.java b/integration/java/src/main/java/org/apache/beam/integration/nexmark/AuctionPrice.java
new file mode 100644
index 0000000..4f25a9b
--- /dev/null
+++ b/integration/java/src/main/java/org/apache/beam/integration/nexmark/AuctionPrice.java
@@ -0,0 +1,91 @@
+/*
+ * 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.beam.integration.nexmark;
+
+import org.apache.beam.sdk.coders.AtomicCoder;
+import org.apache.beam.sdk.coders.Coder;
+import org.apache.beam.sdk.coders.CoderException;
+import org.apache.beam.sdk.coders.VarLongCoder;
+
+import com.fasterxml.jackson.annotation.JsonProperty;
+import com.fasterxml.jackson.core.JsonProcessingException;
+
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
+import java.io.Serializable;
+
+/**
+ * Result of {@link Query2}.
+ */
+public class AuctionPrice implements KnownSize, Serializable {
+  private static final Coder<Long> LONG_CODER = VarLongCoder.of();
+
+  public static final Coder<AuctionPrice> CODER = new AtomicCoder<AuctionPrice>() {
+    @Override
+    public void encode(AuctionPrice value, OutputStream outStream,
+        Coder.Context context)
+        throws CoderException, IOException {
+      LONG_CODER.encode(value.auction, outStream, Context.NESTED);
+      LONG_CODER.encode(value.price, outStream, Context.NESTED);
+    }
+
+    @Override
+    public AuctionPrice decode(
+        InputStream inStream, Coder.Context context)
+        throws CoderException, IOException {
+      long auction = LONG_CODER.decode(inStream, Context.NESTED);
+      long price = LONG_CODER.decode(inStream, Context.NESTED);
+      return new AuctionPrice(auction, price);
+    }
+  };
+
+  @JsonProperty
+  public final long auction;
+
+  /** Price in cents. */
+  @JsonProperty
+  public final long price;
+
+  // For Avro only.
+  @SuppressWarnings("unused")
+  private AuctionPrice() {
+    auction = 0;
+    price = 0;
+  }
+
+  public AuctionPrice(long auction, long price) {
+    this.auction = auction;
+    this.price = price;
+  }
+
+  @Override
+  public long sizeInBytes() {
+    return 8 + 8;
+  }
+
+  @Override
+  public String toString() {
+    try {
+      return NexmarkUtils.MAPPER.writeValueAsString(this);
+    } catch (JsonProcessingException e) {
+      throw new RuntimeException(e);
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/1f08970a/integration/java/src/main/java/org/apache/beam/integration/nexmark/BEAM_ON_FLINK_ON_GCP.md
----------------------------------------------------------------------
diff --git a/integration/java/src/main/java/org/apache/beam/integration/nexmark/BEAM_ON_FLINK_ON_GCP.md b/integration/java/src/main/java/org/apache/beam/integration/nexmark/BEAM_ON_FLINK_ON_GCP.md
new file mode 100644
index 0000000..d1b51e8
--- /dev/null
+++ b/integration/java/src/main/java/org/apache/beam/integration/nexmark/BEAM_ON_FLINK_ON_GCP.md
@@ -0,0 +1,282 @@
+<!--
+    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.
+-->
+
+# Running NexMark on Beam on Flink on Google Compute Platform
+
+Here's how to create a cluster of VMs on Google Compute Platform, deploy
+Flink to them, and invoke a NexMark pipeline using the Beam-on-Flink
+runner.
+
+These instructions are somewhat baroque and I hope they can be
+simplified over time.
+
+## Prerequisites
+
+You'll need:
+
+* the Google Cloud SDK
+* a clone of the Beam repository
+* a Flink binary distribution
+* a project on Google Compute Platform.
+
+## Establish the shell environment
+
+```
+# Beam root
+BEAM=<path to Beam source directory>
+# Flink root
+FLINK_VER=flink-1.0.3
+FLINK=<path to Flink distribution directory>
+# Google Cloud project
+PROJECT=<your project id>
+# Google Cloud zone
+ZONE=<your project zone>
+# Cloud commands
+GCLOUD=<path to gcloud command>
+GSUTIL=<path to gsutil command>
+```
+
+## Establish VM names for Flink master and workers
+
+```
+MASTER=flink-m
+NUM_WORKERS=5
+WORKERS=""
+for (( i = 0; i < $NUM_WORKERS; i++ )); do
+  WORKERS="$WORKERS flink-w-$i"
+done
+ALL="$MASTER $WORKERS"
+```
+
+## Build Beam
+
+```
+( cd $BEAM && mvn clean install )
+```
+
+## Bring up the cluster
+
+Establish project defaults and authenticate:
+```
+$GCLOUD init
+$GCLOUD auth login
+```
+
+Build Google Cloud Dataproc cluster:
+```
+$GCLOUD beta dataproc clusters create \
+  --project=$PROJECT \
+  --zone=$ZONE \
+  --bucket=nexmark \
+  --scopes=cloud-platform \
+  --num-workers=$NUM_WORKERS \
+  --image-version=preview \
+  flink
+```
+
+Force google_compute_engine ssh keys to be generated locally:
+```
+$GCLOUD compute ssh \
+  --project=$PROJECT \
+  --zone=$ZONE \
+  $MASTER \
+  --command 'exit'
+```
+
+Open ports on the VMs:
+```
+$GCLOUD compute firewall-rules create allow-monitoring --allow tcp:8080-8081
+$GCLOUD compute firewall-rules create allow-debug --allow tcp:5555
+```
+
+Establish keys on master and workers
+**CAUTION:** This will leave your private key on your master VM.
+Better would be to create a key just for inter-worker ssh.
+```
+for m in $ALL; do
+  echo "*** $m ***"
+  $GCLOUD beta compute scp \
+    --project=$PROJECT \
+    --zone=$ZONE \
+    ~/.ssh/google_compute_engine.pub $m:~/.ssh/
+done
+$GCLOUD beta compute scp \
+  --project=$PROJECT \
+  --zone=$ZONE \
+  ~/.ssh/google_compute_engine $MASTER:~/.ssh/
+```
+
+Collect IP addresses for workers:
+```
+MASTER_EXT_IP=$($GCLOUD compute instances describe \
+ --project=$PROJECT \
+  --zone=$ZONE \
+  $MASTER | grep natIP: | sed 's/[ ]*natIP:[ ]*//')
+MASTER_IP=$($GCLOUD compute instances describe \
+ --project=$PROJECT \
+  --zone=$ZONE \
+  $MASTER | grep networkIP: | sed 's/[ ]*networkIP:[ ]*//')
+WORKER_IPS=""
+for m in $WORKERS; do
+  echo "*** $m ***"
+  WORKER_IP=$($GCLOUD compute instances describe \
+    --project=$PROJECT \
+    --zone=$ZONE \
+    $m | grep networkIP: | sed 's/[ ]*networkIP:[ ]*//')
+  WORKER_IPS="$WORKER_IPS $WORKER_IP"
+done
+```
+
+Configure Flink:
+```
+cat $FLINK/conf/flink-conf.yaml \
+  | sed "s|.*\(jobmanager.rpc.address\):.*|\1: $MASTER_IP|g" \
+  | sed "s|.*\(jobmanager.heap.mb\):.*|\1: 4096|g" \
+  | sed "s|.*\(taskmanager.heap.mb\):.*|\1: 8192|g" \
+  | sed "s|.*\(parallelism.default\):.*|\1: $(($NUM_WORKERS * 4))|g" \
+  | sed "s|.*\(fs.hdfs.hadoopconf\):.*|\1: /etc/hadoop/conf|g" \
+  | sed "s|.*\(taskmanager.numberOfTaskSlots\):.*|\1: 4|g" \
+  | sed "s|.*\(jobmanager.web.submit.enable\):.*|\1: false|g" \
+  | sed "s|.*\(env.ssh.opts\):.*||g" \
+  > ~/flink-conf.yaml
+cat $FLINK/conf/log4j.properties \
+  | sed "s|.*\(log4j.rootLogger\)=.*|\1=ERROR, file|g" \
+  > ~/log4j.properties
+echo "env.ssh.opts: -i /home/$USER/.ssh/google_compute_engine -o StrictHostKeyChecking=no" >> ~/flink-conf.yaml
+echo "$MASTER_IP:8081" > ~/masters
+echo -n > ~/slaves
+for ip in $WORKER_IPS; do
+  echo $ip >> ~/slaves
+done
+cp -f \
+  ~/flink-conf.yaml \
+  ~/masters ~/slaves \
+  ~/log4j.properties \
+  $FLINK/conf/
+```
+
+Package configured Flink for distribution to workers:
+```
+( cd ~/ && tar -cvzf ~/flink.tgz $FLINK/* )
+```
+
+Distribute:
+```
+$GSUTIL cp ~/flink.tgz gs://nexmark
+for m in $ALL; do
+  echo "*** $m ***"
+  $GCLOUD compute ssh \
+    --project=$PROJECT \
+    --zone=$ZONE \
+    $m \
+    --command 'gsutil cp gs://nexmark/flink.tgz ~/ && tar -xvzf ~/flink.tgz'
+done
+```
+
+Start the Flink cluster:
+```
+$GCLOUD compute ssh \
+  --project=$PROJECT \
+  --zone=$ZONE \
+  $MASTER \
+  --command "~/$FLINK_VER/bin/start-cluster.sh"
+```
+
+Bring up the Flink monitoring UI:
+```
+/usr/bin/google-chrome $MASTER_EXT_IP:8081 &
+```
+
+## Run NexMark
+
+Distribute the Beam + NexMark jar to all workers:
+```
+$GSUTIL cp $BEAM/integration/java/target/java-integration-all-bundled-0.2.0-incubating-SNAPSHOT.jar gs://nexmark
+for m in $ALL; do
+  echo "*** $m ***"
+  $GCLOUD compute ssh \
+    --project=$PROJECT \
+    --zone=$ZONE \
+    $m \
+    --command "gsutil cp gs://nexmark/java-integration-all-bundled-0.2.0-incubating-SNAPSHOT.jar ~/$FLINK_VER/lib/"
+done
+```
+
+Create a Pubsub topic and subscription for testing:
+```
+$GCLOUD alpha pubsub \
+  --project=$PROJECT \
+  topics create flink_test
+
+$GCLOUD alpha pubsub \
+  --project=$PROJECT \
+  subscriptions create flink_test \
+  --topic flink_test \
+  --ack-deadline=60 \
+  --topic-project=$PROJECT
+```
+
+Launch!
+**NOTE:** As of flink-1.0.3 this will throw a `NullPointerException`
+in `org.apache.beam.sdk.io.PubsubUnboundedSink$WriterFn.startBundle`.
+See Jira issue [BEAM-196](https://issues.apache.org/jira/browse/BEAM-196).
+
+```
+$GCLOUD compute ssh \
+  --project=$PROJECT \
+  --zone=$ZONE \
+  $MASTER \
+  --command "~/$FLINK_VER/bin/flink run \
+  -c org.apache.beam.integration.nexmark.NexmarkFlinkDriver \
+  ~/$FLINK_VER/lib/java-integration-all-bundled-0.2.0-incubating-SNAPSHOT.jar \
+  --project=$PROJECT \
+  --streaming=true \
+  --query=0 \
+  --sourceType=PUBSUB \
+  --pubSubMode=COMBINED \
+  --pubsubTopic=flink_test \
+  --resourceNameMode=VERBATIM \
+  --manageResources=false \
+  --monitorJobs=false \
+  --numEventGenerators=5 \
+  --firstEventRate=1000 \
+  --nextEventRate=1000 \
+  --isRateLimited=true \
+  --numEvents=0 \
+  --useWallclockEventTime=true \
+  --usePubsubPublishTime=true"
+```
+
+## Teardown the cluster
+
+Stop the Flink cluster:
+```
+$GCLOUD compute ssh \
+  --project=$PROJECT \
+  --zone=$ZONE \
+  $MASTER \
+  --command "~/$FLINK_VER/bin/stop-cluster.sh"
+```
+
+Teardown the Dataproc cluster:
+```
+$GCLOUD beta dataproc clusters delete \
+  --project=$PROJECT \
+  flink
+```

http://git-wip-us.apache.org/repos/asf/beam/blob/1f08970a/integration/java/src/main/java/org/apache/beam/integration/nexmark/Bid.java
----------------------------------------------------------------------
diff --git a/integration/java/src/main/java/org/apache/beam/integration/nexmark/Bid.java b/integration/java/src/main/java/org/apache/beam/integration/nexmark/Bid.java
new file mode 100644
index 0000000..ce2184b
--- /dev/null
+++ b/integration/java/src/main/java/org/apache/beam/integration/nexmark/Bid.java
@@ -0,0 +1,178 @@
+/*
+ * 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.beam.integration.nexmark;
+
+import org.apache.beam.sdk.coders.AtomicCoder;
+import org.apache.beam.sdk.coders.Coder;
+import org.apache.beam.sdk.coders.CoderException;
+import org.apache.beam.sdk.coders.StringUtf8Coder;
+import org.apache.beam.sdk.coders.VarLongCoder;
+
+import com.fasterxml.jackson.annotation.JsonProperty;
+import com.fasterxml.jackson.core.JsonProcessingException;
+
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
+import java.io.Serializable;
+import java.util.Comparator;
+
+/**
+ * A bid for an item on auction.
+ */
+public class Bid implements KnownSize, Serializable {
+  private static final Coder<Long> LONG_CODER = VarLongCoder.of();
+  private static final Coder<String> STRING_CODER = StringUtf8Coder.of();
+
+  public static final Coder<Bid> CODER = new AtomicCoder<Bid>() {
+    @Override
+    public void encode(Bid value, OutputStream outStream,
+        Coder.Context context)
+        throws CoderException, IOException {
+      LONG_CODER.encode(value.auction, outStream, Context.NESTED);
+      LONG_CODER.encode(value.bidder, outStream, Context.NESTED);
+      LONG_CODER.encode(value.price, outStream, Context.NESTED);
+      LONG_CODER.encode(value.dateTime, outStream, Context.NESTED);
+      STRING_CODER.encode(value.extra, outStream, Context.NESTED);
+    }
+
+    @Override
+    public Bid decode(
+        InputStream inStream, Coder.Context context)
+        throws CoderException, IOException {
+      long auction = LONG_CODER.decode(inStream, Context.NESTED);
+      long bidder = LONG_CODER.decode(inStream, Context.NESTED);
+      long price = LONG_CODER.decode(inStream, Context.NESTED);
+      long dateTime = LONG_CODER.decode(inStream, Context.NESTED);
+      String extra = STRING_CODER.decode(inStream, Context.NESTED);
+      return new Bid(auction, bidder, price, dateTime, extra);
+    }
+  };
+
+  /**
+   * Comparator to order bids by ascending price then descending time
+   * (for finding winning bids).
+   */
+  public static final Comparator<Bid> PRICE_THEN_DESCENDING_TIME = new Comparator<Bid>() {
+    @Override
+    public int compare(Bid left, Bid right) {
+      int i = Double.compare(left.price, right.price);
+      if (i != 0) {
+        return i;
+      }
+      return Long.compare(right.dateTime, left.dateTime);
+    }
+  };
+
+  /**
+   * Comparator to order bids by ascending time then ascending price.
+   * (for finding most recent bids).
+   */
+  public static final Comparator<Bid> ASCENDING_TIME_THEN_PRICE = new Comparator<Bid>() {
+    @Override
+    public int compare(Bid left, Bid right) {
+      int i = Long.compare(left.dateTime, right.dateTime);
+      if (i != 0) {
+        return i;
+      }
+      return Double.compare(left.price, right.price);
+    }
+  };
+
+  /** Id of auction this bid is for. */
+  @JsonProperty
+  public final long auction; // foreign key: Auction.id
+
+  /** Id of person bidding in auction. */
+  @JsonProperty
+  public final long bidder; // foreign key: Person.id
+
+  /** Price of bid, in cents. */
+  @JsonProperty
+  public final long price;
+
+  /**
+   * Instant at which bid was made (ms since epoch).
+   * NOTE: This may be earlier than the system's event time.
+   */
+  @JsonProperty
+  public final long dateTime;
+
+  /** Additional arbitrary payload for performance testing. */
+  @JsonProperty
+  public final String extra;
+
+  // For Avro only.
+  @SuppressWarnings("unused")
+  private Bid() {
+    auction = 0;
+    bidder = 0;
+    price = 0;
+    dateTime = 0;
+    extra = null;
+  }
+
+  public Bid(long auction, long bidder, long price, long dateTime, String extra) {
+    this.auction = auction;
+    this.bidder = bidder;
+    this.price = price;
+    this.dateTime = dateTime;
+    this.extra = extra;
+  }
+
+  /**
+   * Return a copy of bid which capture the given annotation.
+   * (Used for debugging).
+   */
+  public Bid withAnnotation(String annotation) {
+    return new Bid(auction, bidder, price, dateTime, annotation + ": " + extra);
+  }
+
+  /**
+   * Does bid have {@code annotation}? (Used for debugging.)
+   */
+  public boolean hasAnnotation(String annotation) {
+    return extra.startsWith(annotation + ": ");
+  }
+
+  /**
+   * Remove {@code annotation} from bid. (Used for debugging.)
+   */
+  public Bid withoutAnnotation(String annotation) {
+    if (hasAnnotation(annotation)) {
+      return new Bid(auction, bidder, price, dateTime, extra.substring(annotation.length() + 2));
+    } else {
+      return this;
+    }
+  }
+
+  @Override
+  public long sizeInBytes() {
+    return 8 + 8 + 8 + 8 + extra.length() + 1;
+  }
+
+  @Override
+  public String toString() {
+    try {
+      return NexmarkUtils.MAPPER.writeValueAsString(this);
+    } catch (JsonProcessingException e) {
+      throw new RuntimeException(e);
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/1f08970a/integration/java/src/main/java/org/apache/beam/integration/nexmark/BidsPerSession.java
----------------------------------------------------------------------
diff --git a/integration/java/src/main/java/org/apache/beam/integration/nexmark/BidsPerSession.java b/integration/java/src/main/java/org/apache/beam/integration/nexmark/BidsPerSession.java
new file mode 100644
index 0000000..cfdd170
--- /dev/null
+++ b/integration/java/src/main/java/org/apache/beam/integration/nexmark/BidsPerSession.java
@@ -0,0 +1,89 @@
+/*
+ * 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.beam.integration.nexmark;
+
+import org.apache.beam.sdk.coders.AtomicCoder;
+import org.apache.beam.sdk.coders.Coder;
+import org.apache.beam.sdk.coders.CoderException;
+import org.apache.beam.sdk.coders.VarLongCoder;
+
+import com.fasterxml.jackson.annotation.JsonProperty;
+import com.fasterxml.jackson.core.JsonProcessingException;
+
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
+import java.io.Serializable;
+
+/**
+ * Result of query 11.
+ */
+public class BidsPerSession implements KnownSize, Serializable {
+  private static final Coder<Long> LONG_CODER = VarLongCoder.of();
+
+  public static final Coder<BidsPerSession> CODER = new AtomicCoder<BidsPerSession>() {
+    @Override
+    public void encode(BidsPerSession value, OutputStream outStream,
+        Coder.Context context)
+        throws CoderException, IOException {
+      LONG_CODER.encode(value.personId, outStream, Context.NESTED);
+      LONG_CODER.encode(value.bidsPerSession, outStream, Context.NESTED);
+    }
+
+    @Override
+    public BidsPerSession decode(
+        InputStream inStream, Coder.Context context)
+        throws CoderException, IOException {
+      long personId = LONG_CODER.decode(inStream, Context.NESTED);
+      long bidsPerSession = LONG_CODER.decode(inStream, Context.NESTED);
+      return new BidsPerSession(personId, bidsPerSession);
+    }
+  };
+
+  @JsonProperty
+  public final long personId;
+
+  @JsonProperty
+  public final long bidsPerSession;
+
+  public BidsPerSession() {
+    personId = 0;
+    bidsPerSession = 0;
+  }
+
+  public BidsPerSession(long personId, long bidsPerSession) {
+    this.personId = personId;
+    this.bidsPerSession = bidsPerSession;
+  }
+
+  @Override
+  public long sizeInBytes() {
+    // Two longs.
+    return 8 + 8;
+  }
+
+  @Override
+  public String toString() {
+    try {
+      return NexmarkUtils.MAPPER.writeValueAsString(this);
+    } catch (JsonProcessingException e) {
+      throw new RuntimeException(e);
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/1f08970a/integration/java/src/main/java/org/apache/beam/integration/nexmark/BoundedEventSource.java
----------------------------------------------------------------------
diff --git a/integration/java/src/main/java/org/apache/beam/integration/nexmark/BoundedEventSource.java b/integration/java/src/main/java/org/apache/beam/integration/nexmark/BoundedEventSource.java
new file mode 100644
index 0000000..f6cc16a
--- /dev/null
+++ b/integration/java/src/main/java/org/apache/beam/integration/nexmark/BoundedEventSource.java
@@ -0,0 +1,197 @@
+/*
+ * 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.beam.integration.nexmark;
+
+import org.apache.beam.sdk.coders.Coder;
+import org.apache.beam.sdk.io.BoundedSource;
+import org.apache.beam.sdk.options.PipelineOptions;
+import org.apache.beam.sdk.values.TimestampedValue;
+
+import org.joda.time.Instant;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.NoSuchElementException;
+
+import javax.annotation.Nullable;
+
+/**
+ * A custom, bounded source of event records.
+ */
+class BoundedEventSource extends BoundedSource<Event> {
+  /** Configuration we generate events against. */
+  private final GeneratorConfig config;
+
+  /** How many bounded sources to create. */
+  private final int numEventGenerators;
+
+  public BoundedEventSource(GeneratorConfig config, int numEventGenerators) {
+    this.config = config;
+    this.numEventGenerators = numEventGenerators;
+  }
+
+  /** A reader to pull events from the generator. */
+  private static class EventReader extends BoundedReader<Event> {
+    /**
+     * Event source we purporting to be reading from.
+     * (We can't use Java's capture-outer-class pointer since we must update
+     * this field on calls to splitAtFraction.)
+     */
+    private BoundedEventSource source;
+
+    /** Generator we are reading from. */
+    private final Generator generator;
+
+    private boolean reportedStop;
+
+    @Nullable
+    private TimestampedValue<Event> currentEvent;
+
+    public EventReader(BoundedEventSource source, GeneratorConfig config) {
+      this.source = source;
+      generator = new Generator(config);
+      reportedStop = false;
+    }
+
+    @Override
+    public synchronized boolean start() {
+      NexmarkUtils.info("starting bounded generator %s", generator);
+      return advance();
+    }
+
+    @Override
+    public synchronized boolean advance() {
+      if (!generator.hasNext()) {
+        // No more events.
+        if (!reportedStop) {
+          reportedStop = true;
+          NexmarkUtils.info("stopped bounded generator %s", generator);
+        }
+        return false;
+      }
+      currentEvent = generator.next();
+      return true;
+    }
+
+    @Override
+    public synchronized Event getCurrent() throws NoSuchElementException {
+      if (currentEvent == null) {
+        throw new NoSuchElementException();
+      }
+      return currentEvent.getValue();
+    }
+
+    @Override
+    public synchronized Instant getCurrentTimestamp() throws NoSuchElementException {
+      if (currentEvent == null) {
+        throw new NoSuchElementException();
+      }
+      return currentEvent.getTimestamp();
+    }
+
+    @Override
+    public void close() throws IOException {
+      // Nothing to close.
+    }
+
+    @Override
+    public synchronized Double getFractionConsumed() {
+      return generator.getFractionConsumed();
+    }
+
+    @Override
+    public synchronized BoundedSource<Event> getCurrentSource() {
+      return source;
+    }
+
+    @Override
+    @Nullable
+    public synchronized BoundedEventSource splitAtFraction(double fraction) {
+      long startId = generator.getCurrentConfig().getStartEventId();
+      long stopId = generator.getCurrentConfig().getStopEventId();
+      long size = stopId - startId;
+      long splitEventId = startId + Math.min((int) (size * fraction), size);
+      if (splitEventId <= generator.getNextEventId() || splitEventId == stopId) {
+        // Already passed this position or split results in left or right being empty.
+        NexmarkUtils.info("split failed for bounded generator %s at %f", generator, fraction);
+        return null;
+      }
+
+      NexmarkUtils.info("about to split bounded generator %s at %d", generator, splitEventId);
+
+      // Scale back the event space of the current generator, and return a generator config
+      // representing the event space we just 'stole' from the current generator.
+      GeneratorConfig remainingConfig = generator.splitAtEventId(splitEventId);
+
+      NexmarkUtils.info("split bounded generator into %s and %s", generator, remainingConfig);
+
+      // At this point
+      //   generator.events() ++ new Generator(remainingConfig).events()
+      //   == originalGenerator.events()
+
+      // We need a new source to represent the now smaller key space for this reader, so
+      // that we can maintain the invariant that
+      //   this.getCurrentSource().createReader(...)
+      // will yield the same output as this.
+      source = new BoundedEventSource(generator.getCurrentConfig(), source.numEventGenerators);
+
+      // Return a source from which we may read the 'stolen' event space.
+      return new BoundedEventSource(remainingConfig, source.numEventGenerators);
+    }
+  }
+
+  @Override
+  public List<BoundedEventSource> splitIntoBundles(
+      long desiredBundleSizeBytes, PipelineOptions options) {
+    NexmarkUtils.info("slitting bounded source %s into %d sub-sources", config, numEventGenerators);
+    List<BoundedEventSource> results = new ArrayList<>();
+    // Ignore desiredBundleSizeBytes and use numEventGenerators instead.
+    for (GeneratorConfig subConfig : config.split(numEventGenerators)) {
+      results.add(new BoundedEventSource(subConfig, 1));
+    }
+    return results;
+  }
+
+  @Override
+  public long getEstimatedSizeBytes(PipelineOptions options) {
+    return config.getEstimatedSizeBytes();
+  }
+
+  @Override
+  public boolean producesSortedKeys(PipelineOptions options) {
+    return false;
+  }
+
+  @Override
+  public EventReader createReader(PipelineOptions options) {
+    NexmarkUtils.info("creating initial bounded reader for %s", config);
+    return new EventReader(this, config);
+  }
+
+  @Override
+  public void validate() {
+    // Nothing to validate.
+  }
+
+  @Override
+  public Coder<Event> getDefaultOutputCoder() {
+    return Event.CODER;
+  }
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/1f08970a/integration/java/src/main/java/org/apache/beam/integration/nexmark/CategoryPrice.java
----------------------------------------------------------------------
diff --git a/integration/java/src/main/java/org/apache/beam/integration/nexmark/CategoryPrice.java b/integration/java/src/main/java/org/apache/beam/integration/nexmark/CategoryPrice.java
new file mode 100644
index 0000000..ab5d92d
--- /dev/null
+++ b/integration/java/src/main/java/org/apache/beam/integration/nexmark/CategoryPrice.java
@@ -0,0 +1,100 @@
+/*
+ * 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.beam.integration.nexmark;
+
+import org.apache.beam.sdk.coders.AtomicCoder;
+import org.apache.beam.sdk.coders.Coder;
+import org.apache.beam.sdk.coders.CoderException;
+import org.apache.beam.sdk.coders.VarIntCoder;
+import org.apache.beam.sdk.coders.VarLongCoder;
+
+import com.fasterxml.jackson.annotation.JsonProperty;
+import com.fasterxml.jackson.core.JsonProcessingException;
+
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
+import java.io.Serializable;
+
+/**
+ * Result of {@link Query4}.
+ */
+public class CategoryPrice implements KnownSize, Serializable {
+  private static final Coder<Long> LONG_CODER = VarLongCoder.of();
+  private static final Coder<Integer> INT_CODER = VarIntCoder.of();
+
+  public static final Coder<CategoryPrice> CODER = new AtomicCoder<CategoryPrice>() {
+    @Override
+    public void encode(CategoryPrice value, OutputStream outStream,
+        Coder.Context context)
+        throws CoderException, IOException {
+      LONG_CODER.encode(value.category, outStream, Context.NESTED);
+      LONG_CODER.encode(value.price, outStream, Context.NESTED);
+      INT_CODER.encode(value.isLast ? 1 : 0, outStream, Context.NESTED);
+    }
+
+    @Override
+    public CategoryPrice decode(
+        InputStream inStream, Coder.Context context)
+        throws CoderException, IOException {
+      long category = LONG_CODER.decode(inStream, Context.NESTED);
+      long price = LONG_CODER.decode(inStream, Context.NESTED);
+      boolean isLast = INT_CODER.decode(inStream, context) != 0;
+      return new CategoryPrice(category, price, isLast);
+    }
+  };
+
+  @JsonProperty
+  public final long category;
+
+  /** Price in cents. */
+  @JsonProperty
+  public final long price;
+
+  @JsonProperty
+  public final boolean isLast;
+
+  // For Avro only.
+  @SuppressWarnings("unused")
+  private CategoryPrice() {
+    category = 0;
+    price = 0;
+    isLast = false;
+  }
+
+  public CategoryPrice(long category, long price, boolean isLast) {
+    this.category = category;
+    this.price = price;
+    this.isLast = isLast;
+  }
+
+  @Override
+  public long sizeInBytes() {
+    return 8 + 8 + 1;
+  }
+
+  @Override
+  public String toString() {
+    try {
+      return NexmarkUtils.MAPPER.writeValueAsString(this);
+    } catch (JsonProcessingException e) {
+      throw new RuntimeException(e);
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/1f08970a/integration/java/src/main/java/org/apache/beam/integration/nexmark/Done.java
----------------------------------------------------------------------
diff --git a/integration/java/src/main/java/org/apache/beam/integration/nexmark/Done.java b/integration/java/src/main/java/org/apache/beam/integration/nexmark/Done.java
new file mode 100644
index 0000000..659da44
--- /dev/null
+++ b/integration/java/src/main/java/org/apache/beam/integration/nexmark/Done.java
@@ -0,0 +1,83 @@
+/*
+ * 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.beam.integration.nexmark;
+
+import org.apache.beam.sdk.coders.AtomicCoder;
+import org.apache.beam.sdk.coders.Coder;
+import org.apache.beam.sdk.coders.CoderException;
+import org.apache.beam.sdk.coders.StringUtf8Coder;
+
+import com.fasterxml.jackson.annotation.JsonProperty;
+import com.fasterxml.jackson.core.JsonProcessingException;
+
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
+import java.io.Serializable;
+
+/**
+ * Result of query 10.
+ */
+public class Done implements KnownSize, Serializable {
+  private static final Coder<String> STRING_CODER = StringUtf8Coder.of();
+
+  public static final Coder<Done> CODER = new AtomicCoder<Done>() {
+    @Override
+    public void encode(Done value, OutputStream outStream,
+        Coder.Context context)
+        throws CoderException, IOException {
+      STRING_CODER.encode(value.message, outStream, Context.NESTED);
+    }
+
+    @Override
+    public Done decode(
+        InputStream inStream, Coder.Context context)
+        throws CoderException, IOException {
+      String message = STRING_CODER.decode(inStream, Context.NESTED);
+      return new Done(message);
+    }
+  };
+
+  @JsonProperty
+  public final String message;
+
+  // For Avro only.
+  @SuppressWarnings("unused")
+  public Done() {
+    message = null;
+  }
+
+  public Done(String message) {
+    this.message = message;
+  }
+
+  @Override
+  public long sizeInBytes() {
+    return message.length();
+  }
+
+  @Override
+  public String toString() {
+    try {
+      return NexmarkUtils.MAPPER.writeValueAsString(this);
+    } catch (JsonProcessingException e) {
+      throw new RuntimeException(e);
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/1f08970a/integration/java/src/main/java/org/apache/beam/integration/nexmark/Event.java
----------------------------------------------------------------------
diff --git a/integration/java/src/main/java/org/apache/beam/integration/nexmark/Event.java b/integration/java/src/main/java/org/apache/beam/integration/nexmark/Event.java
new file mode 100644
index 0000000..a382b8e
--- /dev/null
+++ b/integration/java/src/main/java/org/apache/beam/integration/nexmark/Event.java
@@ -0,0 +1,181 @@
+/*
+ * 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.beam.integration.nexmark;
+
+import org.apache.beam.sdk.coders.AtomicCoder;
+import org.apache.beam.sdk.coders.Coder;
+import org.apache.beam.sdk.coders.CoderException;
+import org.apache.beam.sdk.coders.VarIntCoder;
+
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
+import java.io.Serializable;
+
+import javax.annotation.Nullable;
+
+/**
+ * An event in the auction system, either a (new) {@link Person}, a (new) {@link Auction},
+ * or a {@link Bid}.
+ */
+public class Event implements KnownSize, Serializable {
+  private static final Coder<Integer> INT_CODER = VarIntCoder.of();
+
+  public static final Coder<Event> CODER = new AtomicCoder<Event>() {
+    @Override
+    public void encode(Event value, OutputStream outStream, Coder.Context context)
+        throws CoderException, IOException {
+      if (value.newPerson != null) {
+        INT_CODER.encode(0, outStream, Context.NESTED);
+        Person.CODER.encode(value.newPerson, outStream, Context.NESTED);
+      } else if (value.newAuction != null) {
+        INT_CODER.encode(1, outStream, Context.NESTED);
+        Auction.CODER.encode(value.newAuction, outStream, Context.NESTED);
+      } else if (value.bid != null) {
+        INT_CODER.encode(2, outStream, Context.NESTED);
+        Bid.CODER.encode(value.bid, outStream, Context.NESTED);
+      } else {
+        throw new RuntimeException("invalid event");
+      }
+    }
+
+    @Override
+    public Event decode(
+        InputStream inStream, Coder.Context context)
+        throws CoderException, IOException {
+      int tag = INT_CODER.decode(inStream, context);
+      if (tag == 0) {
+        Person person = Person.CODER.decode(inStream, Context.NESTED);
+        return new Event(person);
+      } else if (tag == 1) {
+        Auction auction = Auction.CODER.decode(inStream, Context.NESTED);
+        return new Event(auction);
+      } else if (tag == 2) {
+        Bid bid = Bid.CODER.decode(inStream, Context.NESTED);
+        return new Event(bid);
+      } else {
+        throw new RuntimeException("invalid event encoding");
+      }
+    }
+  };
+
+  @Nullable
+  @org.apache.avro.reflect.Nullable
+  public final Person newPerson;
+
+  @Nullable
+  @org.apache.avro.reflect.Nullable
+  public final Auction newAuction;
+
+  @Nullable
+  @org.apache.avro.reflect.Nullable
+  public final Bid bid;
+
+  // For Avro only.
+  @SuppressWarnings("unused")
+  private Event() {
+    newPerson = null;
+    newAuction = null;
+    bid = null;
+  }
+
+  public Event(Person newPerson) {
+    this.newPerson = newPerson;
+    newAuction = null;
+    bid = null;
+  }
+
+  public Event(Auction newAuction) {
+    newPerson = null;
+    this.newAuction = newAuction;
+    bid = null;
+  }
+
+  public Event(Bid bid) {
+    newPerson = null;
+    newAuction = null;
+    this.bid = bid;
+  }
+
+  /**
+   * Return a copy of event which captures {@code annotation}.
+   * (Used for debugging).
+   */
+  public Event withAnnotation(String annotation) {
+    if (newPerson != null) {
+      return new Event(newPerson.withAnnotation(annotation));
+    } else if (newAuction != null) {
+      return new Event(newAuction.withAnnotation(annotation));
+    } else {
+      return new Event(bid.withAnnotation(annotation));
+    }
+  }
+
+  /**
+   * Does event have {@code annotation}? (Used for debugging.)
+   */
+  public boolean hasAnnotation(String annotation) {
+    if (newPerson != null) {
+      return newPerson.hasAnnotation(annotation);
+    } else if (newAuction != null) {
+      return newAuction.hasAnnotation(annotation);
+    } else {
+      return bid.hasAnnotation(annotation);
+    }
+  }
+
+  /**
+   * Remove {@code annotation} from event. (Used for debugging.)
+   */
+  public Event withoutAnnotation(String annotation) {
+    if (newPerson != null) {
+      return new Event(newPerson.withoutAnnotation(annotation));
+    } else if (newAuction != null) {
+      return new Event(newAuction.withoutAnnotation(annotation));
+    } else {
+      return new Event(bid.withoutAnnotation(annotation));
+    }
+  }
+
+  @Override
+  public long sizeInBytes() {
+    if (newPerson != null) {
+      return 1 + newPerson.sizeInBytes();
+    } else if (newAuction != null) {
+      return 1 + newAuction.sizeInBytes();
+    } else if (bid != null) {
+      return 1 + bid.sizeInBytes();
+    } else {
+      throw new RuntimeException("invalid event");
+    }
+  }
+
+  @Override
+  public String toString() {
+    if (newPerson != null) {
+      return newPerson.toString();
+    } else if (newAuction != null) {
+      return newAuction.toString();
+    } else if (bid != null) {
+      return bid.toString();
+    } else {
+      throw new RuntimeException("invalid event");
+    }
+  }
+}


[32/55] [abbrv] beam git commit: Disable use of GcsIOChannelFactory on query10

Posted by ie...@apache.org.
Disable use of GcsIOChannelFactory on query10

Make NexmarkOptions depend on GcpOptions instead of PubsubOptions

issue #21


Project: http://git-wip-us.apache.org/repos/asf/beam/repo
Commit: http://git-wip-us.apache.org/repos/asf/beam/commit/902050b0
Tree: http://git-wip-us.apache.org/repos/asf/beam/tree/902050b0
Diff: http://git-wip-us.apache.org/repos/asf/beam/diff/902050b0

Branch: refs/heads/master
Commit: 902050b0b276e22ab002e8efb390bbaa01e18e99
Parents: 1541fad
Author: Ismaël Mejía <ie...@apache.org>
Authored: Fri May 5 10:47:46 2017 +0200
Committer: Ismaël Mejía <ie...@gmail.com>
Committed: Wed Aug 23 19:07:28 2017 +0200

----------------------------------------------------------------------
 integration/java/nexmark/pom.xml                       |  5 -----
 .../beam/integration/nexmark/NexmarkOptions.java       |  7 ++++++-
 .../beam/integration/nexmark/queries/Query10.java      | 13 +++++++------
 3 files changed, 13 insertions(+), 12 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/beam/blob/902050b0/integration/java/nexmark/pom.xml
----------------------------------------------------------------------
diff --git a/integration/java/nexmark/pom.xml b/integration/java/nexmark/pom.xml
index 8a65c0f..35fe0f3 100644
--- a/integration/java/nexmark/pom.xml
+++ b/integration/java/nexmark/pom.xml
@@ -218,11 +218,6 @@
       <artifactId>google-api-services-bigquery</artifactId>
     </dependency>
 
-    <dependency>
-      <groupId>com.google.cloud.bigdataoss</groupId>
-      <artifactId>gcsio</artifactId>
-    </dependency>
-
     <!-- Extra libraries -->
     <dependency>
       <groupId>com.fasterxml.jackson.core</groupId>

http://git-wip-us.apache.org/repos/asf/beam/blob/902050b0/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkOptions.java
----------------------------------------------------------------------
diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkOptions.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkOptions.java
index e1c1af2..f162fd6 100644
--- a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkOptions.java
+++ b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkOptions.java
@@ -18,14 +18,19 @@
 package org.apache.beam.integration.nexmark;
 
 import javax.annotation.Nullable;
+import org.apache.beam.sdk.extensions.gcp.options.GcpOptions;
 import org.apache.beam.sdk.io.gcp.pubsub.PubsubOptions;
+import org.apache.beam.sdk.options.ApplicationNameOptions;
 import org.apache.beam.sdk.options.Default;
 import org.apache.beam.sdk.options.Description;
+import org.apache.beam.sdk.options.PipelineOptions;
+import org.apache.beam.sdk.options.StreamingOptions;
 
 /**
  * Command line flags.
  */
-public interface NexmarkOptions extends PubsubOptions {
+public interface NexmarkOptions
+    extends ApplicationNameOptions, GcpOptions, PipelineOptions, StreamingOptions {
   @Description("Which suite to run. Default is to use command line arguments for one job.")
   @Default.Enum("DEFAULT")
   NexmarkSuite getSuite();

http://git-wip-us.apache.org/repos/asf/beam/blob/902050b0/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query10.java
----------------------------------------------------------------------
diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query10.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query10.java
index 516dab1..c868666 100644
--- a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query10.java
+++ b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query10.java
@@ -50,7 +50,6 @@ import org.apache.beam.sdk.transforms.windowing.PaneInfo;
 import org.apache.beam.sdk.transforms.windowing.PaneInfo.Timing;
 import org.apache.beam.sdk.transforms.windowing.Repeatedly;
 import org.apache.beam.sdk.transforms.windowing.Window;
-import org.apache.beam.sdk.util.GcsIOChannelFactory;
 import org.apache.beam.sdk.values.KV;
 import org.apache.beam.sdk.values.PCollection;
 import org.joda.time.Duration;
@@ -133,11 +132,13 @@ public class Query10 extends NexmarkQuery {
    */
   private WritableByteChannel openWritableGcsFile(GcsOptions options, String filename)
       throws IOException {
-    WritableByteChannel channel =
-            GcsIOChannelFactory.fromOptions(options).create(filename, "text/plain");
-    checkState(channel instanceof GoogleCloudStorageWriteChannel);
-    ((GoogleCloudStorageWriteChannel) channel).setUploadBufferSize(CHANNEL_BUFFER);
-    return channel;
+    //TODO Decide what to do about this one
+//    WritableByteChannel channel =
+//            GcsIOChannelFactory.fromOptions(options).create(filename, "text/plain");
+//    checkState(channel instanceof GoogleCloudStorageWriteChannel);
+//    ((GoogleCloudStorageWriteChannel) channel).setUploadBufferSize(CHANNEL_BUFFER);
+//    return channel;
+    throw new UnsupportedOperationException("Disabled after removal of GcsIOChannelFactory");
   }
 
   /** Return a short string to describe {@code timing}. */


[08/55] [abbrv] beam git commit: Refactor classes into packages

Posted by ie...@apache.org.
http://git-wip-us.apache.org/repos/asf/beam/blob/a7f9f7d0/integration/java/nexmark/src/test/java/org/apache/beam/integration/nexmark/sources/UnboundedEventSourceTest.java
----------------------------------------------------------------------
diff --git a/integration/java/nexmark/src/test/java/org/apache/beam/integration/nexmark/sources/UnboundedEventSourceTest.java b/integration/java/nexmark/src/test/java/org/apache/beam/integration/nexmark/sources/UnboundedEventSourceTest.java
new file mode 100644
index 0000000..15e17a8
--- /dev/null
+++ b/integration/java/nexmark/src/test/java/org/apache/beam/integration/nexmark/sources/UnboundedEventSourceTest.java
@@ -0,0 +1,110 @@
+/*
+ * 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.beam.integration.nexmark.sources;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+
+import java.io.IOException;
+import java.util.HashSet;
+import java.util.Random;
+import java.util.Set;
+
+import org.apache.beam.integration.nexmark.NexmarkConfiguration;
+import org.apache.beam.integration.nexmark.model.Event;
+import org.apache.beam.sdk.Pipeline;
+import org.apache.beam.sdk.io.UnboundedSource.CheckpointMark;
+import org.apache.beam.sdk.io.UnboundedSource.UnboundedReader;
+import org.apache.beam.sdk.options.PipelineOptions;
+import org.apache.beam.sdk.testing.TestPipeline;
+
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.JUnit4;
+
+/**
+ * Test UnboundedEventSource.
+ */
+@RunWith(JUnit4.class)
+public class UnboundedEventSourceTest {
+  private GeneratorConfig makeConfig(long n) {
+    return new GeneratorConfig(
+        NexmarkConfiguration.DEFAULT, System.currentTimeMillis(), 0, n, 0);
+  }
+
+  /**
+   * Helper for tracking which ids we've seen (so we can detect dups) and
+   * confirming reading events match the model events.
+   */
+  private static class EventIdChecker {
+    private Set<Long> seenPersonIds = new HashSet<>();
+    private Set<Long> seenAuctionIds = new HashSet<>();
+
+    public void add(Event event) {
+      if (event.newAuction != null) {
+        assertTrue(seenAuctionIds.add(event.newAuction.id));
+      } else if (event.newPerson != null) {
+        assertTrue(seenPersonIds.add(event.newPerson.id));
+      }
+    }
+
+    public void add(int n, UnboundedReader<Event> reader, Generator modelGenerator)
+        throws IOException {
+      for (int i = 0; i < n; i++) {
+        assertTrue(modelGenerator.hasNext());
+        Event modelEvent = modelGenerator.next().getValue();
+        assertTrue(reader.advance());
+        Event actualEvent = reader.getCurrent();
+        assertEquals(modelEvent.toString(), actualEvent.toString());
+        add(actualEvent);
+      }
+    }
+  }
+
+  /**
+   * Check aggressively checkpointing and resuming a reader gives us exactly the
+   * same event stream as reading directly.
+   */
+  @Test
+  public void resumeFromCheckpoint() throws IOException {
+    Random random = new Random(297);
+    int n = 47293;
+    GeneratorConfig config = makeConfig(n);
+    Generator modelGenerator = new Generator(config);
+
+    EventIdChecker checker = new EventIdChecker();
+    PipelineOptions options = TestPipeline.testingPipelineOptions();
+    Pipeline p = TestPipeline.create(options);
+    UnboundedEventSource source = new UnboundedEventSource(config, 1, 0, false);
+    UnboundedReader<Event> reader = source.createReader(options, null);
+
+    while (n > 0) {
+      int m = Math.min(459 + random.nextInt(455), n);
+      System.out.printf("reading %d...\n", m);
+      checker.add(m, reader, modelGenerator);
+      n -= m;
+      System.out.printf("splitting with %d remaining...\n", n);
+      CheckpointMark checkpointMark = reader.getCheckpointMark();
+      assertTrue(checkpointMark instanceof Generator.Checkpoint);
+      reader = source.createReader(options, (Generator.Checkpoint) checkpointMark);
+    }
+
+    assertFalse(reader.advance());
+  }
+}


[49/55] [abbrv] beam git commit: Move module beam-integration-java-nexmark to beam-sdks-java-nexmark

Posted by ie...@apache.org.
http://git-wip-us.apache.org/repos/asf/beam/blob/f4333df7/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/AbstractSimulator.java
----------------------------------------------------------------------
diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/AbstractSimulator.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/AbstractSimulator.java
deleted file mode 100644
index 1395182..0000000
--- a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/AbstractSimulator.java
+++ /dev/null
@@ -1,211 +0,0 @@
-/*
- * 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.beam.integration.nexmark.queries;
-
-import java.util.ArrayList;
-import java.util.Iterator;
-import java.util.List;
-import javax.annotation.Nullable;
-
-import org.apache.beam.integration.nexmark.NexmarkUtils;
-import org.apache.beam.sdk.transforms.windowing.BoundedWindow;
-import org.apache.beam.sdk.values.TimestampedValue;
-import org.joda.time.Duration;
-import org.joda.time.Instant;
-
-/**
- * Abstract base class for simulator of a query.
- *
- * @param <InputT> Type of input elements.
- * @param <OutputT> Type of output elements.
- */
-public abstract class AbstractSimulator<InputT, OutputT> {
-  /** Window size for action bucket sampling. */
-  private static final Duration WINDOW_SIZE = Duration.standardMinutes(1);
-
-  /** Input event stream we should draw from. */
-  private final Iterator<TimestampedValue<InputT>> input;
-
-  /** Set to true when no more results. */
-  private boolean isDone;
-
-  /**
-   * Results which have not yet been returned by the {@link #results} iterator.
-   */
-  private final List<TimestampedValue<OutputT>> pendingResults;
-
-  /**
-   * Current window timestamp (ms since epoch).
-   */
-  private long currentWindow;
-
-  /**
-   * Number of (possibly intermediate) results for the current window.
-   */
-  private long currentCount;
-
-  /**
-   * Result counts per window which have not yet been returned by the {@link #resultsPerWindow}
-   * iterator.
-   */
-  private final List<Long> pendingCounts;
-
-  public AbstractSimulator(Iterator<TimestampedValue<InputT>> input) {
-    this.input = input;
-    isDone = false;
-    pendingResults = new ArrayList<>();
-    currentWindow = BoundedWindow.TIMESTAMP_MIN_VALUE.getMillis();
-    currentCount = 0;
-    pendingCounts = new ArrayList<>();
-  }
-
-  /** Called by implementors of {@link #run}: Fetch the next input element. */
-  @Nullable
-  TimestampedValue<InputT> nextInput() {
-    if (!input.hasNext()) {
-      return null;
-    }
-    TimestampedValue<InputT> timestampedInput = input.next();
-    NexmarkUtils.info("input: %s", timestampedInput);
-    return timestampedInput;
-  }
-
-  /**
-   * Called by implementors of {@link #run}:  Capture an intermediate result, for the purpose of
-   * recording the expected activity of the query over time.
-   */
-  void addIntermediateResult(TimestampedValue<OutputT> result) {
-    NexmarkUtils.info("intermediate result: %s", result);
-    updateCounts(result.getTimestamp());
-  }
-
-  /**
-   * Called by implementors of {@link #run}: Capture a final result, for the purpose of checking
-   * semantic correctness.
-   */
-  void addResult(TimestampedValue<OutputT> result) {
-    NexmarkUtils.info("result: %s", result);
-    pendingResults.add(result);
-    updateCounts(result.getTimestamp());
-  }
-
-  /**
-   * Update window and counts.
-   */
-  private void updateCounts(Instant timestamp) {
-    long window = timestamp.getMillis() - timestamp.getMillis() % WINDOW_SIZE.getMillis();
-    if (window > currentWindow) {
-      if (currentWindow > BoundedWindow.TIMESTAMP_MIN_VALUE.getMillis()) {
-        pendingCounts.add(currentCount);
-      }
-      currentCount = 0;
-      currentWindow = window;
-    }
-    currentCount++;
-  }
-
-  /** Called by implementors of {@link #run}: Record that no more results will be emitted. */
-  void allDone() {
-    isDone = true;
-  }
-
-  /**
-   * Overridden by derived classes to do the next increment of work. Each call should
-   * call one or more of {@link #nextInput}, {@link #addIntermediateResult}, {@link #addResult}
-   * or {@link #allDone}. It is ok for a single call to emit more than one result via
-   * {@link #addResult}. It is ok for a single call to run the entire simulation, though
-   * this will prevent the {@link #results} and {@link #resultsPerWindow} iterators to
-   * stall.
-   */
-  protected abstract void run();
-
-  /**
-   * Return iterator over all expected timestamped results. The underlying simulator state is
-   * changed. Only one of {@link #results} or {@link #resultsPerWindow} can be called.
-   */
-  public Iterator<TimestampedValue<OutputT>> results() {
-    return new Iterator<TimestampedValue<OutputT>>() {
-      @Override
-      public boolean hasNext() {
-        while (true) {
-          if (!pendingResults.isEmpty()) {
-            return true;
-          }
-          if (isDone) {
-            return false;
-          }
-          run();
-        }
-      }
-
-      @Override
-      public TimestampedValue<OutputT> next() {
-        TimestampedValue<OutputT> result = pendingResults.get(0);
-        pendingResults.remove(0);
-        return result;
-      }
-
-      @Override
-      public void remove() {
-        throw new UnsupportedOperationException();
-      }
-    };
-  }
-
-  /**
-   * Return an iterator over the number of results per {@link #WINDOW_SIZE} period. The underlying
-   * simulator state is changed.  Only one of {@link #results} or {@link #resultsPerWindow} can be
-   * called.
-   */
-  public Iterator<Long> resultsPerWindow() {
-    return new Iterator<Long>() {
-      @Override
-      public boolean hasNext() {
-        while (true) {
-          if (!pendingCounts.isEmpty()) {
-            return true;
-          }
-          if (isDone) {
-            if (currentCount > 0) {
-              pendingCounts.add(currentCount);
-              currentCount = 0;
-              currentWindow = BoundedWindow.TIMESTAMP_MAX_VALUE.getMillis();
-              return true;
-            } else {
-              return false;
-            }
-          }
-          run();
-        }
-      }
-
-      @Override
-      public Long next() {
-        Long result = pendingCounts.get(0);
-        pendingCounts.remove(0);
-        return result;
-      }
-
-      @Override
-      public void remove() {
-        throw new UnsupportedOperationException();
-      }
-    };
-  }
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/f4333df7/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/NexmarkQuery.java
----------------------------------------------------------------------
diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/NexmarkQuery.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/NexmarkQuery.java
deleted file mode 100644
index 8b74282..0000000
--- a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/NexmarkQuery.java
+++ /dev/null
@@ -1,270 +0,0 @@
-/*
- * 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.beam.integration.nexmark.queries;
-
-import org.apache.beam.integration.nexmark.Monitor;
-import org.apache.beam.integration.nexmark.NexmarkConfiguration;
-import org.apache.beam.integration.nexmark.NexmarkUtils;
-import org.apache.beam.integration.nexmark.model.Auction;
-import org.apache.beam.integration.nexmark.model.Bid;
-import org.apache.beam.integration.nexmark.model.Event;
-import org.apache.beam.integration.nexmark.model.KnownSize;
-import org.apache.beam.integration.nexmark.model.Person;
-import org.apache.beam.sdk.metrics.Counter;
-import org.apache.beam.sdk.metrics.Metrics;
-import org.apache.beam.sdk.transforms.DoFn;
-import org.apache.beam.sdk.transforms.Filter;
-import org.apache.beam.sdk.transforms.PTransform;
-import org.apache.beam.sdk.transforms.ParDo;
-import org.apache.beam.sdk.transforms.SerializableFunction;
-import org.apache.beam.sdk.values.KV;
-import org.apache.beam.sdk.values.PCollection;
-import org.apache.beam.sdk.values.TimestampedValue;
-import org.apache.beam.sdk.values.TupleTag;
-import org.joda.time.Instant;
-
-/**
- * Base class for the eight 'NEXMark' queries. Supplies some fragments common to
- * multiple queries.
- */
-public abstract class NexmarkQuery
-    extends PTransform<PCollection<Event>, PCollection<TimestampedValue<KnownSize>>> {
-  public static final TupleTag<Auction> AUCTION_TAG = new TupleTag<>("auctions");
-  public static final TupleTag<Bid> BID_TAG = new TupleTag<>("bids");
-  static final TupleTag<Person> PERSON_TAG = new TupleTag<>("person");
-
-  /** Predicate to detect a new person event. */
-  private static final SerializableFunction<Event, Boolean> IS_NEW_PERSON =
-      new SerializableFunction<Event, Boolean>() {
-        @Override
-        public Boolean apply(Event event) {
-          return event.newPerson != null;
-        }
-      };
-
-  /** DoFn to convert a new person event to a person. */
-  private static final DoFn<Event, Person> AS_PERSON = new DoFn<Event, Person>() {
-    @ProcessElement
-    public void processElement(ProcessContext c) {
-      c.output(c.element().newPerson);
-    }
-  };
-
-  /** Predicate to detect a new auction event. */
-  private static final SerializableFunction<Event, Boolean> IS_NEW_AUCTION =
-      new SerializableFunction<Event, Boolean>() {
-        @Override
-        public Boolean apply(Event event) {
-          return event.newAuction != null;
-        }
-      };
-
-  /** DoFn to convert a new auction event to an auction. */
-  private static final DoFn<Event, Auction> AS_AUCTION = new DoFn<Event, Auction>() {
-    @ProcessElement
-    public void processElement(ProcessContext c) {
-      c.output(c.element().newAuction);
-    }
-  };
-
-  /** Predicate to detect a new bid event. */
-  private static final SerializableFunction<Event, Boolean> IS_BID =
-      new SerializableFunction<Event, Boolean>() {
-        @Override
-        public Boolean apply(Event event) {
-          return event.bid != null;
-        }
-      };
-
-  /** DoFn to convert a bid event to a bid. */
-  private static final DoFn<Event, Bid> AS_BID = new DoFn<Event, Bid>() {
-    @ProcessElement
-    public void processElement(ProcessContext c) {
-      c.output(c.element().bid);
-    }
-  };
-
-  /** Transform to key each person by their id. */
-  static final ParDo.SingleOutput<Person, KV<Long, Person>> PERSON_BY_ID =
-      ParDo.of(new DoFn<Person, KV<Long, Person>>() {
-             @ProcessElement
-             public void processElement(ProcessContext c) {
-               c.output(KV.of(c.element().id, c.element()));
-             }
-           });
-
-  /** Transform to key each auction by its id. */
-  static final ParDo.SingleOutput<Auction, KV<Long, Auction>> AUCTION_BY_ID =
-      ParDo.of(new DoFn<Auction, KV<Long, Auction>>() {
-             @ProcessElement
-             public void processElement(ProcessContext c) {
-               c.output(KV.of(c.element().id, c.element()));
-             }
-           });
-
-  /** Transform to key each auction by its seller id. */
-  static final ParDo.SingleOutput<Auction, KV<Long, Auction>> AUCTION_BY_SELLER =
-      ParDo.of(new DoFn<Auction, KV<Long, Auction>>() {
-             @ProcessElement
-             public void processElement(ProcessContext c) {
-               c.output(KV.of(c.element().seller, c.element()));
-             }
-           });
-
-  /** Transform to key each bid by it's auction id. */
-  static final ParDo.SingleOutput<Bid, KV<Long, Bid>> BID_BY_AUCTION =
-      ParDo.of(new DoFn<Bid, KV<Long, Bid>>() {
-             @ProcessElement
-             public void processElement(ProcessContext c) {
-               c.output(KV.of(c.element().auction, c.element()));
-             }
-           });
-
-  /** Transform to project the auction id from each bid. */
-  static final ParDo.SingleOutput<Bid, Long> BID_TO_AUCTION =
-      ParDo.of(new DoFn<Bid, Long>() {
-             @ProcessElement
-             public void processElement(ProcessContext c) {
-               c.output(c.element().auction);
-             }
-           });
-
-  /** Transform to project the price from each bid. */
-  static final ParDo.SingleOutput<Bid, Long> BID_TO_PRICE =
-      ParDo.of(new DoFn<Bid, Long>() {
-             @ProcessElement
-             public void processElement(ProcessContext c) {
-               c.output(c.element().price);
-             }
-           });
-
-  /** Transform to emit each event with the timestamp embedded within it. */
-  public static final ParDo.SingleOutput<Event, Event> EVENT_TIMESTAMP_FROM_DATA =
-      ParDo.of(new DoFn<Event, Event>() {
-             @ProcessElement
-             public void processElement(ProcessContext c) {
-               Event e = c.element();
-               if (e.bid != null) {
-                 c.outputWithTimestamp(e, new Instant(e.bid.dateTime));
-               } else if (e.newPerson != null) {
-                 c.outputWithTimestamp(e, new Instant(e.newPerson.dateTime));
-               } else if (e.newAuction != null) {
-                 c.outputWithTimestamp(e, new Instant(e.newAuction.dateTime));
-               }
-             }
-           });
-
-  /**
-   * Transform to filter for just the new auction events.
-   */
-  public static final PTransform<PCollection<Event>, PCollection<Auction>> JUST_NEW_AUCTIONS =
-      new PTransform<PCollection<Event>, PCollection<Auction>>("justNewAuctions") {
-        @Override
-        public PCollection<Auction> expand(PCollection<Event> input) {
-          return input.apply("IsNewAuction", Filter.by(IS_NEW_AUCTION))
-                      .apply("AsAuction", ParDo.of(AS_AUCTION));
-        }
-      };
-
-  /**
-   * Transform to filter for just the new person events.
-   */
-  public static final PTransform<PCollection<Event>, PCollection<Person>> JUST_NEW_PERSONS =
-      new PTransform<PCollection<Event>, PCollection<Person>>("justNewPersons") {
-        @Override
-        public PCollection<Person> expand(PCollection<Event> input) {
-          return input.apply("IsNewPerson", Filter.by(IS_NEW_PERSON))
-                      .apply("AsPerson", ParDo.of(AS_PERSON));
-        }
-      };
-
-  /**
-   * Transform to filter for just the bid events.
-   */
-  public static final PTransform<PCollection<Event>, PCollection<Bid>> JUST_BIDS =
-      new PTransform<PCollection<Event>, PCollection<Bid>>("justBids") {
-        @Override
-        public PCollection<Bid> expand(PCollection<Event> input) {
-          return input.apply("IsBid", Filter.by(IS_BID))
-                      .apply("AsBid", ParDo.of(AS_BID));
-        }
-      };
-
-  final NexmarkConfiguration configuration;
-  public final Monitor<Event> eventMonitor;
-  public final Monitor<KnownSize> resultMonitor;
-  private final Monitor<Event> endOfStreamMonitor;
-  private final Counter fatalCounter;
-
-  NexmarkQuery(NexmarkConfiguration configuration, String name) {
-    super(name);
-    this.configuration = configuration;
-    if (configuration.debug) {
-      eventMonitor = new Monitor<>(name + ".Events", "event");
-      resultMonitor = new Monitor<>(name + ".Results", "result");
-      endOfStreamMonitor = new Monitor<>(name + ".EndOfStream", "end");
-      fatalCounter = Metrics.counter(name , "fatal");
-    } else {
-      eventMonitor = null;
-      resultMonitor = null;
-      endOfStreamMonitor = null;
-      fatalCounter = null;
-    }
-  }
-
-  /**
-   * Implement the actual query. All we know about the result is it has a known encoded size.
-   */
-  protected abstract PCollection<KnownSize> applyPrim(PCollection<Event> events);
-
-  @Override
-  public PCollection<TimestampedValue<KnownSize>> expand(PCollection<Event> events) {
-
-    if (configuration.debug) {
-      events =
-          events
-              // Monitor events as they go by.
-              .apply(name + ".Monitor", eventMonitor.getTransform())
-              // Count each type of event.
-              .apply(name + ".Snoop", NexmarkUtils.snoop(name));
-    }
-
-    if (configuration.cpuDelayMs > 0) {
-      // Slow down by pegging one core at 100%.
-      events = events.apply(name + ".CpuDelay",
-              NexmarkUtils.<Event>cpuDelay(name, configuration.cpuDelayMs));
-    }
-
-    if (configuration.diskBusyBytes > 0) {
-      // Slow down by forcing bytes to durable store.
-      events = events.apply(name + ".DiskBusy",
-              NexmarkUtils.<Event>diskBusy(configuration.diskBusyBytes));
-    }
-
-    // Run the query.
-    PCollection<KnownSize> queryResults = applyPrim(events);
-
-    if (configuration.debug) {
-      // Monitor results as they go by.
-      queryResults = queryResults.apply(name + ".Debug", resultMonitor.getTransform());
-    }
-
-    // Timestamp the query results.
-    return queryResults.apply(name + ".Stamp", NexmarkUtils.<KnownSize>stamp(name));
-  }
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/f4333df7/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/NexmarkQueryModel.java
----------------------------------------------------------------------
diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/NexmarkQueryModel.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/NexmarkQueryModel.java
deleted file mode 100644
index bfa668b..0000000
--- a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/NexmarkQueryModel.java
+++ /dev/null
@@ -1,118 +0,0 @@
-/*
- * 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.beam.integration.nexmark.queries;
-
-import java.io.Serializable;
-import java.util.ArrayList;
-import java.util.Collection;
-import java.util.HashSet;
-import java.util.Iterator;
-import java.util.List;
-import java.util.Set;
-
-import org.apache.beam.integration.nexmark.NexmarkConfiguration;
-import org.apache.beam.integration.nexmark.model.KnownSize;
-import org.apache.beam.sdk.transforms.SerializableFunction;
-import org.apache.beam.sdk.values.TimestampedValue;
-
-
-import org.hamcrest.core.IsEqual;
-import org.joda.time.Duration;
-import org.joda.time.Instant;
-import org.junit.Assert;
-
-/**
- * Base class for models of the eight NEXMark queries. Provides an assertion function which can be
- * applied against the actual query results to check their consistency with the model.
- */
-public abstract class NexmarkQueryModel implements Serializable {
-  public final NexmarkConfiguration configuration;
-
-  NexmarkQueryModel(NexmarkConfiguration configuration) {
-    this.configuration = configuration;
-  }
-
-  /**
-   * Return the start of the most recent window of {@code size} and {@code period} which ends
-   * strictly before {@code timestamp}.
-   */
-  static Instant windowStart(Duration size, Duration period, Instant timestamp) {
-    long ts = timestamp.getMillis();
-    long p = period.getMillis();
-    long lim = ts - ts % p;
-    long s = size.getMillis();
-    return new Instant(lim - s);
-  }
-
-  /** Convert {@code itr} to strings capturing values, timestamps and order. */
-  static <T> List<String> toValueTimestampOrder(Iterator<TimestampedValue<T>> itr) {
-    List<String> strings = new ArrayList<>();
-    while (itr.hasNext()) {
-      strings.add(itr.next().toString());
-    }
-    return strings;
-  }
-
-  /** Convert {@code itr} to strings capturing values and order. */
-  static <T> List<String> toValueOrder(Iterator<TimestampedValue<T>> itr) {
-    List<String> strings = new ArrayList<>();
-    while (itr.hasNext()) {
-      strings.add(itr.next().getValue().toString());
-    }
-    return strings;
-  }
-
-  /** Convert {@code itr} to strings capturing values only. */
-  static <T> Set<String> toValue(Iterator<TimestampedValue<T>> itr) {
-    Set<String> strings = new HashSet<>();
-    while (itr.hasNext()) {
-      strings.add(itr.next().getValue().toString());
-    }
-    return strings;
-  }
-
-  /** Return simulator for query. */
-  public abstract AbstractSimulator<?, ?> simulator();
-
-  /** Return sub-sequence of results which are significant for model. */
-  Iterable<TimestampedValue<KnownSize>> relevantResults(
-      Iterable<TimestampedValue<KnownSize>> results) {
-    return results;
-  }
-
-  /**
-   * Convert iterator of elements to collection of strings to use when testing coherence of model
-   * against actual query results.
-   */
-  protected abstract <T> Collection<String> toCollection(Iterator<TimestampedValue<T>> itr);
-
-  /** Return assertion to use on results of pipeline for this query. */
-  public SerializableFunction<Iterable<TimestampedValue<KnownSize>>, Void> assertionFor() {
-    final Collection<String> expectedStrings = toCollection(simulator().results());
-
-    return new SerializableFunction<Iterable<TimestampedValue<KnownSize>>, Void>() {
-      @Override
-      public Void apply(Iterable<TimestampedValue<KnownSize>> actual) {
-      Collection<String> actualStrings = toCollection(relevantResults(actual).iterator());
-        Assert.assertThat("wrong pipeline output", actualStrings,
-          IsEqual.equalTo(expectedStrings));
-        return null;
-      }
-    };
-  }
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/f4333df7/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query0.java
----------------------------------------------------------------------
diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query0.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query0.java
deleted file mode 100644
index 00a49a8..0000000
--- a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query0.java
+++ /dev/null
@@ -1,71 +0,0 @@
-/*
- * 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.beam.integration.nexmark.queries;
-
-import java.io.ByteArrayInputStream;
-import java.io.ByteArrayOutputStream;
-import java.io.IOException;
-
-import org.apache.beam.integration.nexmark.NexmarkConfiguration;
-import org.apache.beam.integration.nexmark.NexmarkUtils;
-import org.apache.beam.integration.nexmark.model.Event;
-import org.apache.beam.integration.nexmark.model.KnownSize;
-import org.apache.beam.sdk.coders.Coder;
-import org.apache.beam.sdk.coders.CoderException;
-import org.apache.beam.sdk.metrics.Counter;
-import org.apache.beam.sdk.metrics.Metrics;
-import org.apache.beam.sdk.transforms.DoFn;
-import org.apache.beam.sdk.transforms.ParDo;
-import org.apache.beam.sdk.values.PCollection;
-
-/**
- * Query 0: Pass events through unchanged. However, force them to do a round trip through
- * serialization so that we measure the impact of the choice of coders.
- */
-public class Query0 extends NexmarkQuery {
-  public Query0(NexmarkConfiguration configuration) {
-    super(configuration, "Query0");
-  }
-
-  private PCollection<Event> applyTyped(PCollection<Event> events) {
-    final Coder<Event> coder = events.getCoder();
-    return events
-        // Force round trip through coder.
-        .apply(name + ".Serialize",
-            ParDo.of(new DoFn<Event, Event>() {
-                  private final Counter bytesMetric =
-                    Metrics.counter(name , "bytes");
-
-                  @ProcessElement
-                  public void processElement(ProcessContext c) throws CoderException, IOException {
-                    ByteArrayOutputStream outStream = new ByteArrayOutputStream();
-                    coder.encode(c.element(), outStream, Coder.Context.OUTER);
-                    byte[] byteArray = outStream.toByteArray();
-                    bytesMetric.inc((long) byteArray.length);
-                    ByteArrayInputStream inStream = new ByteArrayInputStream(byteArray);
-                    Event event = coder.decode(inStream, Coder.Context.OUTER);
-                    c.output(event);
-                  }
-                }));
-  }
-
-  @Override
-  protected PCollection<KnownSize> applyPrim(PCollection<Event> events) {
-    return NexmarkUtils.castToKnownSize(name, applyTyped(events));
-  }
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/f4333df7/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query0Model.java
----------------------------------------------------------------------
diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query0Model.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query0Model.java
deleted file mode 100644
index e2522b8..0000000
--- a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query0Model.java
+++ /dev/null
@@ -1,64 +0,0 @@
-/*
- * 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.beam.integration.nexmark.queries;
-
-import java.util.Collection;
-import java.util.Iterator;
-
-import org.apache.beam.integration.nexmark.NexmarkConfiguration;
-import org.apache.beam.integration.nexmark.NexmarkUtils;
-import org.apache.beam.integration.nexmark.model.Event;
-import org.apache.beam.sdk.values.TimestampedValue;
-
-/**
- * A direct implementation of {@link Query0}.
- */
-public class Query0Model extends NexmarkQueryModel {
-  /**
-   * Simulator for query 0.
-   */
-  private static class Simulator extends AbstractSimulator<Event, Event> {
-    public Simulator(NexmarkConfiguration configuration) {
-      super(NexmarkUtils.standardEventIterator(configuration));
-    }
-
-    @Override
-    protected void run() {
-      TimestampedValue<Event> timestampedEvent = nextInput();
-      if (timestampedEvent == null) {
-        allDone();
-        return;
-      }
-      addResult(timestampedEvent);
-    }
-  }
-
-  public Query0Model(NexmarkConfiguration configuration) {
-    super(configuration);
-  }
-
-  @Override
-  public AbstractSimulator<?, ?> simulator() {
-    return new Simulator(configuration);
-  }
-
-  @Override
-  protected <T> Collection<String> toCollection(Iterator<TimestampedValue<T>> itr) {
-    return toValueTimestampOrder(itr);
-  }
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/f4333df7/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query1.java
----------------------------------------------------------------------
diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query1.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query1.java
deleted file mode 100644
index 8d90b70..0000000
--- a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query1.java
+++ /dev/null
@@ -1,67 +0,0 @@
-/*
- * 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.beam.integration.nexmark.queries;
-
-import org.apache.beam.integration.nexmark.NexmarkConfiguration;
-import org.apache.beam.integration.nexmark.NexmarkUtils;
-import org.apache.beam.integration.nexmark.model.Bid;
-import org.apache.beam.integration.nexmark.model.Event;
-import org.apache.beam.integration.nexmark.model.KnownSize;
-import org.apache.beam.sdk.transforms.DoFn;
-import org.apache.beam.sdk.transforms.ParDo;
-import org.apache.beam.sdk.values.PCollection;
-
-/**
- * Query 1, 'Currency Conversion'. Convert each bid value from dollars to euros.
- * In CQL syntax:
- *
- * <pre>
- * SELECT Istream(auction, DOLTOEUR(price), bidder, datetime)
- * FROM bid [ROWS UNBOUNDED];
- * </pre>
- *
- * <p>To make things more interesting, allow the 'currency conversion' to be arbitrarily
- * slowed down.
- */
-public class Query1 extends NexmarkQuery {
-  public Query1(NexmarkConfiguration configuration) {
-    super(configuration, "Query1");
-  }
-
-  private PCollection<Bid> applyTyped(PCollection<Event> events) {
-    return events
-        // Only want the bid events.
-        .apply(JUST_BIDS)
-
-        // Map the conversion function over all bids.
-        .apply(name + ".ToEuros",
-            ParDo.of(new DoFn<Bid, Bid>() {
-                  @ProcessElement
-                  public void processElement(ProcessContext c) {
-                    Bid bid = c.element();
-                    c.output(new Bid(
-                        bid.auction, bid.bidder, (bid.price * 89) / 100, bid.dateTime, bid.extra));
-                  }
-                }));
-  }
-
-  @Override
-  protected PCollection<KnownSize> applyPrim(PCollection<Event> events) {
-    return NexmarkUtils.castToKnownSize(name, applyTyped(events));
-  }
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/f4333df7/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query10.java
----------------------------------------------------------------------
diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query10.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query10.java
deleted file mode 100644
index 378d01e..0000000
--- a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query10.java
+++ /dev/null
@@ -1,367 +0,0 @@
-/*
- * 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.beam.integration.nexmark.queries;
-
-import java.io.IOException;
-import java.io.OutputStream;
-import java.io.Serializable;
-import java.nio.channels.Channels;
-import java.nio.channels.WritableByteChannel;
-import java.util.concurrent.ThreadLocalRandom;
-import javax.annotation.Nullable;
-import org.apache.beam.integration.nexmark.NexmarkConfiguration;
-import org.apache.beam.integration.nexmark.NexmarkUtils;
-import org.apache.beam.integration.nexmark.model.Done;
-import org.apache.beam.integration.nexmark.model.Event;
-import org.apache.beam.integration.nexmark.model.KnownSize;
-import org.apache.beam.sdk.coders.Coder;
-import org.apache.beam.sdk.extensions.gcp.options.GcsOptions;
-import org.apache.beam.sdk.metrics.Counter;
-import org.apache.beam.sdk.metrics.Metrics;
-import org.apache.beam.sdk.transforms.DoFn;
-import org.apache.beam.sdk.transforms.GroupByKey;
-import org.apache.beam.sdk.transforms.ParDo;
-import org.apache.beam.sdk.transforms.windowing.AfterEach;
-import org.apache.beam.sdk.transforms.windowing.AfterFirst;
-import org.apache.beam.sdk.transforms.windowing.AfterPane;
-import org.apache.beam.sdk.transforms.windowing.AfterProcessingTime;
-import org.apache.beam.sdk.transforms.windowing.AfterWatermark;
-import org.apache.beam.sdk.transforms.windowing.BoundedWindow;
-import org.apache.beam.sdk.transforms.windowing.FixedWindows;
-import org.apache.beam.sdk.transforms.windowing.PaneInfo;
-import org.apache.beam.sdk.transforms.windowing.PaneInfo.Timing;
-import org.apache.beam.sdk.transforms.windowing.Repeatedly;
-import org.apache.beam.sdk.transforms.windowing.Window;
-import org.apache.beam.sdk.values.KV;
-import org.apache.beam.sdk.values.PCollection;
-import org.joda.time.Duration;
-import org.joda.time.Instant;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-/**
- * Query "10", 'Log to sharded files' (Not in original suite.)
- *
- * <p>Every windowSizeSec, save all events from the last period into 2*maxWorkers log files.
- */
-public class Query10 extends NexmarkQuery {
-  private static final Logger LOG = LoggerFactory.getLogger(Query10.class);
-  private static final int CHANNEL_BUFFER = 8 << 20; // 8MB
-  private static final int NUM_SHARDS_PER_WORKER = 5;
-  private static final Duration LATE_BATCHING_PERIOD = Duration.standardSeconds(10);
-
-  /**
-   * Capture everything we need to know about the records in a single output file.
-   */
-  private static class OutputFile implements Serializable {
-    /** Maximum possible timestamp of records in file. */
-    private final Instant maxTimestamp;
-    /** Shard within window. */
-    private final String shard;
-    /** Index of file in all files in shard. */
-    private final long index;
-    /** Timing of records in this file. */
-    private final PaneInfo.Timing timing;
-    /** Path to file containing records, or {@literal null} if no output required. */
-    @Nullable
-    private final String filename;
-
-    public OutputFile(
-        Instant maxTimestamp,
-        String shard,
-        long index,
-        PaneInfo.Timing timing,
-        @Nullable String filename) {
-      this.maxTimestamp = maxTimestamp;
-      this.shard = shard;
-      this.index = index;
-      this.timing = timing;
-      this.filename = filename;
-    }
-
-    @Override
-    public String toString() {
-      return String.format("%s %s %d %s %s%n", maxTimestamp, shard, index, timing, filename);
-    }
-  }
-
-  /**
-   * GCS uri prefix for all log and 'finished' files. If null they won't be written.
-   */
-  @Nullable
-  private String outputPath;
-
-  /**
-   * Maximum number of workers, used to determine log sharding factor.
-   */
-  private int maxNumWorkers;
-
-  public Query10(NexmarkConfiguration configuration) {
-    super(configuration, "Query10");
-  }
-
-  public void setOutputPath(@Nullable String outputPath) {
-    this.outputPath = outputPath;
-  }
-
-  public void setMaxNumWorkers(int maxNumWorkers) {
-    this.maxNumWorkers = maxNumWorkers;
-  }
-
-  /**
-   * Return channel for writing bytes to GCS.
-   */
-  private WritableByteChannel openWritableGcsFile(GcsOptions options, String filename)
-      throws IOException {
-    //TODO
-    // Fix after PR: right now this is a specific Google added use case
-    // Discuss it on ML: shall we keep GCS or use HDFS or use a generic beam filesystem way.
-    throw new UnsupportedOperationException("Disabled after removal of GcsIOChannelFactory");
-  }
-
-  /** Return a short string to describe {@code timing}. */
-  private String timingToString(PaneInfo.Timing timing) {
-    switch (timing) {
-      case EARLY:
-        return "E";
-      case ON_TIME:
-        return "O";
-      case LATE:
-        return "L";
-    }
-    throw new RuntimeException(); // cases are exhaustive
-  }
-
-  /** Construct an {@link OutputFile} for {@code pane} in {@code window} for {@code shard}. */
-  private OutputFile outputFileFor(BoundedWindow window, String shard, PaneInfo pane) {
-    @Nullable String filename =
-        outputPath == null
-        ? null
-        : String.format("%s/LOG-%s-%s-%03d-%s-%x",
-            outputPath, window.maxTimestamp(), shard, pane.getIndex(),
-            timingToString(pane.getTiming()),
-            ThreadLocalRandom.current().nextLong());
-    return new OutputFile(window.maxTimestamp(), shard, pane.getIndex(),
-        pane.getTiming(), filename);
-  }
-
-  /**
-   * Return path to which we should write the index for {@code window}, or {@literal null}
-   * if no output required.
-   */
-  @Nullable
-  private String indexPathFor(BoundedWindow window) {
-    if (outputPath == null) {
-      return null;
-    }
-    return String.format("%s/INDEX-%s", outputPath, window.maxTimestamp());
-  }
-
-  private PCollection<Done> applyTyped(PCollection<Event> events) {
-    final int numLogShards = maxNumWorkers * NUM_SHARDS_PER_WORKER;
-
-    return events
-      .apply(name + ".ShardEvents",
-        ParDo.of(new DoFn<Event, KV<String, Event>>() {
-          private final Counter lateCounter = Metrics.counter(name , "actuallyLateEvent");
-          private final Counter onTimeCounter = Metrics.counter(name , "onTimeCounter");
-
-          @ProcessElement
-          public void processElement(ProcessContext c) {
-            if (c.element().hasAnnotation("LATE")) {
-              lateCounter.inc();
-              LOG.info("Observed late: %s", c.element());
-            } else {
-              onTimeCounter.inc();
-            }
-            int shardNum = (int) Math.abs((long) c.element().hashCode() % numLogShards);
-            String shard = String.format("shard-%05d-of-%05d", shardNum, numLogShards);
-            c.output(KV.of(shard, c.element()));
-          }
-        }))
-      .apply(name + ".WindowEvents",
-        Window.<KV<String, Event>>into(
-          FixedWindows.of(Duration.standardSeconds(configuration.windowSizeSec)))
-          .triggering(AfterEach.inOrder(
-              Repeatedly
-                  .forever(AfterPane.elementCountAtLeast(configuration.maxLogEvents))
-                  .orFinally(AfterWatermark.pastEndOfWindow()),
-              Repeatedly.forever(
-                  AfterFirst.of(AfterPane.elementCountAtLeast(configuration.maxLogEvents),
-                      AfterProcessingTime.pastFirstElementInPane()
-                                         .plusDelayOf(LATE_BATCHING_PERIOD)))))
-          .discardingFiredPanes()
-          // Use a 1 day allowed lateness so that any forgotten hold will stall the
-          // pipeline for that period and be very noticeable.
-          .withAllowedLateness(Duration.standardDays(1)))
-      .apply(name + ".GroupByKey", GroupByKey.<String, Event>create())
-      .apply(name + ".CheckForLateEvents",
-        ParDo.of(new DoFn<KV<String, Iterable<Event>>,
-                 KV<String, Iterable<Event>>>() {
-          private final Counter earlyCounter = Metrics.counter(name , "earlyShard");
-          private final Counter onTimeCounter = Metrics.counter(name , "onTimeShard");
-          private final Counter lateCounter = Metrics.counter(name , "lateShard");
-          private final Counter unexpectedLatePaneCounter =
-            Metrics.counter(name , "ERROR_unexpectedLatePane");
-          private final Counter unexpectedOnTimeElementCounter =
-            Metrics.counter(name , "ERROR_unexpectedOnTimeElement");
-
-          @ProcessElement
-          public void processElement(ProcessContext c, BoundedWindow window) {
-            int numLate = 0;
-            int numOnTime = 0;
-            for (Event event : c.element().getValue()) {
-              if (event.hasAnnotation("LATE")) {
-                numLate++;
-              } else {
-                numOnTime++;
-              }
-            }
-            String shard = c.element().getKey();
-            LOG.info(String.format(
-                "%s with timestamp %s has %d actually late and %d on-time "
-                    + "elements in pane %s for window %s",
-                shard, c.timestamp(), numLate, numOnTime, c.pane(),
-                window.maxTimestamp()));
-            if (c.pane().getTiming() == PaneInfo.Timing.LATE) {
-              if (numLate == 0) {
-                LOG.error(
-                    "ERROR! No late events in late pane for %s", shard);
-                unexpectedLatePaneCounter.inc();
-              }
-              if (numOnTime > 0) {
-                LOG.error(
-                    "ERROR! Have %d on-time events in late pane for %s",
-                    numOnTime, shard);
-                unexpectedOnTimeElementCounter.inc();
-              }
-              lateCounter.inc();
-            } else if (c.pane().getTiming() == PaneInfo.Timing.EARLY) {
-              if (numOnTime + numLate < configuration.maxLogEvents) {
-                LOG.error(
-                    "ERROR! Only have %d events in early pane for %s",
-                    numOnTime + numLate, shard);
-              }
-              earlyCounter.inc();
-            } else {
-              onTimeCounter.inc();
-            }
-            c.output(c.element());
-          }
-        }))
-      .apply(name + ".UploadEvents",
-        ParDo.of(new DoFn<KV<String, Iterable<Event>>,
-                 KV<Void, OutputFile>>() {
-          private final Counter savedFileCounter = Metrics.counter(name , "savedFile");
-          private final Counter writtenRecordsCounter = Metrics.counter(name , "writtenRecords");
-
-            @ProcessElement
-            public void processElement(ProcessContext c, BoundedWindow window)
-                    throws IOException {
-              String shard = c.element().getKey();
-              GcsOptions options = c.getPipelineOptions().as(GcsOptions.class);
-              OutputFile outputFile = outputFileFor(window, shard, c.pane());
-              LOG.info(String.format(
-                  "Writing %s with record timestamp %s, window timestamp %s, pane %s",
-                  shard, c.timestamp(), window.maxTimestamp(), c.pane()));
-              if (outputFile.filename != null) {
-                LOG.info("Beginning write to '%s'", outputFile.filename);
-                int n = 0;
-                try (OutputStream output =
-                         Channels.newOutputStream(openWritableGcsFile(options, outputFile
-                             .filename))) {
-                  for (Event event : c.element().getValue()) {
-                    Event.CODER.encode(event, output, Coder.Context.OUTER);
-                    writtenRecordsCounter.inc();
-                    if (++n % 10000 == 0) {
-                      LOG.info("So far written %d records to '%s'", n,
-                          outputFile.filename);
-                    }
-                  }
-                }
-                LOG.info("Written all %d records to '%s'", n, outputFile.filename);
-              }
-              savedFileCounter.inc();
-              c.output(KV.<Void, OutputFile>of(null, outputFile));
-            }
-          }))
-      // Clear fancy triggering from above.
-      .apply(name + ".WindowLogFiles", Window.<KV<Void, OutputFile>>into(
-        FixedWindows.of(Duration.standardSeconds(configuration.windowSizeSec)))
-        .triggering(AfterWatermark.pastEndOfWindow())
-        // We expect no late data here, but we'll assume the worst so we can detect any.
-        .withAllowedLateness(Duration.standardDays(1))
-        .discardingFiredPanes())
-      // this GroupByKey allows to have one file per window
-      .apply(name + ".GroupByKey2", GroupByKey.<Void, OutputFile>create())
-      .apply(name + ".Index",
-        ParDo.of(new DoFn<KV<Void, Iterable<OutputFile>>, Done>() {
-          private final Counter unexpectedLateCounter =
-            Metrics.counter(name , "ERROR_unexpectedLate");
-          private final Counter unexpectedEarlyCounter =
-              Metrics.counter(name , "ERROR_unexpectedEarly");
-          private final Counter unexpectedIndexCounter =
-              Metrics.counter(name , "ERROR_unexpectedIndex");
-          private final Counter finalizedCounter = Metrics.counter(name , "indexed");
-
-          @ProcessElement
-          public void processElement(ProcessContext c, BoundedWindow window)
-                  throws IOException {
-            if (c.pane().getTiming() == Timing.LATE) {
-              unexpectedLateCounter.inc();
-              LOG.error("ERROR! Unexpected LATE pane: %s", c.pane());
-            } else if (c.pane().getTiming() == Timing.EARLY) {
-              unexpectedEarlyCounter.inc();
-              LOG.error("ERROR! Unexpected EARLY pane: %s", c.pane());
-            } else if (c.pane().getTiming() == Timing.ON_TIME
-                && c.pane().getIndex() != 0) {
-              unexpectedIndexCounter.inc();
-              LOG.error("ERROR! Unexpected ON_TIME pane index: %s", c.pane());
-            } else {
-              GcsOptions options = c.getPipelineOptions().as(GcsOptions.class);
-              LOG.info(
-                  "Index with record timestamp %s, window timestamp %s, pane %s",
-                  c.timestamp(), window.maxTimestamp(), c.pane());
-
-              @Nullable String filename = indexPathFor(window);
-              if (filename != null) {
-                LOG.info("Beginning write to '%s'", filename);
-                int n = 0;
-                try (OutputStream output =
-                         Channels.newOutputStream(
-                             openWritableGcsFile(options, filename))) {
-                  for (OutputFile outputFile : c.element().getValue()) {
-                    output.write(outputFile.toString().getBytes("UTF-8"));
-                    n++;
-                  }
-                }
-                LOG.info("Written all %d lines to '%s'", n, filename);
-              }
-              c.output(
-                  new Done("written for timestamp " + window.maxTimestamp()));
-              finalizedCounter.inc();
-            }
-          }
-        }));
-  }
-
-  @Override
-  protected PCollection<KnownSize> applyPrim(PCollection<Event> events) {
-    return NexmarkUtils.castToKnownSize(name, applyTyped(events));
-  }
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/f4333df7/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query11.java
----------------------------------------------------------------------
diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query11.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query11.java
deleted file mode 100644
index 6db9bcf..0000000
--- a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query11.java
+++ /dev/null
@@ -1,79 +0,0 @@
-/*
- * 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.beam.integration.nexmark.queries;
-
-import org.apache.beam.integration.nexmark.NexmarkConfiguration;
-import org.apache.beam.integration.nexmark.NexmarkUtils;
-import org.apache.beam.integration.nexmark.model.Bid;
-import org.apache.beam.integration.nexmark.model.BidsPerSession;
-import org.apache.beam.integration.nexmark.model.Event;
-import org.apache.beam.integration.nexmark.model.KnownSize;
-import org.apache.beam.sdk.transforms.Count;
-import org.apache.beam.sdk.transforms.DoFn;
-import org.apache.beam.sdk.transforms.ParDo;
-import org.apache.beam.sdk.transforms.windowing.AfterPane;
-import org.apache.beam.sdk.transforms.windowing.Repeatedly;
-import org.apache.beam.sdk.transforms.windowing.Sessions;
-import org.apache.beam.sdk.transforms.windowing.Window;
-import org.apache.beam.sdk.values.KV;
-import org.apache.beam.sdk.values.PCollection;
-import org.joda.time.Duration;
-
-/**
- * Query "11", 'User sessions' (Not in original suite.)
- *
- * <p>Group bids by the same user into sessions with {@code windowSizeSec} max gap.
- * However limit the session to at most {@code maxLogEvents}. Emit the number of
- * bids per session.
- */
-public class Query11 extends NexmarkQuery {
-  public Query11(NexmarkConfiguration configuration) {
-    super(configuration, "Query11");
-  }
-
-  private PCollection<BidsPerSession> applyTyped(PCollection<Event> events) {
-    PCollection<Long> bidders = events.apply(JUST_BIDS).apply(name + ".Rekey",
-        ParDo.of(new DoFn<Bid, Long>() {
-
-          @ProcessElement public void processElement(ProcessContext c) {
-            Bid bid = c.element();
-            c.output(bid.bidder);
-          }
-        }));
-
-    PCollection<Long> biddersWindowed = bidders.apply(
-        Window.<Long>into(
-          Sessions.withGapDuration(Duration.standardSeconds(configuration.windowSizeSec)))
-            .triggering(
-                Repeatedly.forever(AfterPane.elementCountAtLeast(configuration.maxLogEvents)))
-            .discardingFiredPanes()
-            .withAllowedLateness(Duration.standardSeconds(configuration.occasionalDelaySec / 2)));
-    return biddersWindowed.apply(Count.<Long>perElement())
-        .apply(name + ".ToResult", ParDo.of(new DoFn<KV<Long, Long>, BidsPerSession>() {
-
-          @ProcessElement public void processElement(ProcessContext c) {
-            c.output(new BidsPerSession(c.element().getKey(), c.element().getValue()));
-          }
-        }));
-  }
-
-  @Override
-  protected PCollection<KnownSize> applyPrim(PCollection<Event> events) {
-    return NexmarkUtils.castToKnownSize(name, applyTyped(events));
-  }
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/f4333df7/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query12.java
----------------------------------------------------------------------
diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query12.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query12.java
deleted file mode 100644
index 20f45fb..0000000
--- a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query12.java
+++ /dev/null
@@ -1,80 +0,0 @@
-/*
- * 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.beam.integration.nexmark.queries;
-
-import org.apache.beam.integration.nexmark.NexmarkConfiguration;
-import org.apache.beam.integration.nexmark.NexmarkUtils;
-import org.apache.beam.integration.nexmark.model.Bid;
-import org.apache.beam.integration.nexmark.model.BidsPerSession;
-import org.apache.beam.integration.nexmark.model.Event;
-import org.apache.beam.integration.nexmark.model.KnownSize;
-import org.apache.beam.sdk.transforms.Count;
-import org.apache.beam.sdk.transforms.DoFn;
-import org.apache.beam.sdk.transforms.ParDo;
-import org.apache.beam.sdk.transforms.windowing.AfterProcessingTime;
-import org.apache.beam.sdk.transforms.windowing.GlobalWindows;
-import org.apache.beam.sdk.transforms.windowing.Repeatedly;
-import org.apache.beam.sdk.transforms.windowing.Window;
-import org.apache.beam.sdk.values.KV;
-import org.apache.beam.sdk.values.PCollection;
-import org.joda.time.Duration;
-
-/**
- * Query "12", 'Processing time windows' (Not in original suite.)
- *
- * <p>Group bids by the same user into processing time windows of windowSize. Emit the count
- * of bids per window.
- */
-public class Query12 extends NexmarkQuery {
-  public Query12(NexmarkConfiguration configuration) {
-    super(configuration, "Query12");
-  }
-
-  private PCollection<BidsPerSession> applyTyped(PCollection<Event> events) {
-    return events
-        .apply(JUST_BIDS)
-        .apply(ParDo.of(new DoFn<Bid, Long>() {
-          @ProcessElement
-          public void processElement(ProcessContext c){
-            c.output(c.element().bidder);
-          }
-        }))
-        .apply(Window.<Long>into(new GlobalWindows())
-            .triggering(
-                Repeatedly.forever(
-                    AfterProcessingTime.pastFirstElementInPane()
-                                       .plusDelayOf(
-                                           Duration.standardSeconds(configuration.windowSizeSec))))
-            .discardingFiredPanes()
-            .withAllowedLateness(Duration.ZERO))
-        .apply(Count.<Long>perElement())
-        .apply(name + ".ToResult",
-            ParDo.of(new DoFn<KV<Long, Long>, BidsPerSession>() {
-                   @ProcessElement
-                   public void processElement(ProcessContext c) {
-                     c.output(
-                         new BidsPerSession(c.element().getKey(), c.element().getValue()));
-                   }
-                 }));
-  }
-
-  @Override
-  protected PCollection<KnownSize> applyPrim(PCollection<Event> events) {
-    return NexmarkUtils.castToKnownSize(name, applyTyped(events));
-  }
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/f4333df7/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query1Model.java
----------------------------------------------------------------------
diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query1Model.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query1Model.java
deleted file mode 100644
index f07db80..0000000
--- a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query1Model.java
+++ /dev/null
@@ -1,76 +0,0 @@
-/*
- * 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.beam.integration.nexmark.queries;
-
-import java.io.Serializable;
-import java.util.Collection;
-import java.util.Iterator;
-
-import org.apache.beam.integration.nexmark.NexmarkConfiguration;
-import org.apache.beam.integration.nexmark.NexmarkUtils;
-import org.apache.beam.integration.nexmark.model.Bid;
-import org.apache.beam.integration.nexmark.model.Event;
-import org.apache.beam.sdk.values.TimestampedValue;
-
-/**
- * A direct implementation of {@link Query1}.
- */
-public class Query1Model extends NexmarkQueryModel implements Serializable {
-  /**
-   * Simulator for query 1.
-   */
-  private static class Simulator extends AbstractSimulator<Event, Bid> {
-    public Simulator(NexmarkConfiguration configuration) {
-      super(NexmarkUtils.standardEventIterator(configuration));
-    }
-
-    @Override
-    protected void run() {
-      TimestampedValue<Event> timestampedEvent = nextInput();
-      if (timestampedEvent == null) {
-        allDone();
-        return;
-      }
-      Event event = timestampedEvent.getValue();
-      if (event.bid == null) {
-        // Ignore non-bid events.
-        return;
-      }
-      Bid bid = event.bid;
-      Bid resultBid =
-          new Bid(bid.auction, bid.bidder, bid.price * 89 / 100, bid.dateTime, bid.extra);
-      TimestampedValue<Bid> result =
-          TimestampedValue.of(resultBid, timestampedEvent.getTimestamp());
-      addResult(result);
-    }
-  }
-
-  public Query1Model(NexmarkConfiguration configuration) {
-    super(configuration);
-  }
-
-  @Override
-  public AbstractSimulator<?, ?> simulator() {
-    return new Simulator(configuration);
-  }
-
-  @Override
-  protected <T> Collection<String> toCollection(Iterator<TimestampedValue<T>> itr) {
-    return toValueTimestampOrder(itr);
-  }
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/f4333df7/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query2.java
----------------------------------------------------------------------
diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query2.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query2.java
deleted file mode 100644
index a365b97..0000000
--- a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query2.java
+++ /dev/null
@@ -1,79 +0,0 @@
-/*
- * 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.beam.integration.nexmark.queries;
-
-import org.apache.beam.integration.nexmark.NexmarkConfiguration;
-import org.apache.beam.integration.nexmark.NexmarkUtils;
-import org.apache.beam.integration.nexmark.model.AuctionPrice;
-import org.apache.beam.integration.nexmark.model.Bid;
-import org.apache.beam.integration.nexmark.model.Event;
-import org.apache.beam.integration.nexmark.model.KnownSize;
-import org.apache.beam.sdk.transforms.DoFn;
-import org.apache.beam.sdk.transforms.Filter;
-import org.apache.beam.sdk.transforms.ParDo;
-import org.apache.beam.sdk.transforms.SerializableFunction;
-import org.apache.beam.sdk.values.PCollection;
-
-/**
- * Query 2, 'Filtering. Find bids with specific auction ids and show their bid price.
- * In CQL syntax:
- *
- * <pre>
- * SELECT Rstream(auction, price)
- * FROM Bid [NOW]
- * WHERE auction = 1007 OR auction = 1020 OR auction = 2001 OR auction = 2019 OR auction = 2087;
- * </pre>
- *
- * <p>As written that query will only yield a few hundred results over event streams of
- * arbitrary size. To make it more interesting we instead choose bids for every
- * {@code auctionSkip}'th auction.
- */
-public class Query2 extends NexmarkQuery {
-  public Query2(NexmarkConfiguration configuration) {
-    super(configuration, "Query2");
-  }
-
-  private PCollection<AuctionPrice> applyTyped(PCollection<Event> events) {
-    return events
-        // Only want the bid events.
-        .apply(JUST_BIDS)
-
-        // Select just the bids for the auctions we care about.
-        .apply(Filter.by(new SerializableFunction<Bid, Boolean>() {
-          @Override
-          public Boolean apply(Bid bid) {
-            return bid.auction % configuration.auctionSkip == 0;
-          }
-        }))
-
-        // Project just auction id and price.
-        .apply(name + ".Project",
-            ParDo.of(new DoFn<Bid, AuctionPrice>() {
-                  @ProcessElement
-                  public void processElement(ProcessContext c) {
-                    Bid bid = c.element();
-                    c.output(new AuctionPrice(bid.auction, bid.price));
-                  }
-                }));
-  }
-
-  @Override
-  protected PCollection<KnownSize> applyPrim(PCollection<Event> events) {
-    return NexmarkUtils.castToKnownSize(name, applyTyped(events));
-  }
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/f4333df7/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query2Model.java
----------------------------------------------------------------------
diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query2Model.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query2Model.java
deleted file mode 100644
index e00992f..0000000
--- a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query2Model.java
+++ /dev/null
@@ -1,80 +0,0 @@
-/*
- * 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.beam.integration.nexmark.queries;
-
-import java.io.Serializable;
-import java.util.Collection;
-import java.util.Iterator;
-
-import org.apache.beam.integration.nexmark.NexmarkConfiguration;
-import org.apache.beam.integration.nexmark.NexmarkUtils;
-import org.apache.beam.integration.nexmark.model.AuctionPrice;
-import org.apache.beam.integration.nexmark.model.Bid;
-import org.apache.beam.integration.nexmark.model.Event;
-import org.apache.beam.sdk.values.TimestampedValue;
-
-/**
- * A direct implementation of {@link Query2}.
- */
-public class Query2Model extends NexmarkQueryModel implements Serializable {
-  /**
-   * Simulator for query 2.
-   */
-  private class Simulator extends AbstractSimulator<Event, AuctionPrice> {
-    public Simulator(NexmarkConfiguration configuration) {
-      super(NexmarkUtils.standardEventIterator(configuration));
-    }
-
-    @Override
-    protected void run() {
-      TimestampedValue<Event> timestampedEvent = nextInput();
-      if (timestampedEvent == null) {
-        allDone();
-        return;
-      }
-      Event event = timestampedEvent.getValue();
-      if (event.bid == null) {
-        // Ignore non bid events.
-        return;
-      }
-      Bid bid = event.bid;
-      if (bid.auction % configuration.auctionSkip != 0) {
-        // Ignore bids for auctions we don't care about.
-        return;
-      }
-      AuctionPrice auctionPrice = new AuctionPrice(bid.auction, bid.price);
-      TimestampedValue<AuctionPrice> result =
-          TimestampedValue.of(auctionPrice, timestampedEvent.getTimestamp());
-      addResult(result);
-    }
-  }
-
-  public Query2Model(NexmarkConfiguration configuration) {
-    super(configuration);
-  }
-
-  @Override
-  public AbstractSimulator<?, ?> simulator() {
-    return new Simulator(configuration);
-  }
-
-  @Override
-  protected <T> Collection<String> toCollection(Iterator<TimestampedValue<T>> itr) {
-    return toValueTimestampOrder(itr);
-  }
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/f4333df7/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query3.java
----------------------------------------------------------------------
diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query3.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query3.java
deleted file mode 100644
index f2b66d7..0000000
--- a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query3.java
+++ /dev/null
@@ -1,301 +0,0 @@
-/*
- * 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.beam.integration.nexmark.queries;
-
-import java.util.ArrayList;
-import java.util.List;
-import org.apache.beam.integration.nexmark.NexmarkConfiguration;
-import org.apache.beam.integration.nexmark.NexmarkUtils;
-import org.apache.beam.integration.nexmark.model.Auction;
-import org.apache.beam.integration.nexmark.model.Event;
-import org.apache.beam.integration.nexmark.model.KnownSize;
-import org.apache.beam.integration.nexmark.model.NameCityStateId;
-import org.apache.beam.integration.nexmark.model.Person;
-import org.apache.beam.sdk.coders.ListCoder;
-import org.apache.beam.sdk.metrics.Counter;
-import org.apache.beam.sdk.metrics.Metrics;
-import org.apache.beam.sdk.state.StateSpec;
-import org.apache.beam.sdk.state.StateSpecs;
-import org.apache.beam.sdk.state.TimeDomain;
-import org.apache.beam.sdk.state.Timer;
-import org.apache.beam.sdk.state.TimerSpec;
-import org.apache.beam.sdk.state.TimerSpecs;
-import org.apache.beam.sdk.state.ValueState;
-import org.apache.beam.sdk.transforms.DoFn;
-import org.apache.beam.sdk.transforms.Filter;
-import org.apache.beam.sdk.transforms.ParDo;
-import org.apache.beam.sdk.transforms.SerializableFunction;
-import org.apache.beam.sdk.transforms.join.CoGbkResult;
-import org.apache.beam.sdk.transforms.join.CoGroupByKey;
-import org.apache.beam.sdk.transforms.join.KeyedPCollectionTuple;
-import org.apache.beam.sdk.transforms.windowing.AfterPane;
-import org.apache.beam.sdk.transforms.windowing.GlobalWindows;
-import org.apache.beam.sdk.transforms.windowing.Repeatedly;
-import org.apache.beam.sdk.transforms.windowing.Window;
-import org.apache.beam.sdk.values.KV;
-import org.apache.beam.sdk.values.PCollection;
-import org.joda.time.Duration;
-import org.joda.time.Instant;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-/**
- * Query 3, 'Local Item Suggestion'. Who is selling in OR, ID or CA in category 10, and for what
- * auction ids? In CQL syntax:
- *
- * <pre>
- * SELECT Istream(P.name, P.city, P.state, A.id)
- * FROM Auction A [ROWS UNBOUNDED], Person P [ROWS UNBOUNDED]
- * WHERE A.seller = P.id AND (P.state = `OR' OR P.state = `ID' OR P.state = `CA') AND A.category
- * = 10;
- * </pre>
- *
- * <p>We'll implement this query to allow 'new auction' events to come before the 'new person'
- * events for the auction seller. Those auctions will be stored until the matching person is seen.
- * Then all subsequent auctions for a person will use the stored person record.
- *
- * <p>A real system would use an external system to maintain the id-to-person association.
- */
-public class Query3 extends NexmarkQuery {
-
-  private static final Logger LOG = LoggerFactory.getLogger(Query3.class);
-  private final JoinDoFn joinDoFn;
-
-  public Query3(NexmarkConfiguration configuration) {
-    super(configuration, "Query3");
-    joinDoFn = new JoinDoFn(name, configuration.maxAuctionsWaitingTime);
-  }
-
-  private PCollection<NameCityStateId> applyTyped(PCollection<Event> events) {
-    int numEventsInPane = 30;
-
-    PCollection<Event> eventsWindowed =
-        events.apply(
-            Window.<Event>into(new GlobalWindows())
-                .triggering(Repeatedly.forever((AfterPane.elementCountAtLeast(numEventsInPane))))
-                .discardingFiredPanes()
-                .withAllowedLateness(Duration.ZERO));
-    PCollection<KV<Long, Auction>> auctionsBySellerId =
-        eventsWindowed
-            // Only want the new auction events.
-            .apply(JUST_NEW_AUCTIONS)
-
-            // We only want auctions in category 10.
-            .apply(
-                name + ".InCategory",
-                Filter.by(
-                    new SerializableFunction<Auction, Boolean>() {
-
-                      @Override
-                      public Boolean apply(Auction auction) {
-                        return auction.category == 10;
-                      }
-                    }))
-
-            // Key auctions by their seller id.
-            .apply("AuctionBySeller", AUCTION_BY_SELLER);
-
-    PCollection<KV<Long, Person>> personsById =
-        eventsWindowed
-            // Only want the new people events.
-            .apply(JUST_NEW_PERSONS)
-
-            // We only want people in OR, ID, CA.
-            .apply(
-                name + ".InState",
-                Filter.by(
-                    new SerializableFunction<Person, Boolean>() {
-
-                      @Override
-                      public Boolean apply(Person person) {
-                        return person.state.equals("OR")
-                            || person.state.equals("ID")
-                            || person.state.equals("CA");
-                      }
-                    }))
-
-            // Key people by their id.
-            .apply("PersonById", PERSON_BY_ID);
-
-    return
-    // Join auctions and people.
-    // concatenate KeyedPCollections
-    KeyedPCollectionTuple.of(AUCTION_TAG, auctionsBySellerId)
-        .and(PERSON_TAG, personsById)
-        // group auctions and persons by personId
-        .apply(CoGroupByKey.<Long>create())
-        .apply(name + ".Join", ParDo.of(joinDoFn))
-
-        // Project what we want.
-        .apply(
-            name + ".Project",
-            ParDo.of(
-                new DoFn<KV<Auction, Person>, NameCityStateId>() {
-
-                  @ProcessElement
-                  public void processElement(ProcessContext c) {
-                    Auction auction = c.element().getKey();
-                    Person person = c.element().getValue();
-                    c.output(
-                        new NameCityStateId(person.name, person.city, person.state, auction.id));
-                  }
-                }));
-  }
-
-  @Override
-  protected PCollection<KnownSize> applyPrim(PCollection<Event> events) {
-    return NexmarkUtils.castToKnownSize(name, applyTyped(events));
-  }
-
-  /**
-   * Join {@code auctions} and {@code people} by person id and emit their cross-product one pair at
-   * a time.
-   *
-   * <p>We know a person may submit any number of auctions. Thus new person event must have the
-   * person record stored in persistent state in order to match future auctions by that person.
-   *
-   * <p>However we know that each auction is associated with at most one person, so only need to
-   * store auction records in persistent state until we have seen the corresponding person record.
-   * And of course may have already seen that record.
-   */
-  private static class JoinDoFn extends DoFn<KV<Long, CoGbkResult>, KV<Auction, Person>> {
-
-    private final int maxAuctionsWaitingTime;
-    private static final String AUCTIONS = "auctions";
-    private static final String PERSON = "person";
-
-    @StateId(PERSON)
-    private static final StateSpec<ValueState<Person>> personSpec =
-        StateSpecs.value(Person.CODER);
-
-    private static final String PERSON_STATE_EXPIRING = "personStateExpiring";
-
-    @StateId(AUCTIONS)
-    private final StateSpec<ValueState<List<Auction>>> auctionsSpec =
-        StateSpecs.value(ListCoder.of(Auction.CODER));
-
-    @TimerId(PERSON_STATE_EXPIRING)
-    private final TimerSpec timerSpec = TimerSpecs.timer(TimeDomain.EVENT_TIME);
-
-    // Used to refer the metrics namespace
-    private final String name;
-
-    private final Counter newAuctionCounter;
-    private final Counter newPersonCounter;
-    private final Counter newNewOutputCounter;
-    private final Counter newOldOutputCounter;
-    private final Counter oldNewOutputCounter;
-    private final Counter fatalCounter;
-
-    private JoinDoFn(String name, int maxAuctionsWaitingTime) {
-      this.name = name;
-      this.maxAuctionsWaitingTime = maxAuctionsWaitingTime;
-      newAuctionCounter = Metrics.counter(name, "newAuction");
-      newPersonCounter = Metrics.counter(name, "newPerson");
-      newNewOutputCounter = Metrics.counter(name, "newNewOutput");
-      newOldOutputCounter = Metrics.counter(name, "newOldOutput");
-      oldNewOutputCounter = Metrics.counter(name, "oldNewOutput");
-      fatalCounter = Metrics.counter(name , "fatal");
-    }
-
-    @ProcessElement
-    public void processElement(
-        ProcessContext c,
-        @TimerId(PERSON_STATE_EXPIRING) Timer timer,
-        @StateId(PERSON) ValueState<Person> personState,
-        @StateId(AUCTIONS) ValueState<List<Auction>> auctionsState) {
-      // We would *almost* implement this by  rewindowing into the global window and
-      // running a combiner over the result. The combiner's accumulator would be the
-      // state we use below. However, combiners cannot emit intermediate results, thus
-      // we need to wait for the pending ReduceFn API.
-
-      Person existingPerson = personState.read();
-      if (existingPerson != null) {
-        // We've already seen the new person event for this person id.
-        // We can join with any new auctions on-the-fly without needing any
-        // additional persistent state.
-        for (Auction newAuction : c.element().getValue().getAll(AUCTION_TAG)) {
-          newAuctionCounter.inc();
-          newOldOutputCounter.inc();
-          c.output(KV.of(newAuction, existingPerson));
-        }
-        return;
-      }
-
-      Person theNewPerson = null;
-      for (Person newPerson : c.element().getValue().getAll(PERSON_TAG)) {
-        if (theNewPerson == null) {
-          theNewPerson = newPerson;
-        } else {
-          if (theNewPerson.equals(newPerson)) {
-            LOG.error("Duplicate person {}", theNewPerson);
-          } else {
-            LOG.error("Conflicting persons {} and {}", theNewPerson, newPerson);
-          }
-          fatalCounter.inc();
-          continue;
-        }
-        newPersonCounter.inc();
-        // We've now seen the person for this person id so can flush any
-        // pending auctions for the same seller id (an auction is done by only one seller).
-        List<Auction> pendingAuctions = auctionsState.read();
-        if (pendingAuctions != null) {
-          for (Auction pendingAuction : pendingAuctions) {
-            oldNewOutputCounter.inc();
-            c.output(KV.of(pendingAuction, newPerson));
-          }
-          auctionsState.clear();
-        }
-        // Also deal with any new auctions.
-        for (Auction newAuction : c.element().getValue().getAll(AUCTION_TAG)) {
-          newAuctionCounter.inc();
-          newNewOutputCounter.inc();
-          c.output(KV.of(newAuction, newPerson));
-        }
-        // Remember this person for any future auctions.
-        personState.write(newPerson);
-        //set a time out to clear this state
-        Instant firingTime = new Instant(newPerson.dateTime)
-                                  .plus(Duration.standardSeconds(maxAuctionsWaitingTime));
-        timer.set(firingTime);
-      }
-      if (theNewPerson != null) {
-        return;
-      }
-
-      // We'll need to remember the auctions until we see the corresponding
-      // new person event.
-      List<Auction> pendingAuctions = auctionsState.read();
-      if (pendingAuctions == null) {
-        pendingAuctions = new ArrayList<>();
-      }
-      for (Auction newAuction : c.element().getValue().getAll(AUCTION_TAG)) {
-        newAuctionCounter.inc();
-        pendingAuctions.add(newAuction);
-      }
-      auctionsState.write(pendingAuctions);
-    }
-
-    @OnTimer(PERSON_STATE_EXPIRING)
-    public void onTimerCallback(
-        OnTimerContext context,
-        @StateId(PERSON) ValueState<Person> personState) {
-        personState.clear();
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/f4333df7/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query3Model.java
----------------------------------------------------------------------
diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query3Model.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query3Model.java
deleted file mode 100644
index f415709..0000000
--- a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query3Model.java
+++ /dev/null
@@ -1,124 +0,0 @@
-/*
- * 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.beam.integration.nexmark.queries;
-
-import com.google.common.collect.ArrayListMultimap;
-import com.google.common.collect.Multimap;
-
-import java.io.Serializable;
-import java.util.Collection;
-import java.util.HashMap;
-import java.util.Iterator;
-import java.util.Map;
-
-import org.apache.beam.integration.nexmark.NexmarkConfiguration;
-import org.apache.beam.integration.nexmark.NexmarkUtils;
-import org.apache.beam.integration.nexmark.model.Auction;
-import org.apache.beam.integration.nexmark.model.Event;
-import org.apache.beam.integration.nexmark.model.NameCityStateId;
-import org.apache.beam.integration.nexmark.model.Person;
-import org.apache.beam.sdk.values.TimestampedValue;
-import org.joda.time.Instant;
-
-/**
- * A direct implementation of {@link Query3}.
- */
-public class Query3Model extends NexmarkQueryModel implements Serializable {
-  /**
-   * Simulator for query 3.
-   */
-  private static class Simulator extends AbstractSimulator<Event, NameCityStateId> {
-    /** Auctions, indexed by seller id. */
-    private final Multimap<Long, Auction> newAuctions;
-
-    /** Persons, indexed by id. */
-    private final Map<Long, Person> newPersons;
-
-    public Simulator(NexmarkConfiguration configuration) {
-      super(NexmarkUtils.standardEventIterator(configuration));
-      newPersons = new HashMap<>();
-      newAuctions = ArrayListMultimap.create();
-    }
-
-    /**
-     * Capture new result.
-     */
-    private void addResult(Auction auction, Person person, Instant timestamp) {
-      TimestampedValue<NameCityStateId> result = TimestampedValue.of(
-          new NameCityStateId(person.name, person.city, person.state, auction.id), timestamp);
-      addResult(result);
-    }
-
-    @Override
-    protected void run() {
-      TimestampedValue<Event> timestampedEvent = nextInput();
-      if (timestampedEvent == null) {
-        allDone();
-        return;
-      }
-      Event event = timestampedEvent.getValue();
-      if (event.bid != null) {
-        // Ignore bid events.
-        return;
-      }
-
-      Instant timestamp = timestampedEvent.getTimestamp();
-
-      if (event.newAuction != null) {
-        // Only want auctions in category 10.
-        if (event.newAuction.category == 10) {
-          // Join new auction with existing person, if any.
-          Person person = newPersons.get(event.newAuction.seller);
-          if (person != null) {
-            addResult(event.newAuction, person, timestamp);
-          } else {
-            // Remember auction for future new person event.
-            newAuctions.put(event.newAuction.seller, event.newAuction);
-          }
-        }
-      } else {
-        // Only want people in OR, ID or CA.
-        if (event.newPerson.state.equals("OR") || event.newPerson.state.equals("ID")
-            || event.newPerson.state.equals("CA")) {
-          // Join new person with existing auctions.
-          for (Auction auction : newAuctions.get(event.newPerson.id)) {
-            addResult(auction, event.newPerson, timestamp);
-          }
-          // We'll never need these auctions again.
-          newAuctions.removeAll(event.newPerson.id);
-          // Remember person for future auctions.
-          newPersons.put(event.newPerson.id, event.newPerson);
-        }
-      }
-    }
-  }
-
-  public Query3Model(NexmarkConfiguration configuration) {
-    super(configuration);
-  }
-
-  @Override
-  public AbstractSimulator<?, ?> simulator() {
-    return new Simulator(configuration);
-  }
-
-  @Override
-  protected <T> Collection<String> toCollection(Iterator<TimestampedValue<T>> itr) {
-    return toValue(itr);
-  }
-}


[14/55] [abbrv] beam git commit: Refactor classes into packages

Posted by ie...@apache.org.
http://git-wip-us.apache.org/repos/asf/beam/blob/a7f9f7d0/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkRunner.java
----------------------------------------------------------------------
diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkRunner.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkRunner.java
index d311dc4..e8d791f 100644
--- a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkRunner.java
+++ b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkRunner.java
@@ -26,7 +26,6 @@ import com.google.common.base.Strings;
 import com.google.common.collect.ImmutableList;
 import com.google.common.collect.Iterables;
 import com.google.common.collect.Lists;
-
 import java.io.IOException;
 import java.util.ArrayList;
 import java.util.Arrays;
@@ -35,6 +34,35 @@ import java.util.Collections;
 import java.util.List;
 import java.util.concurrent.ThreadLocalRandom;
 import javax.annotation.Nullable;
+import org.apache.beam.integration.nexmark.io.PubsubHelper;
+import org.apache.beam.integration.nexmark.model.Auction;
+import org.apache.beam.integration.nexmark.model.Bid;
+import org.apache.beam.integration.nexmark.model.Event;
+import org.apache.beam.integration.nexmark.model.KnownSize;
+import org.apache.beam.integration.nexmark.model.Person;
+import org.apache.beam.integration.nexmark.queries.Query0;
+import org.apache.beam.integration.nexmark.queries.Query0Model;
+import org.apache.beam.integration.nexmark.queries.Query1;
+import org.apache.beam.integration.nexmark.queries.Query10;
+import org.apache.beam.integration.nexmark.queries.Query11;
+import org.apache.beam.integration.nexmark.queries.Query12;
+import org.apache.beam.integration.nexmark.queries.Query1Model;
+import org.apache.beam.integration.nexmark.queries.Query2;
+import org.apache.beam.integration.nexmark.queries.Query2Model;
+import org.apache.beam.integration.nexmark.queries.Query3;
+import org.apache.beam.integration.nexmark.queries.Query3Model;
+import org.apache.beam.integration.nexmark.queries.Query4;
+import org.apache.beam.integration.nexmark.queries.Query4Model;
+import org.apache.beam.integration.nexmark.queries.Query5;
+import org.apache.beam.integration.nexmark.queries.Query5Model;
+import org.apache.beam.integration.nexmark.queries.Query6;
+import org.apache.beam.integration.nexmark.queries.Query6Model;
+import org.apache.beam.integration.nexmark.queries.Query7;
+import org.apache.beam.integration.nexmark.queries.Query7Model;
+import org.apache.beam.integration.nexmark.queries.Query8;
+import org.apache.beam.integration.nexmark.queries.Query8Model;
+import org.apache.beam.integration.nexmark.queries.Query9;
+import org.apache.beam.integration.nexmark.queries.Query9Model;
 import org.apache.beam.sdk.AggregatorRetrievalException;
 import org.apache.beam.sdk.Pipeline;
 import org.apache.beam.sdk.PipelineResult;
@@ -56,7 +84,7 @@ import org.joda.time.Duration;
 /**
  * Run a single Nexmark query using a given configuration.
  */
-public abstract class NexmarkRunner<OptionT extends Options> {
+public abstract class NexmarkRunner<OptionT extends NexmarkOptions> {
   /**
    * Minimum number of samples needed for 'stead-state' rate calculation.
    */
@@ -84,7 +112,7 @@ public abstract class NexmarkRunner<OptionT extends Options> {
    */
   private static final Duration STUCK_TERMINATE_DELAY = Duration.standardDays(3);
   /**
-   * Options shared by all runs.
+   * NexmarkOptions shared by all runs.
    */
   protected final OptionT options;
 
@@ -359,7 +387,7 @@ public abstract class NexmarkRunner<OptionT extends Options> {
     return perf;
   }
 
-  String getJobId(PipelineResult job) {
+  protected String getJobId(PipelineResult job) {
     return "";
   }
 
@@ -461,7 +489,7 @@ public abstract class NexmarkRunner<OptionT extends Options> {
   /**
    * Build and run a pipeline using specified options.
    */
-  protected interface PipelineBuilder<OptionT extends Options> {
+  protected interface PipelineBuilder<OptionT extends NexmarkOptions> {
     void build(OptionT publishOnlyOptions);
   }
 
@@ -966,7 +994,7 @@ public abstract class NexmarkRunner<OptionT extends Options> {
             // We'll shutdown the publisher job when we notice the main job has finished.
             invokeBuilderForPublishOnlyPipeline(new PipelineBuilder() {
               @Override
-              public void build(Options publishOnlyOptions) {
+              public void build(NexmarkOptions publishOnlyOptions) {
                 Pipeline sp = Pipeline.create(options);
                 NexmarkUtils.setupPipeline(configuration.coderStrategy, sp);
                 publisherMonitor = new Monitor<Event>(queryName, "publisher");

http://git-wip-us.apache.org/repos/asf/beam/blob/a7f9f7d0/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkSparkDriver.java
----------------------------------------------------------------------
diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkSparkDriver.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkSparkDriver.java
deleted file mode 100644
index a46d38a..0000000
--- a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkSparkDriver.java
+++ /dev/null
@@ -1,46 +0,0 @@
-/*
- * 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.beam.integration.nexmark;
-
-import org.apache.beam.runners.spark.SparkPipelineOptions;
-import org.apache.beam.sdk.options.PipelineOptionsFactory;
-
-/**
- * Run NexMark queries using the Spark runner.
- */
-class NexmarkSparkDriver extends NexmarkDriver<NexmarkSparkDriver.NexmarkSparkOptions> {
-    /**
-     * Command line flags.
-     */
-    public interface NexmarkSparkOptions extends Options, SparkPipelineOptions {
-    }
-
-    /**
-     * Entry point.
-     */
-    public static void main(String[] args) {
-        NexmarkSparkOptions options =
-                PipelineOptionsFactory.fromArgs(args)
-                        .withValidation()
-                        .as(NexmarkSparkOptions.class);
-//        options.setRunner(org.apache.beam.runners.spark.SparkRunner.class);
-        options.setRunner(org.apache.beam.runners.spark.SparkRunnerDebugger.class);
-        NexmarkSparkRunner runner = new NexmarkSparkRunner(options);
-        new NexmarkSparkDriver().runAll(options, runner);
-    }
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/a7f9f7d0/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkSparkRunner.java
----------------------------------------------------------------------
diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkSparkRunner.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkSparkRunner.java
deleted file mode 100644
index 30ae9ca..0000000
--- a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkSparkRunner.java
+++ /dev/null
@@ -1,54 +0,0 @@
-/*
- * 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.beam.integration.nexmark;
-
-/**
- * Run a query using the Spark runner.
- */
-public class NexmarkSparkRunner extends NexmarkRunner<NexmarkSparkDriver.NexmarkSparkOptions> {
-    @Override
-    protected boolean isStreaming() {
-        return options.isStreaming();
-    }
-
-    @Override
-    protected int coresPerWorker() {
-        return 4;
-    }
-
-    @Override
-    protected int maxNumWorkers() {
-        return 5;
-    }
-
-    @Override
-    protected void invokeBuilderForPublishOnlyPipeline(
-            PipelineBuilder builder) {
-        builder.build(options);
-    }
-
-    @Override
-    protected void waitForPublisherPreload() {
-        throw new UnsupportedOperationException();
-    }
-
-
-    public NexmarkSparkRunner(NexmarkSparkDriver.NexmarkSparkOptions options) {
-        super(options);
-    }
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/a7f9f7d0/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkSuite.java
----------------------------------------------------------------------
diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkSuite.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkSuite.java
index bc47540..be7d7b8 100644
--- a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkSuite.java
+++ b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkSuite.java
@@ -100,7 +100,7 @@ public enum NexmarkSuite {
    * with any set command line flags, except for --isStreaming which is only respected for
    * the {@link #DEFAULT} suite.
    */
-  public Iterable<NexmarkConfiguration> getConfigurations(Options options) {
+  public Iterable<NexmarkConfiguration> getConfigurations(NexmarkOptions options) {
     Set<NexmarkConfiguration> results = new LinkedHashSet<>();
     for (NexmarkConfiguration configuration : configurations) {
       NexmarkConfiguration result = configuration.clone();

http://git-wip-us.apache.org/repos/asf/beam/blob/a7f9f7d0/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkUtils.java
----------------------------------------------------------------------
diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkUtils.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkUtils.java
index f7417d3..b0421a4 100644
--- a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkUtils.java
+++ b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkUtils.java
@@ -20,14 +20,30 @@ package org.apache.beam.integration.nexmark;
 import com.fasterxml.jackson.databind.ObjectMapper;
 import com.google.common.collect.ImmutableList;
 import com.google.common.hash.Hashing;
-
 import java.io.IOException;
 import java.io.InputStream;
 import java.io.OutputStream;
 import java.nio.charset.StandardCharsets;
 import java.util.Iterator;
 import java.util.List;
-
+import org.apache.beam.integration.nexmark.model.Auction;
+import org.apache.beam.integration.nexmark.model.AuctionBid;
+import org.apache.beam.integration.nexmark.model.AuctionCount;
+import org.apache.beam.integration.nexmark.model.AuctionPrice;
+import org.apache.beam.integration.nexmark.model.Bid;
+import org.apache.beam.integration.nexmark.model.BidsPerSession;
+import org.apache.beam.integration.nexmark.model.CategoryPrice;
+import org.apache.beam.integration.nexmark.model.Done;
+import org.apache.beam.integration.nexmark.model.Event;
+import org.apache.beam.integration.nexmark.model.IdNameReserve;
+import org.apache.beam.integration.nexmark.model.KnownSize;
+import org.apache.beam.integration.nexmark.model.NameCityStateId;
+import org.apache.beam.integration.nexmark.model.Person;
+import org.apache.beam.integration.nexmark.model.SellerPrice;
+import org.apache.beam.integration.nexmark.sources.BoundedEventSource;
+import org.apache.beam.integration.nexmark.sources.Generator;
+import org.apache.beam.integration.nexmark.sources.GeneratorConfig;
+import org.apache.beam.integration.nexmark.sources.UnboundedEventSource;
 import org.apache.beam.sdk.Pipeline;
 import org.apache.beam.sdk.coders.AvroCoder;
 import org.apache.beam.sdk.coders.ByteArrayCoder;
@@ -67,7 +83,7 @@ public class NexmarkUtils {
   /**
    * Mapper for (de)serializing JSON.
    */
-  static final ObjectMapper MAPPER = new ObjectMapper();
+  public static final ObjectMapper MAPPER = new ObjectMapper();
 
   /**
    * Possible sources for events.
@@ -382,7 +398,8 @@ public class NexmarkUtils {
    */
   public static PTransform<PBegin, PCollection<Event>> batchEventsSource(
           NexmarkConfiguration configuration) {
-    return Read.from(new BoundedEventSource(standardGeneratorConfig(configuration), configuration.numEventGenerators));
+    return Read.from(new BoundedEventSource(standardGeneratorConfig(configuration),
+      configuration.numEventGenerators));
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/beam/blob/a7f9f7d0/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/Options.java
----------------------------------------------------------------------
diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/Options.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/Options.java
deleted file mode 100644
index 388473d..0000000
--- a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/Options.java
+++ /dev/null
@@ -1,386 +0,0 @@
-/*
- * 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.beam.integration.nexmark;
-
-import javax.annotation.Nullable;
-
-import org.apache.beam.sdk.options.Default;
-import org.apache.beam.sdk.options.Description;
-import org.apache.beam.sdk.options.PubsubOptions;
-
-/**
- * Command line flags.
- */
-public interface Options extends PubsubOptions {
-  @Description("Which suite to run. Default is to use command line arguments for one job.")
-  @Default.Enum("DEFAULT")
-  NexmarkSuite getSuite();
-
-  void setSuite(NexmarkSuite suite);
-
-  @Description("If true, and using the DataflowPipelineRunner, monitor the jobs as they run.")
-  @Default.Boolean(false)
-  boolean getMonitorJobs();
-
-  void setMonitorJobs(boolean monitorJobs);
-
-  @Description("Where the events come from.")
-  @Nullable
-  NexmarkUtils.SourceType getSourceType();
-
-  void setSourceType(NexmarkUtils.SourceType sourceType);
-
-  @Description("Prefix for input files if using avro input")
-  @Nullable
-  String getInputPath();
-
-  void setInputPath(String inputPath);
-
-  @Description("Where results go.")
-  @Nullable
-  NexmarkUtils.SinkType getSinkType();
-
-  void setSinkType(NexmarkUtils.SinkType sinkType);
-
-  @Description("Which mode to run in when source is PUBSUB.")
-  @Nullable
-  NexmarkUtils.PubSubMode getPubSubMode();
-
-  void setPubSubMode(NexmarkUtils.PubSubMode pubSubMode);
-
-  @Description("Which query to run.")
-  @Nullable
-  Integer getQuery();
-
-  void setQuery(Integer query);
-
-  @Description("Prefix for output files if using text output for results or running Query 10.")
-  @Nullable
-  String getOutputPath();
-
-  void setOutputPath(String outputPath);
-
-  @Description("Base name of pubsub topic to publish to in streaming mode.")
-  @Nullable
-  @Default.String("nexmark")
-  String getPubsubTopic();
-
-  void setPubsubTopic(String pubsubTopic);
-
-  @Description("Base name of pubsub subscription to read from in streaming mode.")
-  @Nullable
-  @Default.String("nexmark")
-  String getPubsubSubscription();
-
-  void setPubsubSubscription(String pubsubSubscription);
-
-  @Description("Base name of BigQuery table name if using BigQuery output.")
-  @Nullable
-  @Default.String("nexmark")
-  String getBigQueryTable();
-
-  void setBigQueryTable(String bigQueryTable);
-
-  @Description("Approximate number of events to generate. "
-               + "Zero for effectively unlimited in streaming mode.")
-  @Nullable
-  Long getNumEvents();
-
-  void setNumEvents(Long numEvents);
-
-  @Description("Time in seconds to preload the subscription with data, at the initial input rate "
-               + "of the pipeline.")
-  @Nullable
-  Integer getPreloadSeconds();
-
-  void setPreloadSeconds(Integer preloadSeconds);
-
-  @Description("Number of unbounded sources to create events.")
-  @Nullable
-  Integer getNumEventGenerators();
-
-  void setNumEventGenerators(Integer numEventGenerators);
-
-  @Description("Shape of event rate curve.")
-  @Nullable
-  NexmarkUtils.RateShape getRateShape();
-
-  void setRateShape(NexmarkUtils.RateShape rateShape);
-
-  @Description("Initial overall event rate (in --rateUnit).")
-  @Nullable
-  Integer getFirstEventRate();
-
-  void setFirstEventRate(Integer firstEventRate);
-
-  @Description("Next overall event rate (in --rateUnit).")
-  @Nullable
-  Integer getNextEventRate();
-
-  void setNextEventRate(Integer nextEventRate);
-
-  @Description("Unit for rates.")
-  @Nullable
-  NexmarkUtils.RateUnit getRateUnit();
-
-  void setRateUnit(NexmarkUtils.RateUnit rateUnit);
-
-  @Description("Overall period of rate shape, in seconds.")
-  @Nullable
-  Integer getRatePeriodSec();
-
-  void setRatePeriodSec(Integer ratePeriodSec);
-
-  @Description("If true, relay events in real time in streaming mode.")
-  @Nullable
-  Boolean getIsRateLimited();
-
-  void setIsRateLimited(Boolean isRateLimited);
-
-  @Description("If true, use wallclock time as event time. Otherwise, use a deterministic"
-               + " time in the past so that multiple runs will see exactly the same event streams"
-               + " and should thus have exactly the same results.")
-  @Nullable
-  Boolean getUseWallclockEventTime();
-
-  void setUseWallclockEventTime(Boolean useWallclockEventTime);
-
-  @Description("Assert pipeline results match model results.")
-  @Nullable
-  boolean getAssertCorrectness();
-
-  void setAssertCorrectness(boolean assertCorrectness);
-
-  @Description("Log all input events.")
-  @Nullable
-  boolean getLogEvents();
-
-  void setLogEvents(boolean logEvents);
-
-  @Description("Log all query results.")
-  @Nullable
-  boolean getLogResults();
-
-  void setLogResults(boolean logResults);
-
-  @Description("Average size in bytes for a person record.")
-  @Nullable
-  Integer getAvgPersonByteSize();
-
-  void setAvgPersonByteSize(Integer avgPersonByteSize);
-
-  @Description("Average size in bytes for an auction record.")
-  @Nullable
-  Integer getAvgAuctionByteSize();
-
-  void setAvgAuctionByteSize(Integer avgAuctionByteSize);
-
-  @Description("Average size in bytes for a bid record.")
-  @Nullable
-  Integer getAvgBidByteSize();
-
-  void setAvgBidByteSize(Integer avgBidByteSize);
-
-  @Description("Ratio of bids for 'hot' auctions above the background.")
-  @Nullable
-  Integer getHotAuctionRatio();
-
-  void setHotAuctionRatio(Integer hotAuctionRatio);
-
-  @Description("Ratio of auctions for 'hot' sellers above the background.")
-  @Nullable
-  Integer getHotSellersRatio();
-
-  void setHotSellersRatio(Integer hotSellersRatio);
-
-  @Description("Ratio of auctions for 'hot' bidders above the background.")
-  @Nullable
-  Integer getHotBiddersRatio();
-
-  void setHotBiddersRatio(Integer hotBiddersRatio);
-
-  @Description("Window size in seconds.")
-  @Nullable
-  Long getWindowSizeSec();
-
-  void setWindowSizeSec(Long windowSizeSec);
-
-  @Description("Window period in seconds.")
-  @Nullable
-  Long getWindowPeriodSec();
-
-  void setWindowPeriodSec(Long windowPeriodSec);
-
-  @Description("If in streaming mode, the holdback for watermark in seconds.")
-  @Nullable
-  Long getWatermarkHoldbackSec();
-
-  void setWatermarkHoldbackSec(Long watermarkHoldbackSec);
-
-  @Description("Roughly how many auctions should be in flight for each generator.")
-  @Nullable
-  Integer getNumInFlightAuctions();
-
-  void setNumInFlightAuctions(Integer numInFlightAuctions);
-
-
-  @Description("Maximum number of people to consider as active for placing auctions or bids.")
-  @Nullable
-  Integer getNumActivePeople();
-
-  void setNumActivePeople(Integer numActivePeople);
-
-  @Description("Filename of perf data to append to.")
-  @Nullable
-  String getPerfFilename();
-
-  void setPerfFilename(String perfFilename);
-
-  @Description("Filename of baseline perf data to read from.")
-  @Nullable
-  String getBaselineFilename();
-
-  void setBaselineFilename(String baselineFilename);
-
-  @Description("Filename of summary perf data to append to.")
-  @Nullable
-  String getSummaryFilename();
-
-  void setSummaryFilename(String summaryFilename);
-
-  @Description("Filename for javascript capturing all perf data and any baselines.")
-  @Nullable
-  String getJavascriptFilename();
-
-  void setJavascriptFilename(String javascriptFilename);
-
-  @Description("If true, don't run the actual query. Instead, calculate the distribution "
-               + "of number of query results per (event time) minute according to the query model.")
-  @Nullable
-  boolean getJustModelResultRate();
-
-  void setJustModelResultRate(boolean justModelResultRate);
-
-  @Description("Coder strategy to use.")
-  @Nullable
-  NexmarkUtils.CoderStrategy getCoderStrategy();
-
-  void setCoderStrategy(NexmarkUtils.CoderStrategy coderStrategy);
-
-  @Description("Delay, in milliseconds, for each event. We will peg one core for this "
-               + "number of milliseconds to simulate CPU-bound computation.")
-  @Nullable
-  Long getCpuDelayMs();
-
-  void setCpuDelayMs(Long cpuDelayMs);
-
-  @Description("Extra data, in bytes, to save to persistent state for each event. "
-               + "This will force I/O all the way to durable storage to simulate an "
-               + "I/O-bound computation.")
-  @Nullable
-  Long getDiskBusyBytes();
-
-  void setDiskBusyBytes(Long diskBusyBytes);
-
-  @Description("Skip factor for query 2. We select bids for every {@code auctionSkip}'th auction")
-  @Nullable
-  Integer getAuctionSkip();
-
-  void setAuctionSkip(Integer auctionSkip);
-
-  @Description("Fanout for queries 4 (groups by category id) and 7 (finds a global maximum).")
-  @Nullable
-  Integer getFanout();
-
-  void setFanout(Integer fanout);
-
-  @Description("Length of occasional delay to impose on events (in seconds).")
-  @Nullable
-  Long getOccasionalDelaySec();
-
-  void setOccasionalDelaySec(Long occasionalDelaySec);
-
-  @Description("Probability that an event will be delayed by delayS.")
-  @Nullable
-  Double getProbDelayedEvent();
-
-  void setProbDelayedEvent(Double probDelayedEvent);
-
-  @Description("Maximum size of each log file (in events). For Query10 only.")
-  @Nullable
-  Integer getMaxLogEvents();
-
-  void setMaxLogEvents(Integer maxLogEvents);
-
-  @Description("How to derive names of resources.")
-  @Default.Enum("QUERY_AND_SALT")
-  NexmarkUtils.ResourceNameMode getResourceNameMode();
-
-  void setResourceNameMode(NexmarkUtils.ResourceNameMode mode);
-
-  @Description("If true, manage the creation and cleanup of topics, subscriptions and gcs files.")
-  @Default.Boolean(true)
-  boolean getManageResources();
-
-  void setManageResources(boolean manageResources);
-
-  @Description("If true, use pub/sub publish time instead of event time.")
-  @Nullable
-  Boolean getUsePubsubPublishTime();
-
-  void setUsePubsubPublishTime(Boolean usePubsubPublishTime);
-
-  @Description("Number of events in out-of-order groups. 1 implies no out-of-order events. "
-               + "1000 implies every 1000 events per generator are emitted in pseudo-random order.")
-  @Nullable
-  Long getOutOfOrderGroupSize();
-
-  void setOutOfOrderGroupSize(Long outOfOrderGroupSize);
-
-  @Description("If false, do not add the Monitor and Snoop transforms.")
-  @Nullable
-  Boolean getDebug();
-
-  void setDebug(Boolean value);
-
-  @Description("If set, cancel running pipelines after this long")
-  @Nullable
-  Long getRunningTimeMinutes();
-
-  void setRunningTimeMinutes(Long value);
-
-  @Description("If set and --monitorJobs is true, check that the system watermark is never more "
-               + "than this far behind real time")
-  @Nullable
-  Long getMaxSystemLagSeconds();
-
-  void setMaxSystemLagSeconds(Long value);
-
-  @Description("If set and --monitorJobs is true, check that the data watermark is never more "
-               + "than this far behind real time")
-  @Nullable
-  Long getMaxDataLagSeconds();
-
-  void setMaxDataLagSeconds(Long value);
-
-  @Description("Only start validating watermarks after this many seconds")
-  @Nullable
-  Long getWatermarkValidationDelaySeconds();
-
-  void setWatermarkValidationDelaySeconds(Long value);
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/a7f9f7d0/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/Person.java
----------------------------------------------------------------------
diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/Person.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/Person.java
deleted file mode 100644
index 251a6ee..0000000
--- a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/Person.java
+++ /dev/null
@@ -1,165 +0,0 @@
-/*
- * 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.beam.integration.nexmark;
-
-import com.fasterxml.jackson.annotation.JsonProperty;
-import com.fasterxml.jackson.core.JsonProcessingException;
-
-import java.io.IOException;
-import java.io.InputStream;
-import java.io.OutputStream;
-import java.io.Serializable;
-
-import org.apache.beam.sdk.coders.AtomicCoder;
-import org.apache.beam.sdk.coders.Coder;
-import org.apache.beam.sdk.coders.CoderException;
-import org.apache.beam.sdk.coders.StringUtf8Coder;
-import org.apache.beam.sdk.coders.VarLongCoder;
-
-/**
- * A person either creating an auction or making a bid.
- */
-public class Person implements KnownSize, Serializable {
-  private static final Coder<Long> LONG_CODER = VarLongCoder.of();
-  private static final Coder<String> STRING_CODER = StringUtf8Coder.of();
-  public static final Coder<Person> CODER = new AtomicCoder<Person>() {
-    @Override
-    public void encode(Person value, OutputStream outStream,
-        Coder.Context context)
-        throws CoderException, IOException {
-      LONG_CODER.encode(value.id, outStream, Context.NESTED);
-      STRING_CODER.encode(value.name, outStream, Context.NESTED);
-      STRING_CODER.encode(value.emailAddress, outStream, Context.NESTED);
-      STRING_CODER.encode(value.creditCard, outStream, Context.NESTED);
-      STRING_CODER.encode(value.city, outStream, Context.NESTED);
-      STRING_CODER.encode(value.state, outStream, Context.NESTED);
-      LONG_CODER.encode(value.dateTime, outStream, Context.NESTED);
-      STRING_CODER.encode(value.extra, outStream, Context.NESTED);
-    }
-
-    @Override
-    public Person decode(
-        InputStream inStream, Coder.Context context)
-        throws CoderException, IOException {
-      long id = LONG_CODER.decode(inStream, Context.NESTED);
-      String name = STRING_CODER.decode(inStream, Context.NESTED);
-      String emailAddress = STRING_CODER.decode(inStream, Context.NESTED);
-      String creditCard = STRING_CODER.decode(inStream, Context.NESTED);
-      String city = STRING_CODER.decode(inStream, Context.NESTED);
-      String state = STRING_CODER.decode(inStream, Context.NESTED);
-      long dateTime = LONG_CODER.decode(inStream, Context.NESTED);
-      String extra = STRING_CODER.decode(inStream, Context.NESTED);
-      return new Person(id, name, emailAddress, creditCard, city, state, dateTime, extra);
-    }
-  };
-
-  /** Id of person. */
-  @JsonProperty
-  public final long id; // primary key
-
-  /** Extra person properties. */
-  @JsonProperty
-  public final String name;
-
-  @JsonProperty
-  public final String emailAddress;
-
-  @JsonProperty
-  public final String creditCard;
-
-  @JsonProperty
-  public final String city;
-
-  @JsonProperty
-  public final String state;
-
-  @JsonProperty
-  public final long dateTime;
-
-  /** Additional arbitrary payload for performance testing. */
-  @JsonProperty
-  public final String extra;
-
-  // For Avro only.
-  @SuppressWarnings("unused")
-  private Person() {
-    id = 0;
-    name = null;
-    emailAddress = null;
-    creditCard = null;
-    city = null;
-    state = null;
-    dateTime = 0;
-    extra = null;
-  }
-
-  public Person(long id, String name, String emailAddress, String creditCard, String city,
-      String state, long dateTime, String extra) {
-    this.id = id;
-    this.name = name;
-    this.emailAddress = emailAddress;
-    this.creditCard = creditCard;
-    this.city = city;
-    this.state = state;
-    this.dateTime = dateTime;
-    this.extra = extra;
-  }
-
-  /**
-   * Return a copy of person which capture the given annotation.
-   * (Used for debugging).
-   */
-  public Person withAnnotation(String annotation) {
-    return new Person(id, name, emailAddress, creditCard, city, state, dateTime,
-        annotation + ": " + extra);
-  }
-
-  /**
-   * Does person have {@code annotation}? (Used for debugging.)
-   */
-  public boolean hasAnnotation(String annotation) {
-    return extra.startsWith(annotation + ": ");
-  }
-
-  /**
-   * Remove {@code annotation} from person. (Used for debugging.)
-   */
-  public Person withoutAnnotation(String annotation) {
-    if (hasAnnotation(annotation)) {
-      return new Person(id, name, emailAddress, creditCard, city, state, dateTime,
-          extra.substring(annotation.length() + 2));
-    } else {
-      return this;
-    }
-  }
-
-  @Override
-  public long sizeInBytes() {
-    return 8 + name.length() + 1 + emailAddress.length() + 1 + creditCard.length() + 1
-        + city.length() + 1 + state.length() + 8 + 1 + extra.length() + 1;
-  }
-
-  @Override
-  public String toString() {
-    try {
-      return NexmarkUtils.MAPPER.writeValueAsString(this);
-    } catch (JsonProcessingException e) {
-      throw new RuntimeException(e);
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/a7f9f7d0/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/PubsubHelper.java
----------------------------------------------------------------------
diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/PubsubHelper.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/PubsubHelper.java
deleted file mode 100644
index a79a25b..0000000
--- a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/PubsubHelper.java
+++ /dev/null
@@ -1,216 +0,0 @@
-/*
- * 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.beam.integration.nexmark;
-
-import java.io.IOException;
-import java.util.ArrayList;
-import java.util.Collection;
-import java.util.List;
-
-import org.apache.beam.sdk.options.PubsubOptions;
-import org.apache.beam.sdk.util.PubsubClient;
-import org.apache.beam.sdk.util.PubsubJsonClient;
-
-/**
- * Helper for working with pubsub.
- */
-public class PubsubHelper implements AutoCloseable {
-  /**
-   * Underlying pub/sub client.
-   */
-  private final PubsubClient pubsubClient;
-
-  /**
-   * Project id.
-   */
-  private final String projectId;
-
-  /**
-   * Topics we should delete on close.
-   */
-  private final List<PubsubClient.TopicPath> createdTopics;
-
-  /**
-   * Subscriptions we should delete on close.
-   */
-  private final List<PubsubClient.SubscriptionPath> createdSubscriptions;
-
-  private PubsubHelper(PubsubClient pubsubClient, String projectId) {
-    this.pubsubClient = pubsubClient;
-    this.projectId = projectId;
-    createdTopics = new ArrayList<>();
-    createdSubscriptions = new ArrayList<>();
-  }
-
-  /**
-   * Create a helper.
-   */
-  public static PubsubHelper create(PubsubOptions options) {
-    try {
-      return new PubsubHelper(
-          PubsubJsonClient.FACTORY.newClient(null, null, options),
-          options.getProject());
-    } catch (IOException e) {
-      throw new RuntimeException("Unable to create Pubsub client: ", e);
-    }
-  }
-
-  /**
-   * Create a topic from short name. Delete it if it already exists. Ensure the topic will be
-   * deleted on cleanup. Return full topic name.
-   */
-  public PubsubClient.TopicPath createTopic(String shortTopic) {
-    PubsubClient.TopicPath topic = PubsubClient.topicPathFromName(projectId, shortTopic);
-    try {
-      if (topicExists(shortTopic)) {
-        NexmarkUtils.console("attempting to cleanup topic %s", topic);
-        pubsubClient.deleteTopic(topic);
-      }
-      NexmarkUtils.console("create topic %s", topic);
-      pubsubClient.createTopic(topic);
-      createdTopics.add(topic);
-      return topic;
-    } catch (IOException e) {
-      throw new RuntimeException("Unable to create Pubsub topic " + topic + ": ", e);
-    }
-  }
-
-  /**
-   * Create a topic from short name if it does not already exist. The topic will not be
-   * deleted on cleanup. Return full topic name.
-   */
-  public PubsubClient.TopicPath createOrReuseTopic(String shortTopic) {
-    PubsubClient.TopicPath topic = PubsubClient.topicPathFromName(projectId, shortTopic);
-    try {
-      if (topicExists(shortTopic)) {
-        NexmarkUtils.console("topic %s already exists", topic);
-        return topic;
-      }
-      NexmarkUtils.console("create topic %s", topic);
-      pubsubClient.createTopic(topic);
-      return topic;
-    } catch (IOException e) {
-      throw new RuntimeException("Unable to create or reuse Pubsub topic " + topic + ": ", e);
-    }
-  }
-
-  /**
-   * Check a topic corresponding to short name exists, and throw exception if not. The
-   * topic will not be deleted on cleanup. Return full topic name.
-   */
-  public PubsubClient.TopicPath reuseTopic(String shortTopic) {
-    PubsubClient.TopicPath topic = PubsubClient.topicPathFromName(projectId, shortTopic);
-    if (topicExists(shortTopic)) {
-      NexmarkUtils.console("reusing existing topic %s", topic);
-      return topic;
-    }
-    throw new RuntimeException("topic '" + topic + "' does not already exist");
-  }
-
-  /**
-   * Does topic corresponding to short name exist?
-   */
-  public boolean topicExists(String shortTopic) {
-    PubsubClient.ProjectPath project = PubsubClient.projectPathFromId(projectId);
-    PubsubClient.TopicPath topic = PubsubClient.topicPathFromName(projectId, shortTopic);
-    try {
-      Collection<PubsubClient.TopicPath> existingTopics = pubsubClient.listTopics(project);
-      return existingTopics.contains(topic);
-    } catch (IOException e) {
-      throw new RuntimeException("Unable to check Pubsub topic " + topic + ": ", e);
-    }
-  }
-
-  /**
-   * Create subscription from short name. Delete subscription if it already exists. Ensure the
-   * subscription will be deleted on cleanup. Return full subscription name.
-   */
-  public PubsubClient.SubscriptionPath createSubscription(
-      String shortTopic, String shortSubscription) {
-    PubsubClient.TopicPath topic = PubsubClient.topicPathFromName(projectId, shortTopic);
-    PubsubClient.SubscriptionPath subscription =
-        PubsubClient.subscriptionPathFromName(projectId, shortSubscription);
-    try {
-      if (subscriptionExists(shortTopic, shortSubscription)) {
-        NexmarkUtils.console("attempting to cleanup subscription %s", subscription);
-        pubsubClient.deleteSubscription(subscription);
-      }
-      NexmarkUtils.console("create subscription %s", subscription);
-      pubsubClient.createSubscription(topic, subscription, 60);
-      createdSubscriptions.add(subscription);
-    } catch (IOException e) {
-      throw new RuntimeException("Unable to create Pubsub subscription " + subscription + ": ", e);
-    }
-    return subscription;
-  }
-
-  /**
-   * Check a subscription corresponding to short name exists, and throw exception if not. The
-   * subscription will not be deleted on cleanup. Return full topic name.
-   */
-  public PubsubClient.SubscriptionPath reuseSubscription(
-      String shortTopic, String shortSubscription) {
-    PubsubClient.SubscriptionPath subscription =
-        PubsubClient.subscriptionPathFromName(projectId, shortSubscription);
-    if (subscriptionExists(shortTopic, shortSubscription)) {
-      NexmarkUtils.console("reusing existing subscription %s", subscription);
-      return subscription;
-    }
-    throw new RuntimeException("subscription'" + subscription + "' does not already exist");
-  }
-
-  /**
-   * Does subscription corresponding to short name exist?
-   */
-  public boolean subscriptionExists(String shortTopic, String shortSubscription) {
-    PubsubClient.ProjectPath project = PubsubClient.projectPathFromId(projectId);
-    PubsubClient.TopicPath topic = PubsubClient.topicPathFromName(projectId, shortTopic);
-    PubsubClient.SubscriptionPath subscription =
-        PubsubClient.subscriptionPathFromName(projectId, shortSubscription);
-    try {
-      Collection<PubsubClient.SubscriptionPath> existingSubscriptions =
-          pubsubClient.listSubscriptions(project, topic);
-      return existingSubscriptions.contains(subscription);
-    } catch (IOException e) {
-      throw new RuntimeException("Unable to check Pubsub subscription" + subscription + ": ", e);
-    }
-  }
-
-  /**
-   * Delete all the subscriptions and topics we created.
-   */
-  @Override
-  public void close() {
-    for (PubsubClient.SubscriptionPath subscription : createdSubscriptions) {
-      try {
-        NexmarkUtils.console("delete subscription %s", subscription);
-        pubsubClient.deleteSubscription(subscription);
-      } catch (IOException ex) {
-        NexmarkUtils.console("could not delete subscription %s", subscription);
-      }
-    }
-    for (PubsubClient.TopicPath topic : createdTopics) {
-      try {
-        NexmarkUtils.console("delete topic %s", topic);
-        pubsubClient.deleteTopic(topic);
-      } catch (IOException ex) {
-        NexmarkUtils.console("could not delete topic %s", topic);
-      }
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/a7f9f7d0/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/Query0.java
----------------------------------------------------------------------
diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/Query0.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/Query0.java
deleted file mode 100644
index e88fce0..0000000
--- a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/Query0.java
+++ /dev/null
@@ -1,67 +0,0 @@
-/*
- * 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.beam.integration.nexmark;
-
-import java.io.ByteArrayInputStream;
-import java.io.ByteArrayOutputStream;
-import java.io.IOException;
-
-import org.apache.beam.sdk.coders.Coder;
-import org.apache.beam.sdk.coders.CoderException;
-import org.apache.beam.sdk.transforms.Aggregator;
-import org.apache.beam.sdk.transforms.DoFn;
-import org.apache.beam.sdk.transforms.ParDo;
-import org.apache.beam.sdk.transforms.Sum;
-import org.apache.beam.sdk.values.PCollection;
-
-/**
- * Query 0: Pass events through unchanged. However, force them to do a round trip through
- * serialization so that we measure the impact of the choice of coders.
- */
-public class Query0 extends NexmarkQuery {
-  public Query0(NexmarkConfiguration configuration) {
-    super(configuration, "Query0");
-  }
-
-  private PCollection<Event> applyTyped(PCollection<Event> events) {
-    final Coder<Event> coder = events.getCoder();
-    return events
-        // Force round trip through coder.
-        .apply(name + ".Serialize",
-            ParDo.of(new DoFn<Event, Event>() {
-                  private final Aggregator<Long, Long> bytes =
-                      createAggregator("bytes", Sum.ofLongs());
-
-                  @ProcessElement
-                  public void processElement(ProcessContext c) throws CoderException, IOException {
-                    ByteArrayOutputStream outStream = new ByteArrayOutputStream();
-                    coder.encode(c.element(), outStream, Coder.Context.OUTER);
-                    byte[] byteArray = outStream.toByteArray();
-                    bytes.addValue((long) byteArray.length);
-                    ByteArrayInputStream inStream = new ByteArrayInputStream(byteArray);
-                    Event event = coder.decode(inStream, Coder.Context.OUTER);
-                    c.output(event);
-                  }
-                }));
-  }
-
-  @Override
-  protected PCollection<KnownSize> applyPrim(PCollection<Event> events) {
-    return NexmarkUtils.castToKnownSize(name, applyTyped(events));
-  }
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/a7f9f7d0/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/Query0Model.java
----------------------------------------------------------------------
diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/Query0Model.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/Query0Model.java
deleted file mode 100644
index 37e3f93..0000000
--- a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/Query0Model.java
+++ /dev/null
@@ -1,62 +0,0 @@
-/*
- * 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.beam.integration.nexmark;
-
-import java.util.Collection;
-import java.util.Iterator;
-
-import org.apache.beam.sdk.values.TimestampedValue;
-
-/**
- * A direct implementation of {@link Query0}.
- */
-public class Query0Model extends NexmarkQueryModel {
-  /**
-   * Simulator for query 0.
-   */
-  private class Simulator extends AbstractSimulator<Event, Event> {
-    public Simulator(NexmarkConfiguration configuration) {
-      super(NexmarkUtils.standardEventIterator(configuration));
-    }
-
-    @Override
-    protected void run() {
-      TimestampedValue<Event> timestampedEvent = nextInput();
-      if (timestampedEvent == null) {
-        allDone();
-        return;
-      }
-      addResult(timestampedEvent);
-      //TODO test fails because offset of some hundreds of ms beween expect and actual
-    }
-  }
-
-  public Query0Model(NexmarkConfiguration configuration) {
-    super(configuration);
-  }
-
-  @Override
-  protected AbstractSimulator<?, ?> simulator() {
-    return new Simulator(configuration);
-  }
-
-  @Override
-  protected <T> Collection<String> toCollection(Iterator<TimestampedValue<T>> itr) {
-    return toValueTimestampOrder(itr);
-  }
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/a7f9f7d0/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/Query1.java
----------------------------------------------------------------------
diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/Query1.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/Query1.java
deleted file mode 100644
index a1ecdeb..0000000
--- a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/Query1.java
+++ /dev/null
@@ -1,62 +0,0 @@
-/*
- * 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.beam.integration.nexmark;
-
-import org.apache.beam.sdk.transforms.DoFn;
-import org.apache.beam.sdk.transforms.ParDo;
-import org.apache.beam.sdk.values.PCollection;
-
-/**
- * Query 1, 'Currency Conversion'. Convert each bid value from dollars to euros.
- * In CQL syntax:
- *
- * <pre>
- * SELECT Istream(auction, DOLTOEUR(price), bidder, datetime)
- * FROM bid [ROWS UNBOUNDED];
- * </pre>
- *
- * <p>To make things more interesting, allow the 'currency conversion' to be arbitrarily
- * slowed down.
- */
-class Query1 extends NexmarkQuery {
-  public Query1(NexmarkConfiguration configuration) {
-    super(configuration, "Query1");
-  }
-
-  private PCollection<Bid> applyTyped(PCollection<Event> events) {
-    return events
-        // Only want the bid events.
-        .apply(JUST_BIDS)
-
-        // Map the conversion function over all bids.
-        .apply(name + ".ToEuros",
-            ParDo.of(new DoFn<Bid, Bid>() {
-                  @ProcessElement
-                  public void processElement(ProcessContext c) {
-                    Bid bid = c.element();
-                    c.output(new Bid(
-                        bid.auction, bid.bidder, (bid.price * 89) / 100, bid.dateTime, bid.extra));
-                  }
-                }));
-  }
-
-  @Override
-  protected PCollection<KnownSize> applyPrim(PCollection<Event> events) {
-    return NexmarkUtils.castToKnownSize(name, applyTyped(events));
-  }
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/a7f9f7d0/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/Query10.java
----------------------------------------------------------------------
diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/Query10.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/Query10.java
deleted file mode 100644
index 7bdcb36..0000000
--- a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/Query10.java
+++ /dev/null
@@ -1,380 +0,0 @@
-/*
- * 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.beam.integration.nexmark;
-
-import static com.google.common.base.Preconditions.checkState;
-
-import com.google.cloud.hadoop.gcsio.GoogleCloudStorageWriteChannel;
-
-import java.io.IOException;
-import java.io.OutputStream;
-import java.io.Serializable;
-import java.nio.channels.Channels;
-import java.nio.channels.WritableByteChannel;
-import java.util.concurrent.ThreadLocalRandom;
-import javax.annotation.Nullable;
-
-import org.apache.beam.sdk.coders.Coder;
-import org.apache.beam.sdk.options.GcsOptions;
-import org.apache.beam.sdk.transforms.Aggregator;
-import org.apache.beam.sdk.transforms.DoFn;
-import org.apache.beam.sdk.transforms.GroupByKey;
-import org.apache.beam.sdk.transforms.ParDo;
-import org.apache.beam.sdk.transforms.Sum;
-import org.apache.beam.sdk.transforms.windowing.AfterEach;
-import org.apache.beam.sdk.transforms.windowing.AfterFirst;
-import org.apache.beam.sdk.transforms.windowing.AfterPane;
-import org.apache.beam.sdk.transforms.windowing.AfterProcessingTime;
-import org.apache.beam.sdk.transforms.windowing.AfterWatermark;
-import org.apache.beam.sdk.transforms.windowing.BoundedWindow;
-import org.apache.beam.sdk.transforms.windowing.FixedWindows;
-import org.apache.beam.sdk.transforms.windowing.PaneInfo;
-import org.apache.beam.sdk.transforms.windowing.PaneInfo.Timing;
-import org.apache.beam.sdk.transforms.windowing.Repeatedly;
-import org.apache.beam.sdk.transforms.windowing.Window;
-import org.apache.beam.sdk.util.GcsIOChannelFactory;
-import org.apache.beam.sdk.values.KV;
-import org.apache.beam.sdk.values.PCollection;
-import org.joda.time.Duration;
-import org.joda.time.Instant;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-
-/**
- * Query "10", 'Log to sharded files' (Not in original suite.)
- *
- * <p>Every windowSizeSec, save all events from the last period into 2*maxWorkers log files.
- */
-class Query10 extends NexmarkQuery {
-  private static final Logger LOG = LoggerFactory.getLogger(Query10.class);
-  private static final int CHANNEL_BUFFER = 8 << 20; // 8MB
-  private static final int NUM_SHARDS_PER_WORKER = 5;
-  private static final Duration LATE_BATCHING_PERIOD = Duration.standardSeconds(10);
-
-  /**
-   * Capture everything we need to know about the records in a single output file.
-   */
-  private static class OutputFile implements Serializable {
-    /** Maximum possible timestamp of records in file. */
-    private final Instant maxTimestamp;
-    /** Shard within window. */
-    private final String shard;
-    /** Index of file in all files in shard. */
-    private final long index;
-    /** Timing of records in this file. */
-    private final PaneInfo.Timing timing;
-    /** Path to file containing records, or {@literal null} if no output required. */
-    @Nullable
-    private final String filename;
-
-    public OutputFile(
-        Instant maxTimestamp,
-        String shard,
-        long index,
-        PaneInfo.Timing timing,
-        @Nullable String filename) {
-      this.maxTimestamp = maxTimestamp;
-      this.shard = shard;
-      this.index = index;
-      this.timing = timing;
-      this.filename = filename;
-    }
-
-    @Override
-    public String toString() {
-      return String.format("%s %s %d %s %s\n", maxTimestamp, shard, index, timing, filename);
-    }
-  }
-
-  /**
-   * GCS uri prefix for all log and 'finished' files. If null they won't be written.
-   */
-  @Nullable
-  private String outputPath;
-
-  /**
-   * Maximum number of workers, used to determine log sharding factor.
-   */
-  private int maxNumWorkers;
-
-  public Query10(NexmarkConfiguration configuration) {
-    super(configuration, "Query10");
-  }
-
-  public void setOutputPath(@Nullable String outputPath) {
-    this.outputPath = outputPath;
-  }
-
-  public void setMaxNumWorkers(int maxNumWorkers) {
-    this.maxNumWorkers = maxNumWorkers;
-  }
-
-  /**
-   * Return channel for writing bytes to GCS.
-   *
-   * @throws IOException
-   */
-  private WritableByteChannel openWritableGcsFile(GcsOptions options, String filename)
-      throws IOException {
-    WritableByteChannel channel =
-            GcsIOChannelFactory.fromOptions(options).create(filename, "text/plain");
-    checkState(channel instanceof GoogleCloudStorageWriteChannel);
-    ((GoogleCloudStorageWriteChannel) channel).setUploadBufferSize(CHANNEL_BUFFER);
-    return channel;
-  }
-
-  /** Return a short string to describe {@code timing}. */
-  private String timingToString(PaneInfo.Timing timing) {
-    switch (timing) {
-      case EARLY:
-        return "E";
-      case ON_TIME:
-        return "O";
-      case LATE:
-        return "L";
-    }
-    throw new RuntimeException(); // cases are exhaustive
-  }
-
-  /** Construct an {@link OutputFile} for {@code pane} in {@code window} for {@code shard}. */
-  private OutputFile outputFileFor(BoundedWindow window, String shard, PaneInfo pane) {
-    @Nullable String filename =
-        outputPath == null
-        ? null
-        : String.format("%s/LOG-%s-%s-%03d-%s-%x",
-            outputPath, window.maxTimestamp(), shard, pane.getIndex(),
-            timingToString(pane.getTiming()),
-            ThreadLocalRandom.current().nextLong());
-    return new OutputFile(window.maxTimestamp(), shard, pane.getIndex(),
-        pane.getTiming(), filename);
-  }
-
-  /**
-   * Return path to which we should write the index for {@code window}, or {@literal null}
-   * if no output required.
-   */
-  @Nullable
-  private String indexPathFor(BoundedWindow window) {
-    if (outputPath == null) {
-      return null;
-    }
-    return String.format("%s/INDEX-%s", outputPath, window.maxTimestamp());
-  }
-
-  private PCollection<Done> applyTyped(PCollection<Event> events) {
-    final int numLogShards = maxNumWorkers * NUM_SHARDS_PER_WORKER;
-
-    return events.apply(name + ".ShardEvents",
-            ParDo.of(new DoFn<Event, KV<String, Event>>() {
-                      final Aggregator<Long, Long> lateCounter =
-                          createAggregator("actuallyLateEvent", Sum.ofLongs());
-                      final Aggregator<Long, Long> onTimeCounter =
-                          createAggregator("actuallyOnTimeEvent", Sum.ofLongs());
-
-                      @ProcessElement
-                      public void processElement(ProcessContext c) {
-                        if (c.element().hasAnnotation("LATE")) {
-                          lateCounter.addValue(1L);
-                          LOG.error("Observed late: %s", c.element());
-                        } else {
-                          onTimeCounter.addValue(1L);
-                        }
-                        int shardNum = (int) Math.abs((long) c.element().hashCode() % numLogShards);
-                        String shard = String.format("shard-%05d-of-%05d", shardNum, numLogShards);
-                        c.output(KV.of(shard, c.element()));
-                      }
-                    }))
-        .apply(name + ".WindowEvents",
-                Window.<KV<String, Event>>into(
-            FixedWindows.of(Duration.standardSeconds(configuration.windowSizeSec)))
-            .triggering(AfterEach.inOrder(
-                Repeatedly
-                    .forever(AfterPane.elementCountAtLeast(configuration.maxLogEvents))
-                    .orFinally(AfterWatermark.pastEndOfWindow()),
-                Repeatedly.forever(
-                    AfterFirst.of(AfterPane.elementCountAtLeast(configuration.maxLogEvents),
-                        AfterProcessingTime.pastFirstElementInPane()
-                                           .plusDelayOf(LATE_BATCHING_PERIOD)))))
-            .discardingFiredPanes()
-            // Use a 1 day allowed lateness so that any forgotten hold will stall the
-            // pipeline for that period and be very noticeable.
-            .withAllowedLateness(Duration.standardDays(1)))
-        .apply(name + ".GroupByKey", GroupByKey.<String, Event>create())
-        .apply(name + ".CheckForLateEvents",
-            ParDo.of(new DoFn<KV<String, Iterable<Event>>,
-                     KV<String, Iterable<Event>>>() {
-                   final Aggregator<Long, Long> earlyCounter =
-                       createAggregator("earlyShard", Sum.ofLongs());
-                   final Aggregator<Long, Long> onTimeCounter =
-                       createAggregator("onTimeShard", Sum.ofLongs());
-                   final Aggregator<Long, Long> lateCounter =
-                       createAggregator("lateShard", Sum.ofLongs());
-                   final Aggregator<Long, Long> unexpectedLatePaneCounter =
-                       createAggregator("ERROR_unexpectedLatePane", Sum.ofLongs());
-                   final Aggregator<Long, Long> unexpectedOnTimeElementCounter =
-                       createAggregator("ERROR_unexpectedOnTimeElement", Sum.ofLongs());
-
-                   @ProcessElement
-                   public void processElement(ProcessContext c, BoundedWindow window) {
-                     int numLate = 0;
-                     int numOnTime = 0;
-                     for (Event event : c.element().getValue()) {
-                       if (event.hasAnnotation("LATE")) {
-                         numLate++;
-                       } else {
-                         numOnTime++;
-                       }
-                     }
-                     String shard = c.element().getKey();
-                     LOG.error(
-                         "%s with timestamp %s has %d actually late and %d on-time "
-                             + "elements in pane %s for window %s",
-                         shard, c.timestamp(), numLate, numOnTime, c.pane(),
-                         window.maxTimestamp());
-                     if (c.pane().getTiming() == PaneInfo.Timing.LATE) {
-                       if (numLate == 0) {
-                         LOG.error(
-                             "ERROR! No late events in late pane for %s", shard);
-                         unexpectedLatePaneCounter.addValue(1L);
-                       }
-                       if (numOnTime > 0) {
-                         LOG.error(
-                             "ERROR! Have %d on-time events in late pane for %s",
-                             numOnTime, shard);
-                         unexpectedOnTimeElementCounter.addValue(1L);
-                       }
-                       lateCounter.addValue(1L);
-                     } else if (c.pane().getTiming() == PaneInfo.Timing.EARLY) {
-                       if (numOnTime + numLate < configuration.maxLogEvents) {
-                         LOG.error(
-                             "ERROR! Only have %d events in early pane for %s",
-                             numOnTime + numLate, shard);
-                       }
-                       earlyCounter.addValue(1L);
-                     } else {
-                       onTimeCounter.addValue(1L);
-                     }
-                     c.output(c.element());
-                   }
-                 }))
-        .apply(name + ".UploadEvents",
-            ParDo.of(new DoFn<KV<String, Iterable<Event>>,
-                     KV<Void, OutputFile>>() {
-                   final Aggregator<Long, Long> savedFileCounter =
-                       createAggregator("savedFile", Sum.ofLongs());
-                   final Aggregator<Long, Long> writtenRecordsCounter =
-                       createAggregator("writtenRecords", Sum.ofLongs());
-
-                   @ProcessElement
-                   public void processElement(ProcessContext c, BoundedWindow window)
-                           throws IOException {
-                     String shard = c.element().getKey();
-                     GcsOptions options = c.getPipelineOptions().as(GcsOptions.class);
-                     OutputFile outputFile = outputFileFor(window, shard, c.pane());
-                     LOG.error(
-                         "Writing %s with record timestamp %s, window timestamp %s, pane %s",
-                         shard, c.timestamp(), window.maxTimestamp(), c.pane());
-                     if (outputFile.filename != null) {
-                       LOG.error("Beginning write to '%s'", outputFile.filename);
-                       int n = 0;
-                       try (OutputStream output =
-                                Channels.newOutputStream(openWritableGcsFile(options, outputFile
-                                    .filename))) {
-                         for (Event event : c.element().getValue()) {
-                           Event.CODER.encode(event, output, Coder.Context.OUTER);
-                           writtenRecordsCounter.addValue(1L);
-                           if (++n % 10000 == 0) {
-                             LOG.error("So far written %d records to '%s'", n,
-                                 outputFile.filename);
-                           }
-                         }
-                       }
-                       LOG.error("Written all %d records to '%s'", n, outputFile.filename);
-                     }
-                     savedFileCounter.addValue(1L);
-                     c.output(KV.<Void, OutputFile>of(null, outputFile));
-                   }
-                 }))
-        // Clear fancy triggering from above.
-        .apply(name + ".WindowLogFiles", Window.<KV<Void, OutputFile>>into(
-            FixedWindows.of(Duration.standardSeconds(configuration.windowSizeSec)))
-            .triggering(AfterWatermark.pastEndOfWindow())
-            // We expect no late data here, but we'll assume the worst so we can detect any.
-            .withAllowedLateness(Duration.standardDays(1))
-            .discardingFiredPanes())
-      // TODO etienne: unnecessary groupByKey? because aggregators are shared in parallel
-      // and Pardo is also in parallel, why group all elements in memory of the same executor?
-      .apply(name + ".GroupByKey2", GroupByKey.<Void, OutputFile>create())
-        .apply(name + ".Index",
-            ParDo.of(new DoFn<KV<Void, Iterable<OutputFile>>, Done>() {
-                   final Aggregator<Long, Long> unexpectedLateCounter =
-                       createAggregator("ERROR_unexpectedLate", Sum.ofLongs());
-                   final Aggregator<Long, Long> unexpectedEarlyCounter =
-                       createAggregator("ERROR_unexpectedEarly", Sum.ofLongs());
-                   final Aggregator<Long, Long> unexpectedIndexCounter =
-                       createAggregator("ERROR_unexpectedIndex", Sum.ofLongs());
-                   final Aggregator<Long, Long> finalizedCounter =
-                       createAggregator("indexed", Sum.ofLongs());
-
-                   @ProcessElement
-                   public void processElement(ProcessContext c, BoundedWindow window)
-                           throws IOException {
-                     if (c.pane().getTiming() == Timing.LATE) {
-                       unexpectedLateCounter.addValue(1L);
-                       LOG.error("ERROR! Unexpected LATE pane: %s", c.pane());
-                     } else if (c.pane().getTiming() == Timing.EARLY) {
-                       unexpectedEarlyCounter.addValue(1L);
-                       LOG.error("ERROR! Unexpected EARLY pane: %s", c.pane());
-                     } else if (c.pane().getTiming() == Timing.ON_TIME
-                         && c.pane().getIndex() != 0) {
-                       unexpectedIndexCounter.addValue(1L);
-                       LOG.error("ERROR! Unexpected ON_TIME pane index: %s", c.pane());
-                     } else {
-                       GcsOptions options = c.getPipelineOptions().as(GcsOptions.class);
-                       LOG.error(
-                           "Index with record timestamp %s, window timestamp %s, pane %s",
-                           c.timestamp(), window.maxTimestamp(), c.pane());
-
-                       @Nullable String filename = indexPathFor(window);
-                       if (filename != null) {
-                         LOG.error("Beginning write to '%s'", filename);
-                         int n = 0;
-                         try (OutputStream output =
-                                  Channels.newOutputStream(
-                                      openWritableGcsFile(options, filename))) {
-                           for (OutputFile outputFile : c.element().getValue()) {
-                             output.write(outputFile.toString().getBytes());
-                             n++;
-                           }
-                         }
-                         LOG.error("Written all %d lines to '%s'", n, filename);
-                       }
-                       c.output(
-                           new Done("written for timestamp " + window.maxTimestamp()));
-                       finalizedCounter.addValue(1L);
-                     }
-                   }
-                 }));
-  }
-
-  @Override
-  protected PCollection<KnownSize> applyPrim(PCollection<Event> events) {
-    return NexmarkUtils.castToKnownSize(name, applyTyped(events));
-  }
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/a7f9f7d0/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/Query11.java
----------------------------------------------------------------------
diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/Query11.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/Query11.java
deleted file mode 100644
index d610b7c..0000000
--- a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/Query11.java
+++ /dev/null
@@ -1,73 +0,0 @@
-/*
- * 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.beam.integration.nexmark;
-
-import org.apache.beam.sdk.transforms.Count;
-import org.apache.beam.sdk.transforms.DoFn;
-import org.apache.beam.sdk.transforms.ParDo;
-import org.apache.beam.sdk.transforms.windowing.AfterPane;
-import org.apache.beam.sdk.transforms.windowing.Repeatedly;
-import org.apache.beam.sdk.transforms.windowing.Sessions;
-import org.apache.beam.sdk.transforms.windowing.Window;
-import org.apache.beam.sdk.values.KV;
-import org.apache.beam.sdk.values.PCollection;
-import org.joda.time.Duration;
-
-/**
- * Query "11", 'User sessions' (Not in original suite.)
- *
- * <p>Group bids by the same user into sessions with {@code windowSizeSec} max gap.
- * However limit the session to at most {@code maxLogEvents}. Emit the number of
- * bids per session.
- */
-class Query11 extends NexmarkQuery {
-  public Query11(NexmarkConfiguration configuration) {
-    super(configuration, "Query11");
-  }
-
-  private PCollection<BidsPerSession> applyTyped(PCollection<Event> events) {
-    return events.apply(JUST_BIDS)
-        .apply(name + ".Rekey",
-          // TODO etienne: why not avoid this ParDo and do a Cont.perElement?
-          ParDo.of(new DoFn<Bid, KV<Long, Void>>() {
-                  @ProcessElement
-                  public void processElement(ProcessContext c) {
-                    Bid bid = c.element();
-                    c.output(KV.of(bid.bidder, (Void) null));
-                  }
-                }))
-        .apply(Window.<KV<Long, Void>>into(
-            Sessions.withGapDuration(Duration.standardSeconds(configuration.windowSizeSec)))
-        .triggering(Repeatedly.forever(AfterPane.elementCountAtLeast(configuration.maxLogEvents)))
-        .discardingFiredPanes()
-        .withAllowedLateness(Duration.standardSeconds(configuration.occasionalDelaySec / 2)))
-        .apply(Count.<Long, Void>perKey())
-        .apply(name + ".ToResult",
-            ParDo.of(new DoFn<KV<Long, Long>, BidsPerSession>() {
-                  @ProcessElement
-                  public void processElement(ProcessContext c) {
-                    c.output(new BidsPerSession(c.element().getKey(), c.element().getValue()));
-                  }
-                }));
-  }
-
-  @Override
-  protected PCollection<KnownSize> applyPrim(PCollection<Event> events) {
-    return NexmarkUtils.castToKnownSize(name, applyTyped(events));
-  }
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/a7f9f7d0/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/Query12.java
----------------------------------------------------------------------
diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/Query12.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/Query12.java
deleted file mode 100644
index 72fbb57..0000000
--- a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/Query12.java
+++ /dev/null
@@ -1,77 +0,0 @@
-/*
- * 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.beam.integration.nexmark;
-
-import org.apache.beam.sdk.transforms.Count;
-import org.apache.beam.sdk.transforms.DoFn;
-import org.apache.beam.sdk.transforms.ParDo;
-import org.apache.beam.sdk.transforms.windowing.AfterProcessingTime;
-import org.apache.beam.sdk.transforms.windowing.GlobalWindows;
-import org.apache.beam.sdk.transforms.windowing.Repeatedly;
-import org.apache.beam.sdk.transforms.windowing.Window;
-import org.apache.beam.sdk.values.KV;
-import org.apache.beam.sdk.values.PCollection;
-import org.joda.time.Duration;
-
-/**
- * Query "12", 'Processing time windows' (Not in original suite.)
- *
- * <p>Group bids by the same user into processing time windows of windowSize. Emit the count
- * of bids per window.
- */
-class Query12 extends NexmarkQuery {
-  public Query12(NexmarkConfiguration configuration) {
-    super(configuration, "Query12");
-  }
-
-  private PCollection<BidsPerSession> applyTyped(PCollection<Event> events) {
-    return events
-        .apply(JUST_BIDS)
-        .apply(name + ".Rekey",
-          // TODO etienne: why not avoid this ParDo and do a Cont.perElement?
-            ParDo.of(new DoFn<Bid, KV<Long, Void>>() {
-                   @ProcessElement
-                   public void processElement(ProcessContext c) {
-                     Bid bid = c.element();
-                     c.output(KV.of(bid.bidder, (Void) null));
-                   }
-                 }))
-        .apply(Window.<KV<Long, Void>>into(new GlobalWindows())
-            .triggering(
-                Repeatedly.forever(
-                    AfterProcessingTime.pastFirstElementInPane()
-                                       .plusDelayOf(
-                                           Duration.standardSeconds(configuration.windowSizeSec))))
-            .discardingFiredPanes()
-            .withAllowedLateness(Duration.ZERO))
-        .apply(Count.<Long, Void>perKey())
-        .apply(name + ".ToResult",
-            ParDo.of(new DoFn<KV<Long, Long>, BidsPerSession>() {
-                   @ProcessElement
-                   public void processElement(ProcessContext c) {
-                     c.output(
-                         new BidsPerSession(c.element().getKey(), c.element().getValue()));
-                   }
-                 }));
-  }
-
-  @Override
-  protected PCollection<KnownSize> applyPrim(PCollection<Event> events) {
-    return NexmarkUtils.castToKnownSize(name, applyTyped(events));
-  }
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/a7f9f7d0/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/Query1Model.java
----------------------------------------------------------------------
diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/Query1Model.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/Query1Model.java
deleted file mode 100644
index 16287e6..0000000
--- a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/Query1Model.java
+++ /dev/null
@@ -1,73 +0,0 @@
-/*
- * 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.beam.integration.nexmark;
-
-import java.io.Serializable;
-import java.util.Collection;
-import java.util.Iterator;
-
-import org.apache.beam.sdk.values.TimestampedValue;
-
-/**
- * A direct implementation of {@link Query1}.
- */
-public class Query1Model extends NexmarkQueryModel implements Serializable {
-  /**
-   * Simulator for query 1.
-   */
-  private class Simulator extends AbstractSimulator<Event, Bid> {
-    public Simulator(NexmarkConfiguration configuration) {
-      super(NexmarkUtils.standardEventIterator(configuration));
-    }
-
-    @Override
-    protected void run() {
-      TimestampedValue<Event> timestampedEvent = nextInput();
-      if (timestampedEvent == null) {
-        allDone();
-        return;
-      }
-      Event event = timestampedEvent.getValue();
-      if (event.bid == null) {
-        // Ignore non-bid events.
-        return;
-      }
-      Bid bid = event.bid;
-      Bid resultBid =
-          new Bid(bid.auction, bid.bidder, bid.price * 89 / 100, bid.dateTime, bid.extra);
-      TimestampedValue<Bid> result =
-          TimestampedValue.of(resultBid, timestampedEvent.getTimestamp());
-      addResult(result);
-      //TODO test fails because offset of some hundreds of ms beween expect and actual
-    }
-  }
-
-  public Query1Model(NexmarkConfiguration configuration) {
-    super(configuration);
-  }
-
-  @Override
-  public AbstractSimulator<?, ?> simulator() {
-    return new Simulator(configuration);
-  }
-
-  @Override
-  protected <T> Collection<String> toCollection(Iterator<TimestampedValue<T>> itr) {
-    return toValueTimestampOrder(itr);
-  }
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/a7f9f7d0/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/Query2.java
----------------------------------------------------------------------
diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/Query2.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/Query2.java
deleted file mode 100644
index 828cdf5..0000000
--- a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/Query2.java
+++ /dev/null
@@ -1,73 +0,0 @@
-/*
- * 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.beam.integration.nexmark;
-
-import org.apache.beam.sdk.transforms.DoFn;
-import org.apache.beam.sdk.transforms.Filter;
-import org.apache.beam.sdk.transforms.ParDo;
-import org.apache.beam.sdk.transforms.SerializableFunction;
-import org.apache.beam.sdk.values.PCollection;
-
-/**
- * Query 2, 'Filtering. Find bids with specific auction ids and show their bid price.
- * In CQL syntax:
- *
- * <pre>
- * SELECT Rstream(auction, price)
- * FROM Bid [NOW]
- * WHERE auction = 1007 OR auction = 1020 OR auction = 2001 OR auction = 2019 OR auction = 2087;
- * </pre>
- *
- * <p>As written that query will only yield a few hundred results over event streams of
- * arbitrary size. To make it more interesting we instead choose bids for every
- * {@code auctionSkip}'th auction.
- */
-class Query2 extends NexmarkQuery {
-  public Query2(NexmarkConfiguration configuration) {
-    super(configuration, "Query2");
-  }
-
-  private PCollection<AuctionPrice> applyTyped(PCollection<Event> events) {
-    return events
-        // Only want the bid events.
-        .apply(JUST_BIDS)
-
-        // Select just the bids for the auctions we care about.
-        .apply(Filter.by(new SerializableFunction<Bid, Boolean>() {
-          @Override
-          public Boolean apply(Bid bid) {
-            return bid.auction % configuration.auctionSkip == 0;
-          }
-        }))
-
-        // Project just auction id and price.
-        .apply(name + ".Project",
-            ParDo.of(new DoFn<Bid, AuctionPrice>() {
-                  @ProcessElement
-                  public void processElement(ProcessContext c) {
-                    Bid bid = c.element();
-                    c.output(new AuctionPrice(bid.auction, bid.price));
-                  }
-                }));
-  }
-
-  @Override
-  protected PCollection<KnownSize> applyPrim(PCollection<Event> events) {
-    return NexmarkUtils.castToKnownSize(name, applyTyped(events));
-  }
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/a7f9f7d0/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/Query2Model.java
----------------------------------------------------------------------
diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/Query2Model.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/Query2Model.java
deleted file mode 100644
index 7769e52..0000000
--- a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/Query2Model.java
+++ /dev/null
@@ -1,75 +0,0 @@
-/*
- * 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.beam.integration.nexmark;
-
-import java.io.Serializable;
-import java.util.Collection;
-import java.util.Iterator;
-
-import org.apache.beam.sdk.values.TimestampedValue;
-
-/**
- * A direct implementation of {@link Query2}.
- */
-public class Query2Model extends NexmarkQueryModel implements Serializable {
-  /**
-   * Simulator for query 2.
-   */
-  private class Simulator extends AbstractSimulator<Event, AuctionPrice> {
-    public Simulator(NexmarkConfiguration configuration) {
-      super(NexmarkUtils.standardEventIterator(configuration));
-    }
-
-    @Override
-    protected void run() {
-      TimestampedValue<Event> timestampedEvent = nextInput();
-      if (timestampedEvent == null) {
-        allDone();
-        return;
-      }
-      Event event = timestampedEvent.getValue();
-      if (event.bid == null) {
-        // Ignore non bid events.
-        return;
-      }
-      Bid bid = event.bid;
-      if (bid.auction % configuration.auctionSkip != 0) {
-        // Ignore bids for auctions we don't care about.
-        return;
-      }
-      AuctionPrice auctionPrice = new AuctionPrice(bid.auction, bid.price);
-      TimestampedValue<AuctionPrice> result =
-          TimestampedValue.of(auctionPrice, timestampedEvent.getTimestamp());
-      addResult(result);
-    }
-  }
-
-  public Query2Model(NexmarkConfiguration configuration) {
-    super(configuration);
-  }
-
-  @Override
-  public AbstractSimulator<?, ?> simulator() {
-    return new Simulator(configuration);
-  }
-
-  @Override
-  protected <T> Collection<String> toCollection(Iterator<TimestampedValue<T>> itr) {
-    return toValueTimestampOrder(itr);
-  }
-}


[11/55] [abbrv] beam git commit: Refactor classes into packages

Posted by ie...@apache.org.
http://git-wip-us.apache.org/repos/asf/beam/blob/a7f9f7d0/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/model/CategoryPrice.java
----------------------------------------------------------------------
diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/model/CategoryPrice.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/model/CategoryPrice.java
new file mode 100644
index 0000000..6512cc1
--- /dev/null
+++ b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/model/CategoryPrice.java
@@ -0,0 +1,100 @@
+/*
+ * 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.beam.integration.nexmark.model;
+
+import com.fasterxml.jackson.annotation.JsonProperty;
+import com.fasterxml.jackson.core.JsonProcessingException;
+
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
+import java.io.Serializable;
+
+import org.apache.beam.integration.nexmark.NexmarkUtils;
+import org.apache.beam.sdk.coders.AtomicCoder;
+import org.apache.beam.sdk.coders.Coder;
+import org.apache.beam.sdk.coders.CoderException;
+import org.apache.beam.sdk.coders.VarIntCoder;
+import org.apache.beam.sdk.coders.VarLongCoder;
+
+/**
+ * Result of Query4.
+ */
+public class CategoryPrice implements KnownSize, Serializable {
+  private static final Coder<Long> LONG_CODER = VarLongCoder.of();
+  private static final Coder<Integer> INT_CODER = VarIntCoder.of();
+
+  public static final Coder<CategoryPrice> CODER = new AtomicCoder<CategoryPrice>() {
+    @Override
+    public void encode(CategoryPrice value, OutputStream outStream,
+        Coder.Context context)
+        throws CoderException, IOException {
+      LONG_CODER.encode(value.category, outStream, Context.NESTED);
+      LONG_CODER.encode(value.price, outStream, Context.NESTED);
+      INT_CODER.encode(value.isLast ? 1 : 0, outStream, Context.NESTED);
+    }
+
+    @Override
+    public CategoryPrice decode(
+        InputStream inStream, Coder.Context context)
+        throws CoderException, IOException {
+      long category = LONG_CODER.decode(inStream, Context.NESTED);
+      long price = LONG_CODER.decode(inStream, Context.NESTED);
+      boolean isLast = INT_CODER.decode(inStream, context) != 0;
+      return new CategoryPrice(category, price, isLast);
+    }
+  };
+
+  @JsonProperty
+  public final long category;
+
+  /** Price in cents. */
+  @JsonProperty
+  public final long price;
+
+  @JsonProperty
+  public final boolean isLast;
+
+  // For Avro only.
+  @SuppressWarnings("unused")
+  private CategoryPrice() {
+    category = 0;
+    price = 0;
+    isLast = false;
+  }
+
+  public CategoryPrice(long category, long price, boolean isLast) {
+    this.category = category;
+    this.price = price;
+    this.isLast = isLast;
+  }
+
+  @Override
+  public long sizeInBytes() {
+    return 8 + 8 + 1;
+  }
+
+  @Override
+  public String toString() {
+    try {
+      return NexmarkUtils.MAPPER.writeValueAsString(this);
+    } catch (JsonProcessingException e) {
+      throw new RuntimeException(e);
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/a7f9f7d0/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/model/Done.java
----------------------------------------------------------------------
diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/model/Done.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/model/Done.java
new file mode 100644
index 0000000..6009463
--- /dev/null
+++ b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/model/Done.java
@@ -0,0 +1,83 @@
+/*
+ * 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.beam.integration.nexmark.model;
+
+import com.fasterxml.jackson.annotation.JsonProperty;
+import com.fasterxml.jackson.core.JsonProcessingException;
+
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
+import java.io.Serializable;
+
+import org.apache.beam.integration.nexmark.NexmarkUtils;
+import org.apache.beam.sdk.coders.AtomicCoder;
+import org.apache.beam.sdk.coders.Coder;
+import org.apache.beam.sdk.coders.CoderException;
+import org.apache.beam.sdk.coders.StringUtf8Coder;
+
+/**
+ * Result of query 10.
+ */
+public class Done implements KnownSize, Serializable {
+  private static final Coder<String> STRING_CODER = StringUtf8Coder.of();
+
+  public static final Coder<Done> CODER = new AtomicCoder<Done>() {
+    @Override
+    public void encode(Done value, OutputStream outStream,
+        Coder.Context context)
+        throws CoderException, IOException {
+      STRING_CODER.encode(value.message, outStream, Context.NESTED);
+    }
+
+    @Override
+    public Done decode(
+        InputStream inStream, Coder.Context context)
+        throws CoderException, IOException {
+      String message = STRING_CODER.decode(inStream, Context.NESTED);
+      return new Done(message);
+    }
+  };
+
+  @JsonProperty
+  public final String message;
+
+  // For Avro only.
+  @SuppressWarnings("unused")
+  public Done() {
+    message = null;
+  }
+
+  public Done(String message) {
+    this.message = message;
+  }
+
+  @Override
+  public long sizeInBytes() {
+    return message.length();
+  }
+
+  @Override
+  public String toString() {
+    try {
+      return NexmarkUtils.MAPPER.writeValueAsString(this);
+    } catch (JsonProcessingException e) {
+      throw new RuntimeException(e);
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/a7f9f7d0/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/model/Event.java
----------------------------------------------------------------------
diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/model/Event.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/model/Event.java
new file mode 100644
index 0000000..8a278bf
--- /dev/null
+++ b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/model/Event.java
@@ -0,0 +1,179 @@
+/*
+ * 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.beam.integration.nexmark.model;
+
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
+import java.io.Serializable;
+import javax.annotation.Nullable;
+
+import org.apache.beam.sdk.coders.AtomicCoder;
+import org.apache.beam.sdk.coders.Coder;
+import org.apache.beam.sdk.coders.CoderException;
+import org.apache.beam.sdk.coders.VarIntCoder;
+
+/**
+ * An event in the auction system, either a (new) {@link Person}, a (new) {@link Auction},
+ * or a {@link Bid}.
+ */
+public class Event implements KnownSize, Serializable {
+  private static final Coder<Integer> INT_CODER = VarIntCoder.of();
+
+  public static final Coder<Event> CODER = new AtomicCoder<Event>() {
+    @Override
+    public void encode(Event value, OutputStream outStream, Coder.Context context)
+        throws CoderException, IOException {
+      if (value.newPerson != null) {
+        INT_CODER.encode(0, outStream, Context.NESTED);
+        Person.CODER.encode(value.newPerson, outStream, Context.NESTED);
+      } else if (value.newAuction != null) {
+        INT_CODER.encode(1, outStream, Context.NESTED);
+        Auction.CODER.encode(value.newAuction, outStream, Context.NESTED);
+      } else if (value.bid != null) {
+        INT_CODER.encode(2, outStream, Context.NESTED);
+        Bid.CODER.encode(value.bid, outStream, Context.NESTED);
+      } else {
+        throw new RuntimeException("invalid event");
+      }
+    }
+
+    @Override
+    public Event decode(
+        InputStream inStream, Coder.Context context)
+        throws CoderException, IOException {
+      int tag = INT_CODER.decode(inStream, context);
+      if (tag == 0) {
+        Person person = Person.CODER.decode(inStream, Context.NESTED);
+        return new Event(person);
+      } else if (tag == 1) {
+        Auction auction = Auction.CODER.decode(inStream, Context.NESTED);
+        return new Event(auction);
+      } else if (tag == 2) {
+        Bid bid = Bid.CODER.decode(inStream, Context.NESTED);
+        return new Event(bid);
+      } else {
+        throw new RuntimeException("invalid event encoding");
+      }
+    }
+  };
+
+  @Nullable
+  @org.apache.avro.reflect.Nullable
+  public final Person newPerson;
+
+  @Nullable
+  @org.apache.avro.reflect.Nullable
+  public final Auction newAuction;
+
+  @Nullable
+  @org.apache.avro.reflect.Nullable
+  public final Bid bid;
+
+  // For Avro only.
+  @SuppressWarnings("unused")
+  private Event() {
+    newPerson = null;
+    newAuction = null;
+    bid = null;
+  }
+
+  public Event(Person newPerson) {
+    this.newPerson = newPerson;
+    newAuction = null;
+    bid = null;
+  }
+
+  public Event(Auction newAuction) {
+    newPerson = null;
+    this.newAuction = newAuction;
+    bid = null;
+  }
+
+  public Event(Bid bid) {
+    newPerson = null;
+    newAuction = null;
+    this.bid = bid;
+  }
+
+  /**
+   * Return a copy of event which captures {@code annotation}.
+   * (Used for debugging).
+   */
+  public Event withAnnotation(String annotation) {
+    if (newPerson != null) {
+      return new Event(newPerson.withAnnotation(annotation));
+    } else if (newAuction != null) {
+      return new Event(newAuction.withAnnotation(annotation));
+    } else {
+      return new Event(bid.withAnnotation(annotation));
+    }
+  }
+
+  /**
+   * Does event have {@code annotation}? (Used for debugging.)
+   */
+  public boolean hasAnnotation(String annotation) {
+    if (newPerson != null) {
+      return newPerson.hasAnnotation(annotation);
+    } else if (newAuction != null) {
+      return newAuction.hasAnnotation(annotation);
+    } else {
+      return bid.hasAnnotation(annotation);
+    }
+  }
+
+  /**
+   * Remove {@code annotation} from event. (Used for debugging.)
+   */
+  public Event withoutAnnotation(String annotation) {
+    if (newPerson != null) {
+      return new Event(newPerson.withoutAnnotation(annotation));
+    } else if (newAuction != null) {
+      return new Event(newAuction.withoutAnnotation(annotation));
+    } else {
+      return new Event(bid.withoutAnnotation(annotation));
+    }
+  }
+
+  @Override
+  public long sizeInBytes() {
+    if (newPerson != null) {
+      return 1 + newPerson.sizeInBytes();
+    } else if (newAuction != null) {
+      return 1 + newAuction.sizeInBytes();
+    } else if (bid != null) {
+      return 1 + bid.sizeInBytes();
+    } else {
+      throw new RuntimeException("invalid event");
+    }
+  }
+
+  @Override
+  public String toString() {
+    if (newPerson != null) {
+      return newPerson.toString();
+    } else if (newAuction != null) {
+      return newAuction.toString();
+    } else if (bid != null) {
+      return bid.toString();
+    } else {
+      throw new RuntimeException("invalid event");
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/a7f9f7d0/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/model/IdNameReserve.java
----------------------------------------------------------------------
diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/model/IdNameReserve.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/model/IdNameReserve.java
new file mode 100644
index 0000000..5d22651
--- /dev/null
+++ b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/model/IdNameReserve.java
@@ -0,0 +1,100 @@
+/*
+ * 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.beam.integration.nexmark.model;
+
+import com.fasterxml.jackson.annotation.JsonProperty;
+import com.fasterxml.jackson.core.JsonProcessingException;
+
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
+import java.io.Serializable;
+
+import org.apache.beam.integration.nexmark.NexmarkUtils;
+import org.apache.beam.sdk.coders.AtomicCoder;
+import org.apache.beam.sdk.coders.Coder;
+import org.apache.beam.sdk.coders.CoderException;
+import org.apache.beam.sdk.coders.StringUtf8Coder;
+import org.apache.beam.sdk.coders.VarLongCoder;
+
+/**
+ * Result type of Query8.
+ */
+public class IdNameReserve implements KnownSize, Serializable {
+  private static final Coder<Long> LONG_CODER = VarLongCoder.of();
+  private static final Coder<String> STRING_CODER = StringUtf8Coder.of();
+
+  public static final Coder<IdNameReserve> CODER = new AtomicCoder<IdNameReserve>() {
+    @Override
+    public void encode(IdNameReserve value, OutputStream outStream,
+        Coder.Context context)
+        throws CoderException, IOException {
+      LONG_CODER.encode(value.id, outStream, Context.NESTED);
+      STRING_CODER.encode(value.name, outStream, Context.NESTED);
+      LONG_CODER.encode(value.reserve, outStream, Context.NESTED);
+    }
+
+    @Override
+    public IdNameReserve decode(
+        InputStream inStream, Coder.Context context)
+        throws CoderException, IOException {
+      long id = LONG_CODER.decode(inStream, Context.NESTED);
+      String name = STRING_CODER.decode(inStream, Context.NESTED);
+      long reserve = LONG_CODER.decode(inStream, Context.NESTED);
+      return new IdNameReserve(id, name, reserve);
+    }
+  };
+
+  @JsonProperty
+  public final long id;
+
+  @JsonProperty
+  public final String name;
+
+  /** Reserve price in cents. */
+  @JsonProperty
+  public final long reserve;
+
+  // For Avro only.
+  @SuppressWarnings("unused")
+  private IdNameReserve() {
+    id = 0;
+    name = null;
+    reserve = 0;
+  }
+
+  public IdNameReserve(long id, String name, long reserve) {
+    this.id = id;
+    this.name = name;
+    this.reserve = reserve;
+  }
+
+  @Override
+  public long sizeInBytes() {
+    return 8 + name.length() + 1 + 8;
+  }
+
+  @Override
+  public String toString() {
+    try {
+      return NexmarkUtils.MAPPER.writeValueAsString(this);
+    } catch (JsonProcessingException e) {
+      throw new RuntimeException(e);
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/a7f9f7d0/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/model/KnownSize.java
----------------------------------------------------------------------
diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/model/KnownSize.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/model/KnownSize.java
new file mode 100644
index 0000000..c742eac
--- /dev/null
+++ b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/model/KnownSize.java
@@ -0,0 +1,26 @@
+/*
+ * 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.beam.integration.nexmark.model;
+
+/**
+ * Interface for elements which can quickly estimate their encoded byte size.
+ */
+public interface KnownSize {
+  long sizeInBytes();
+}
+

http://git-wip-us.apache.org/repos/asf/beam/blob/a7f9f7d0/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/model/NameCityStateId.java
----------------------------------------------------------------------
diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/model/NameCityStateId.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/model/NameCityStateId.java
new file mode 100644
index 0000000..ac22879
--- /dev/null
+++ b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/model/NameCityStateId.java
@@ -0,0 +1,106 @@
+/*
+ * 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.beam.integration.nexmark.model;
+
+import com.fasterxml.jackson.annotation.JsonProperty;
+import com.fasterxml.jackson.core.JsonProcessingException;
+
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
+import java.io.Serializable;
+
+import org.apache.beam.integration.nexmark.NexmarkUtils;
+import org.apache.beam.sdk.coders.AtomicCoder;
+import org.apache.beam.sdk.coders.Coder;
+import org.apache.beam.sdk.coders.CoderException;
+import org.apache.beam.sdk.coders.StringUtf8Coder;
+import org.apache.beam.sdk.coders.VarLongCoder;
+
+/**
+ * Result of Query3.
+ */
+public class NameCityStateId implements KnownSize, Serializable {
+  private static final Coder<Long> LONG_CODER = VarLongCoder.of();
+  private static final Coder<String> STRING_CODER = StringUtf8Coder.of();
+
+  public static final Coder<NameCityStateId> CODER = new AtomicCoder<NameCityStateId>() {
+    @Override
+    public void encode(NameCityStateId value, OutputStream outStream,
+        Coder.Context context)
+        throws CoderException, IOException {
+      STRING_CODER.encode(value.name, outStream, Context.NESTED);
+      STRING_CODER.encode(value.city, outStream, Context.NESTED);
+      STRING_CODER.encode(value.state, outStream, Context.NESTED);
+      LONG_CODER.encode(value.id, outStream, Context.NESTED);
+    }
+
+    @Override
+    public NameCityStateId decode(
+        InputStream inStream, Coder.Context context)
+        throws CoderException, IOException {
+      String name = STRING_CODER.decode(inStream, Context.NESTED);
+      String city = STRING_CODER.decode(inStream, Context.NESTED);
+      String state = STRING_CODER.decode(inStream, Context.NESTED);
+      long id = LONG_CODER.decode(inStream, Context.NESTED);
+      return new NameCityStateId(name, city, state, id);
+    }
+  };
+
+  @JsonProperty
+  public final String name;
+
+  @JsonProperty
+  public final String city;
+
+  @JsonProperty
+  public final String state;
+
+  @JsonProperty
+  public final long id;
+
+  // For Avro only.
+  @SuppressWarnings("unused")
+  private NameCityStateId() {
+    name = null;
+    city = null;
+    state = null;
+    id = 0;
+  }
+
+  public NameCityStateId(String name, String city, String state, long id) {
+    this.name = name;
+    this.city = city;
+    this.state = state;
+    this.id = id;
+  }
+
+  @Override
+  public long sizeInBytes() {
+    return name.length() + 1 + city.length() + 1 + state.length() + 1 + 8;
+  }
+
+  @Override
+  public String toString() {
+    try {
+      return NexmarkUtils.MAPPER.writeValueAsString(this);
+    } catch (JsonProcessingException e) {
+      throw new RuntimeException(e);
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/a7f9f7d0/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/model/Person.java
----------------------------------------------------------------------
diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/model/Person.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/model/Person.java
new file mode 100644
index 0000000..85c7183
--- /dev/null
+++ b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/model/Person.java
@@ -0,0 +1,166 @@
+/*
+ * 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.beam.integration.nexmark.model;
+
+import com.fasterxml.jackson.annotation.JsonProperty;
+import com.fasterxml.jackson.core.JsonProcessingException;
+
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
+import java.io.Serializable;
+
+import org.apache.beam.integration.nexmark.NexmarkUtils;
+import org.apache.beam.sdk.coders.AtomicCoder;
+import org.apache.beam.sdk.coders.Coder;
+import org.apache.beam.sdk.coders.CoderException;
+import org.apache.beam.sdk.coders.StringUtf8Coder;
+import org.apache.beam.sdk.coders.VarLongCoder;
+
+/**
+ * A person either creating an auction or making a bid.
+ */
+public class Person implements KnownSize, Serializable {
+  private static final Coder<Long> LONG_CODER = VarLongCoder.of();
+  private static final Coder<String> STRING_CODER = StringUtf8Coder.of();
+  public static final Coder<Person> CODER = new AtomicCoder<Person>() {
+    @Override
+    public void encode(Person value, OutputStream outStream,
+        Coder.Context context)
+        throws CoderException, IOException {
+      LONG_CODER.encode(value.id, outStream, Context.NESTED);
+      STRING_CODER.encode(value.name, outStream, Context.NESTED);
+      STRING_CODER.encode(value.emailAddress, outStream, Context.NESTED);
+      STRING_CODER.encode(value.creditCard, outStream, Context.NESTED);
+      STRING_CODER.encode(value.city, outStream, Context.NESTED);
+      STRING_CODER.encode(value.state, outStream, Context.NESTED);
+      LONG_CODER.encode(value.dateTime, outStream, Context.NESTED);
+      STRING_CODER.encode(value.extra, outStream, Context.NESTED);
+    }
+
+    @Override
+    public Person decode(
+        InputStream inStream, Coder.Context context)
+        throws CoderException, IOException {
+      long id = LONG_CODER.decode(inStream, Context.NESTED);
+      String name = STRING_CODER.decode(inStream, Context.NESTED);
+      String emailAddress = STRING_CODER.decode(inStream, Context.NESTED);
+      String creditCard = STRING_CODER.decode(inStream, Context.NESTED);
+      String city = STRING_CODER.decode(inStream, Context.NESTED);
+      String state = STRING_CODER.decode(inStream, Context.NESTED);
+      long dateTime = LONG_CODER.decode(inStream, Context.NESTED);
+      String extra = STRING_CODER.decode(inStream, Context.NESTED);
+      return new Person(id, name, emailAddress, creditCard, city, state, dateTime, extra);
+    }
+  };
+
+  /** Id of person. */
+  @JsonProperty
+  public final long id; // primary key
+
+  /** Extra person properties. */
+  @JsonProperty
+  public final String name;
+
+  @JsonProperty
+  public final String emailAddress;
+
+  @JsonProperty
+  public final String creditCard;
+
+  @JsonProperty
+  public final String city;
+
+  @JsonProperty
+  public final String state;
+
+  @JsonProperty
+  public final long dateTime;
+
+  /** Additional arbitrary payload for performance testing. */
+  @JsonProperty
+  public final String extra;
+
+  // For Avro only.
+  @SuppressWarnings("unused")
+  private Person() {
+    id = 0;
+    name = null;
+    emailAddress = null;
+    creditCard = null;
+    city = null;
+    state = null;
+    dateTime = 0;
+    extra = null;
+  }
+
+  public Person(long id, String name, String emailAddress, String creditCard, String city,
+      String state, long dateTime, String extra) {
+    this.id = id;
+    this.name = name;
+    this.emailAddress = emailAddress;
+    this.creditCard = creditCard;
+    this.city = city;
+    this.state = state;
+    this.dateTime = dateTime;
+    this.extra = extra;
+  }
+
+  /**
+   * Return a copy of person which capture the given annotation.
+   * (Used for debugging).
+   */
+  public Person withAnnotation(String annotation) {
+    return new Person(id, name, emailAddress, creditCard, city, state, dateTime,
+        annotation + ": " + extra);
+  }
+
+  /**
+   * Does person have {@code annotation}? (Used for debugging.)
+   */
+  public boolean hasAnnotation(String annotation) {
+    return extra.startsWith(annotation + ": ");
+  }
+
+  /**
+   * Remove {@code annotation} from person. (Used for debugging.)
+   */
+  public Person withoutAnnotation(String annotation) {
+    if (hasAnnotation(annotation)) {
+      return new Person(id, name, emailAddress, creditCard, city, state, dateTime,
+          extra.substring(annotation.length() + 2));
+    } else {
+      return this;
+    }
+  }
+
+  @Override
+  public long sizeInBytes() {
+    return 8 + name.length() + 1 + emailAddress.length() + 1 + creditCard.length() + 1
+        + city.length() + 1 + state.length() + 8 + 1 + extra.length() + 1;
+  }
+
+  @Override
+  public String toString() {
+    try {
+      return NexmarkUtils.MAPPER.writeValueAsString(this);
+    } catch (JsonProcessingException e) {
+      throw new RuntimeException(e);
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/a7f9f7d0/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/model/SellerPrice.java
----------------------------------------------------------------------
diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/model/SellerPrice.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/model/SellerPrice.java
new file mode 100644
index 0000000..b7c2b14
--- /dev/null
+++ b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/model/SellerPrice.java
@@ -0,0 +1,91 @@
+/*
+ * 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.beam.integration.nexmark.model;
+
+import com.fasterxml.jackson.annotation.JsonProperty;
+import com.fasterxml.jackson.core.JsonProcessingException;
+
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
+import java.io.Serializable;
+
+import org.apache.beam.integration.nexmark.NexmarkUtils;
+import org.apache.beam.sdk.coders.AtomicCoder;
+import org.apache.beam.sdk.coders.Coder;
+import org.apache.beam.sdk.coders.CoderException;
+import org.apache.beam.sdk.coders.VarLongCoder;
+
+/**
+ * Result of Query6.
+ */
+public class SellerPrice implements KnownSize, Serializable {
+  private static final Coder<Long> LONG_CODER = VarLongCoder.of();
+
+  public static final Coder<SellerPrice> CODER = new AtomicCoder<SellerPrice>() {
+    @Override
+    public void encode(SellerPrice value, OutputStream outStream,
+        Coder.Context context)
+        throws CoderException, IOException {
+      LONG_CODER.encode(value.seller, outStream, Context.NESTED);
+      LONG_CODER.encode(value.price, outStream, Context.NESTED);
+    }
+
+    @Override
+    public SellerPrice decode(
+        InputStream inStream, Coder.Context context)
+        throws CoderException, IOException {
+      long seller = LONG_CODER.decode(inStream, Context.NESTED);
+      long price = LONG_CODER.decode(inStream, Context.NESTED);
+      return new SellerPrice(seller, price);
+    }
+  };
+
+  @JsonProperty
+  public final long seller;
+
+  /** Price in cents. */
+  @JsonProperty
+  public final long price;
+
+  // For Avro only.
+  @SuppressWarnings("unused")
+  private SellerPrice() {
+    seller = 0;
+    price = 0;
+  }
+
+  public SellerPrice(long seller, long price) {
+    this.seller = seller;
+    this.price = price;
+  }
+
+  @Override
+  public long sizeInBytes() {
+    return 8 + 8;
+  }
+
+  @Override
+  public String toString() {
+    try {
+      return NexmarkUtils.MAPPER.writeValueAsString(this);
+    } catch (JsonProcessingException e) {
+      throw new RuntimeException(e);
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/a7f9f7d0/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/model/package-info.java
----------------------------------------------------------------------
diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/model/package-info.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/model/package-info.java
new file mode 100644
index 0000000..e1d6113
--- /dev/null
+++ b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/model/package-info.java
@@ -0,0 +1,22 @@
+/*
+ * 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.
+ */
+
+/**
+ * Nexmark Benchmark Model.
+ */
+package org.apache.beam.integration.nexmark.model;

http://git-wip-us.apache.org/repos/asf/beam/blob/a7f9f7d0/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/package-info.java
----------------------------------------------------------------------
diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/package-info.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/package-info.java
index 65bf7d4..df6f09f 100644
--- a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/package-info.java
+++ b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/package-info.java
@@ -16,6 +16,6 @@
  * limitations under the License.
  */
 /**
- * Nexmark Benchmark Integration Queries.
+ * Nexmark.
  */
 package org.apache.beam.integration.nexmark;

http://git-wip-us.apache.org/repos/asf/beam/blob/a7f9f7d0/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query0.java
----------------------------------------------------------------------
diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query0.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query0.java
new file mode 100644
index 0000000..f60d5de
--- /dev/null
+++ b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query0.java
@@ -0,0 +1,72 @@
+/*
+ * 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.beam.integration.nexmark.queries;
+
+import java.io.ByteArrayInputStream;
+import java.io.ByteArrayOutputStream;
+import java.io.IOException;
+
+import org.apache.beam.integration.nexmark.NexmarkConfiguration;
+import org.apache.beam.integration.nexmark.NexmarkQuery;
+import org.apache.beam.integration.nexmark.NexmarkUtils;
+import org.apache.beam.integration.nexmark.model.Event;
+import org.apache.beam.integration.nexmark.model.KnownSize;
+import org.apache.beam.sdk.coders.Coder;
+import org.apache.beam.sdk.coders.CoderException;
+import org.apache.beam.sdk.transforms.Aggregator;
+import org.apache.beam.sdk.transforms.DoFn;
+import org.apache.beam.sdk.transforms.ParDo;
+import org.apache.beam.sdk.transforms.Sum;
+import org.apache.beam.sdk.values.PCollection;
+
+/**
+ * Query 0: Pass events through unchanged. However, force them to do a round trip through
+ * serialization so that we measure the impact of the choice of coders.
+ */
+public class Query0 extends NexmarkQuery {
+  public Query0(NexmarkConfiguration configuration) {
+    super(configuration, "Query0");
+  }
+
+  private PCollection<Event> applyTyped(PCollection<Event> events) {
+    final Coder<Event> coder = events.getCoder();
+    return events
+        // Force round trip through coder.
+        .apply(name + ".Serialize",
+            ParDo.of(new DoFn<Event, Event>() {
+                  private final Aggregator<Long, Long> bytes =
+                      createAggregator("bytes", Sum.ofLongs());
+
+                  @ProcessElement
+                  public void processElement(ProcessContext c) throws CoderException, IOException {
+                    ByteArrayOutputStream outStream = new ByteArrayOutputStream();
+                    coder.encode(c.element(), outStream, Coder.Context.OUTER);
+                    byte[] byteArray = outStream.toByteArray();
+                    bytes.addValue((long) byteArray.length);
+                    ByteArrayInputStream inStream = new ByteArrayInputStream(byteArray);
+                    Event event = coder.decode(inStream, Coder.Context.OUTER);
+                    c.output(event);
+                  }
+                }));
+  }
+
+  @Override
+  protected PCollection<KnownSize> applyPrim(PCollection<Event> events) {
+    return NexmarkUtils.castToKnownSize(name, applyTyped(events));
+  }
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/a7f9f7d0/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query0Model.java
----------------------------------------------------------------------
diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query0Model.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query0Model.java
new file mode 100644
index 0000000..991b1d4
--- /dev/null
+++ b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query0Model.java
@@ -0,0 +1,67 @@
+/*
+ * 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.beam.integration.nexmark.queries;
+
+import java.util.Collection;
+import java.util.Iterator;
+
+import org.apache.beam.integration.nexmark.AbstractSimulator;
+import org.apache.beam.integration.nexmark.NexmarkConfiguration;
+import org.apache.beam.integration.nexmark.NexmarkQueryModel;
+import org.apache.beam.integration.nexmark.NexmarkUtils;
+import org.apache.beam.integration.nexmark.model.Event;
+import org.apache.beam.sdk.values.TimestampedValue;
+
+/**
+ * A direct implementation of {@link Query0}.
+ */
+public class Query0Model extends NexmarkQueryModel {
+  /**
+   * Simulator for query 0.
+   */
+  private class Simulator extends AbstractSimulator<Event, Event> {
+    public Simulator(NexmarkConfiguration configuration) {
+      super(NexmarkUtils.standardEventIterator(configuration));
+    }
+
+    @Override
+    protected void run() {
+      TimestampedValue<Event> timestampedEvent = nextInput();
+      if (timestampedEvent == null) {
+        allDone();
+        return;
+      }
+      addResult(timestampedEvent);
+      //TODO test fails because offset of some hundreds of ms beween expect and actual
+    }
+  }
+
+  public Query0Model(NexmarkConfiguration configuration) {
+    super(configuration);
+  }
+
+  @Override
+  protected AbstractSimulator<?, ?> simulator() {
+    return new Simulator(configuration);
+  }
+
+  @Override
+  protected <T> Collection<String> toCollection(Iterator<TimestampedValue<T>> itr) {
+    return toValueTimestampOrder(itr);
+  }
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/a7f9f7d0/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query1.java
----------------------------------------------------------------------
diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query1.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query1.java
new file mode 100644
index 0000000..0be77ce
--- /dev/null
+++ b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query1.java
@@ -0,0 +1,68 @@
+/*
+ * 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.beam.integration.nexmark.queries;
+
+import org.apache.beam.integration.nexmark.NexmarkConfiguration;
+import org.apache.beam.integration.nexmark.NexmarkQuery;
+import org.apache.beam.integration.nexmark.NexmarkUtils;
+import org.apache.beam.integration.nexmark.model.Bid;
+import org.apache.beam.integration.nexmark.model.Event;
+import org.apache.beam.integration.nexmark.model.KnownSize;
+import org.apache.beam.sdk.transforms.DoFn;
+import org.apache.beam.sdk.transforms.ParDo;
+import org.apache.beam.sdk.values.PCollection;
+
+/**
+ * Query 1, 'Currency Conversion'. Convert each bid value from dollars to euros.
+ * In CQL syntax:
+ *
+ * <pre>
+ * SELECT Istream(auction, DOLTOEUR(price), bidder, datetime)
+ * FROM bid [ROWS UNBOUNDED];
+ * </pre>
+ *
+ * <p>To make things more interesting, allow the 'currency conversion' to be arbitrarily
+ * slowed down.
+ */
+public class Query1 extends NexmarkQuery {
+  public Query1(NexmarkConfiguration configuration) {
+    super(configuration, "Query1");
+  }
+
+  private PCollection<Bid> applyTyped(PCollection<Event> events) {
+    return events
+        // Only want the bid events.
+        .apply(JUST_BIDS)
+
+        // Map the conversion function over all bids.
+        .apply(name + ".ToEuros",
+            ParDo.of(new DoFn<Bid, Bid>() {
+                  @ProcessElement
+                  public void processElement(ProcessContext c) {
+                    Bid bid = c.element();
+                    c.output(new Bid(
+                        bid.auction, bid.bidder, (bid.price * 89) / 100, bid.dateTime, bid.extra));
+                  }
+                }));
+  }
+
+  @Override
+  protected PCollection<KnownSize> applyPrim(PCollection<Event> events) {
+    return NexmarkUtils.castToKnownSize(name, applyTyped(events));
+  }
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/a7f9f7d0/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query10.java
----------------------------------------------------------------------
diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query10.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query10.java
new file mode 100644
index 0000000..6912ed1
--- /dev/null
+++ b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query10.java
@@ -0,0 +1,384 @@
+/*
+ * 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.beam.integration.nexmark.queries;
+
+import static com.google.common.base.Preconditions.checkState;
+
+import com.google.cloud.hadoop.gcsio.GoogleCloudStorageWriteChannel;
+import java.io.IOException;
+import java.io.OutputStream;
+import java.io.Serializable;
+import java.nio.channels.Channels;
+import java.nio.channels.WritableByteChannel;
+import java.util.concurrent.ThreadLocalRandom;
+import javax.annotation.Nullable;
+import org.apache.beam.integration.nexmark.NexmarkConfiguration;
+import org.apache.beam.integration.nexmark.NexmarkQuery;
+import org.apache.beam.integration.nexmark.NexmarkUtils;
+import org.apache.beam.integration.nexmark.model.Done;
+import org.apache.beam.integration.nexmark.model.Event;
+import org.apache.beam.integration.nexmark.model.KnownSize;
+import org.apache.beam.sdk.coders.Coder;
+import org.apache.beam.sdk.options.GcsOptions;
+import org.apache.beam.sdk.transforms.Aggregator;
+import org.apache.beam.sdk.transforms.DoFn;
+import org.apache.beam.sdk.transforms.GroupByKey;
+import org.apache.beam.sdk.transforms.ParDo;
+import org.apache.beam.sdk.transforms.Sum;
+import org.apache.beam.sdk.transforms.windowing.AfterEach;
+import org.apache.beam.sdk.transforms.windowing.AfterFirst;
+import org.apache.beam.sdk.transforms.windowing.AfterPane;
+import org.apache.beam.sdk.transforms.windowing.AfterProcessingTime;
+import org.apache.beam.sdk.transforms.windowing.AfterWatermark;
+import org.apache.beam.sdk.transforms.windowing.BoundedWindow;
+import org.apache.beam.sdk.transforms.windowing.FixedWindows;
+import org.apache.beam.sdk.transforms.windowing.PaneInfo;
+import org.apache.beam.sdk.transforms.windowing.PaneInfo.Timing;
+import org.apache.beam.sdk.transforms.windowing.Repeatedly;
+import org.apache.beam.sdk.transforms.windowing.Window;
+import org.apache.beam.sdk.util.GcsIOChannelFactory;
+import org.apache.beam.sdk.values.KV;
+import org.apache.beam.sdk.values.PCollection;
+import org.joda.time.Duration;
+import org.joda.time.Instant;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+/**
+ * Query "10", 'Log to sharded files' (Not in original suite.)
+ *
+ * <p>Every windowSizeSec, save all events from the last period into 2*maxWorkers log files.
+ */
+public class Query10 extends NexmarkQuery {
+  private static final Logger LOG = LoggerFactory.getLogger(Query10.class);
+  private static final int CHANNEL_BUFFER = 8 << 20; // 8MB
+  private static final int NUM_SHARDS_PER_WORKER = 5;
+  private static final Duration LATE_BATCHING_PERIOD = Duration.standardSeconds(10);
+
+  /**
+   * Capture everything we need to know about the records in a single output file.
+   */
+  private static class OutputFile implements Serializable {
+    /** Maximum possible timestamp of records in file. */
+    private final Instant maxTimestamp;
+    /** Shard within window. */
+    private final String shard;
+    /** Index of file in all files in shard. */
+    private final long index;
+    /** Timing of records in this file. */
+    private final PaneInfo.Timing timing;
+    /** Path to file containing records, or {@literal null} if no output required. */
+    @Nullable
+    private final String filename;
+
+    public OutputFile(
+        Instant maxTimestamp,
+        String shard,
+        long index,
+        PaneInfo.Timing timing,
+        @Nullable String filename) {
+      this.maxTimestamp = maxTimestamp;
+      this.shard = shard;
+      this.index = index;
+      this.timing = timing;
+      this.filename = filename;
+    }
+
+    @Override
+    public String toString() {
+      return String.format("%s %s %d %s %s\n", maxTimestamp, shard, index, timing, filename);
+    }
+  }
+
+  /**
+   * GCS uri prefix for all log and 'finished' files. If null they won't be written.
+   */
+  @Nullable
+  private String outputPath;
+
+  /**
+   * Maximum number of workers, used to determine log sharding factor.
+   */
+  private int maxNumWorkers;
+
+  public Query10(NexmarkConfiguration configuration) {
+    super(configuration, "Query10");
+  }
+
+  public void setOutputPath(@Nullable String outputPath) {
+    this.outputPath = outputPath;
+  }
+
+  public void setMaxNumWorkers(int maxNumWorkers) {
+    this.maxNumWorkers = maxNumWorkers;
+  }
+
+  /**
+   * Return channel for writing bytes to GCS.
+   *
+   * @throws IOException
+   */
+  private WritableByteChannel openWritableGcsFile(GcsOptions options, String filename)
+      throws IOException {
+    WritableByteChannel channel =
+            GcsIOChannelFactory.fromOptions(options).create(filename, "text/plain");
+    checkState(channel instanceof GoogleCloudStorageWriteChannel);
+    ((GoogleCloudStorageWriteChannel) channel).setUploadBufferSize(CHANNEL_BUFFER);
+    return channel;
+  }
+
+  /** Return a short string to describe {@code timing}. */
+  private String timingToString(PaneInfo.Timing timing) {
+    switch (timing) {
+      case EARLY:
+        return "E";
+      case ON_TIME:
+        return "O";
+      case LATE:
+        return "L";
+    }
+    throw new RuntimeException(); // cases are exhaustive
+  }
+
+  /** Construct an {@link OutputFile} for {@code pane} in {@code window} for {@code shard}. */
+  private OutputFile outputFileFor(BoundedWindow window, String shard, PaneInfo pane) {
+    @Nullable String filename =
+        outputPath == null
+        ? null
+        : String.format("%s/LOG-%s-%s-%03d-%s-%x",
+            outputPath, window.maxTimestamp(), shard, pane.getIndex(),
+            timingToString(pane.getTiming()),
+            ThreadLocalRandom.current().nextLong());
+    return new OutputFile(window.maxTimestamp(), shard, pane.getIndex(),
+        pane.getTiming(), filename);
+  }
+
+  /**
+   * Return path to which we should write the index for {@code window}, or {@literal null}
+   * if no output required.
+   */
+  @Nullable
+  private String indexPathFor(BoundedWindow window) {
+    if (outputPath == null) {
+      return null;
+    }
+    return String.format("%s/INDEX-%s", outputPath, window.maxTimestamp());
+  }
+
+  private PCollection<Done> applyTyped(PCollection<Event> events) {
+    final int numLogShards = maxNumWorkers * NUM_SHARDS_PER_WORKER;
+
+    return events.apply(name + ".ShardEvents",
+            ParDo.of(new DoFn<Event, KV<String, Event>>() {
+                      final Aggregator<Long, Long> lateCounter =
+                          createAggregator("actuallyLateEvent", Sum.ofLongs());
+                      final Aggregator<Long, Long> onTimeCounter =
+                          createAggregator("actuallyOnTimeEvent", Sum.ofLongs());
+
+                      @ProcessElement
+                      public void processElement(ProcessContext c) {
+                        if (c.element().hasAnnotation("LATE")) {
+                          lateCounter.addValue(1L);
+                          LOG.error("Observed late: %s", c.element());
+                        } else {
+                          onTimeCounter.addValue(1L);
+                        }
+                        int shardNum = (int) Math.abs((long) c.element().hashCode() % numLogShards);
+                        String shard = String.format("shard-%05d-of-%05d", shardNum, numLogShards);
+                        c.output(KV.of(shard, c.element()));
+                      }
+                    }))
+        .apply(name + ".WindowEvents",
+                Window.<KV<String, Event>>into(
+            FixedWindows.of(Duration.standardSeconds(configuration.windowSizeSec)))
+            .triggering(AfterEach.inOrder(
+                Repeatedly
+                    .forever(AfterPane.elementCountAtLeast(configuration.maxLogEvents))
+                    .orFinally(AfterWatermark.pastEndOfWindow()),
+                Repeatedly.forever(
+                    AfterFirst.of(AfterPane.elementCountAtLeast(configuration.maxLogEvents),
+                        AfterProcessingTime.pastFirstElementInPane()
+                                           .plusDelayOf(LATE_BATCHING_PERIOD)))))
+            .discardingFiredPanes()
+            // Use a 1 day allowed lateness so that any forgotten hold will stall the
+            // pipeline for that period and be very noticeable.
+            .withAllowedLateness(Duration.standardDays(1)))
+        .apply(name + ".GroupByKey", GroupByKey.<String, Event>create())
+        .apply(name + ".CheckForLateEvents",
+            ParDo.of(new DoFn<KV<String, Iterable<Event>>,
+                     KV<String, Iterable<Event>>>() {
+                   final Aggregator<Long, Long> earlyCounter =
+                       createAggregator("earlyShard", Sum.ofLongs());
+                   final Aggregator<Long, Long> onTimeCounter =
+                       createAggregator("onTimeShard", Sum.ofLongs());
+                   final Aggregator<Long, Long> lateCounter =
+                       createAggregator("lateShard", Sum.ofLongs());
+                   final Aggregator<Long, Long> unexpectedLatePaneCounter =
+                       createAggregator("ERROR_unexpectedLatePane", Sum.ofLongs());
+                   final Aggregator<Long, Long> unexpectedOnTimeElementCounter =
+                       createAggregator("ERROR_unexpectedOnTimeElement", Sum.ofLongs());
+
+                   @ProcessElement
+                   public void processElement(ProcessContext c, BoundedWindow window) {
+                     int numLate = 0;
+                     int numOnTime = 0;
+                     for (Event event : c.element().getValue()) {
+                       if (event.hasAnnotation("LATE")) {
+                         numLate++;
+                       } else {
+                         numOnTime++;
+                       }
+                     }
+                     String shard = c.element().getKey();
+                     LOG.error(
+                         "%s with timestamp %s has %d actually late and %d on-time "
+                             + "elements in pane %s for window %s",
+                         shard, c.timestamp(), numLate, numOnTime, c.pane(),
+                         window.maxTimestamp());
+                     if (c.pane().getTiming() == PaneInfo.Timing.LATE) {
+                       if (numLate == 0) {
+                         LOG.error(
+                             "ERROR! No late events in late pane for %s", shard);
+                         unexpectedLatePaneCounter.addValue(1L);
+                       }
+                       if (numOnTime > 0) {
+                         LOG.error(
+                             "ERROR! Have %d on-time events in late pane for %s",
+                             numOnTime, shard);
+                         unexpectedOnTimeElementCounter.addValue(1L);
+                       }
+                       lateCounter.addValue(1L);
+                     } else if (c.pane().getTiming() == PaneInfo.Timing.EARLY) {
+                       if (numOnTime + numLate < configuration.maxLogEvents) {
+                         LOG.error(
+                             "ERROR! Only have %d events in early pane for %s",
+                             numOnTime + numLate, shard);
+                       }
+                       earlyCounter.addValue(1L);
+                     } else {
+                       onTimeCounter.addValue(1L);
+                     }
+                     c.output(c.element());
+                   }
+                 }))
+        .apply(name + ".UploadEvents",
+            ParDo.of(new DoFn<KV<String, Iterable<Event>>,
+                     KV<Void, OutputFile>>() {
+                   final Aggregator<Long, Long> savedFileCounter =
+                       createAggregator("savedFile", Sum.ofLongs());
+                   final Aggregator<Long, Long> writtenRecordsCounter =
+                       createAggregator("writtenRecords", Sum.ofLongs());
+
+                   @ProcessElement
+                   public void processElement(ProcessContext c, BoundedWindow window)
+                           throws IOException {
+                     String shard = c.element().getKey();
+                     GcsOptions options = c.getPipelineOptions().as(GcsOptions.class);
+                     OutputFile outputFile = outputFileFor(window, shard, c.pane());
+                     LOG.error(
+                         "Writing %s with record timestamp %s, window timestamp %s, pane %s",
+                         shard, c.timestamp(), window.maxTimestamp(), c.pane());
+                     if (outputFile.filename != null) {
+                       LOG.error("Beginning write to '%s'", outputFile.filename);
+                       int n = 0;
+                       try (OutputStream output =
+                                Channels.newOutputStream(openWritableGcsFile(options, outputFile
+                                    .filename))) {
+                         for (Event event : c.element().getValue()) {
+                           Event.CODER.encode(event, output, Coder.Context.OUTER);
+                           writtenRecordsCounter.addValue(1L);
+                           if (++n % 10000 == 0) {
+                             LOG.error("So far written %d records to '%s'", n,
+                                 outputFile.filename);
+                           }
+                         }
+                       }
+                       LOG.error("Written all %d records to '%s'", n, outputFile.filename);
+                     }
+                     savedFileCounter.addValue(1L);
+                     c.output(KV.<Void, OutputFile>of(null, outputFile));
+                   }
+                 }))
+        // Clear fancy triggering from above.
+        .apply(name + ".WindowLogFiles", Window.<KV<Void, OutputFile>>into(
+            FixedWindows.of(Duration.standardSeconds(configuration.windowSizeSec)))
+            .triggering(AfterWatermark.pastEndOfWindow())
+            // We expect no late data here, but we'll assume the worst so we can detect any.
+            .withAllowedLateness(Duration.standardDays(1))
+            .discardingFiredPanes())
+      // TODO etienne: unnecessary groupByKey? because aggregators are shared in parallel
+      // and Pardo is also in parallel, why group all elements in memory of the same executor?
+      .apply(name + ".GroupByKey2", GroupByKey.<Void, OutputFile>create())
+        .apply(name + ".Index",
+            ParDo.of(new DoFn<KV<Void, Iterable<OutputFile>>, Done>() {
+                   final Aggregator<Long, Long> unexpectedLateCounter =
+                       createAggregator("ERROR_unexpectedLate", Sum.ofLongs());
+                   final Aggregator<Long, Long> unexpectedEarlyCounter =
+                       createAggregator("ERROR_unexpectedEarly", Sum.ofLongs());
+                   final Aggregator<Long, Long> unexpectedIndexCounter =
+                       createAggregator("ERROR_unexpectedIndex", Sum.ofLongs());
+                   final Aggregator<Long, Long> finalizedCounter =
+                       createAggregator("indexed", Sum.ofLongs());
+
+                   @ProcessElement
+                   public void processElement(ProcessContext c, BoundedWindow window)
+                           throws IOException {
+                     if (c.pane().getTiming() == Timing.LATE) {
+                       unexpectedLateCounter.addValue(1L);
+                       LOG.error("ERROR! Unexpected LATE pane: %s", c.pane());
+                     } else if (c.pane().getTiming() == Timing.EARLY) {
+                       unexpectedEarlyCounter.addValue(1L);
+                       LOG.error("ERROR! Unexpected EARLY pane: %s", c.pane());
+                     } else if (c.pane().getTiming() == Timing.ON_TIME
+                         && c.pane().getIndex() != 0) {
+                       unexpectedIndexCounter.addValue(1L);
+                       LOG.error("ERROR! Unexpected ON_TIME pane index: %s", c.pane());
+                     } else {
+                       GcsOptions options = c.getPipelineOptions().as(GcsOptions.class);
+                       LOG.error(
+                           "Index with record timestamp %s, window timestamp %s, pane %s",
+                           c.timestamp(), window.maxTimestamp(), c.pane());
+
+                       @Nullable String filename = indexPathFor(window);
+                       if (filename != null) {
+                         LOG.error("Beginning write to '%s'", filename);
+                         int n = 0;
+                         try (OutputStream output =
+                                  Channels.newOutputStream(
+                                      openWritableGcsFile(options, filename))) {
+                           for (OutputFile outputFile : c.element().getValue()) {
+                             output.write(outputFile.toString().getBytes());
+                             n++;
+                           }
+                         }
+                         LOG.error("Written all %d lines to '%s'", n, filename);
+                       }
+                       c.output(
+                           new Done("written for timestamp " + window.maxTimestamp()));
+                       finalizedCounter.addValue(1L);
+                     }
+                   }
+                 }));
+  }
+
+  @Override
+  protected PCollection<KnownSize> applyPrim(PCollection<Event> events) {
+    return NexmarkUtils.castToKnownSize(name, applyTyped(events));
+  }
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/a7f9f7d0/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query11.java
----------------------------------------------------------------------
diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query11.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query11.java
new file mode 100644
index 0000000..4da99eb
--- /dev/null
+++ b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query11.java
@@ -0,0 +1,80 @@
+/*
+ * 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.beam.integration.nexmark.queries;
+
+import org.apache.beam.integration.nexmark.NexmarkConfiguration;
+import org.apache.beam.integration.nexmark.NexmarkQuery;
+import org.apache.beam.integration.nexmark.NexmarkUtils;
+import org.apache.beam.integration.nexmark.model.Bid;
+import org.apache.beam.integration.nexmark.model.BidsPerSession;
+import org.apache.beam.integration.nexmark.model.Event;
+import org.apache.beam.integration.nexmark.model.KnownSize;
+import org.apache.beam.sdk.transforms.Count;
+import org.apache.beam.sdk.transforms.DoFn;
+import org.apache.beam.sdk.transforms.ParDo;
+import org.apache.beam.sdk.transforms.windowing.AfterPane;
+import org.apache.beam.sdk.transforms.windowing.Repeatedly;
+import org.apache.beam.sdk.transforms.windowing.Sessions;
+import org.apache.beam.sdk.transforms.windowing.Window;
+import org.apache.beam.sdk.values.KV;
+import org.apache.beam.sdk.values.PCollection;
+import org.joda.time.Duration;
+
+/**
+ * Query "11", 'User sessions' (Not in original suite.)
+ *
+ * <p>Group bids by the same user into sessions with {@code windowSizeSec} max gap.
+ * However limit the session to at most {@code maxLogEvents}. Emit the number of
+ * bids per session.
+ */
+public class Query11 extends NexmarkQuery {
+  public Query11(NexmarkConfiguration configuration) {
+    super(configuration, "Query11");
+  }
+
+  private PCollection<BidsPerSession> applyTyped(PCollection<Event> events) {
+    return events.apply(JUST_BIDS)
+        .apply(name + ".Rekey",
+          // TODO etienne: why not avoid this ParDo and do a Cont.perElement?
+          ParDo.of(new DoFn<Bid, KV<Long, Void>>() {
+                  @ProcessElement
+                  public void processElement(ProcessContext c) {
+                    Bid bid = c.element();
+                    c.output(KV.of(bid.bidder, (Void) null));
+                  }
+                }))
+        .apply(Window.<KV<Long, Void>>into(
+            Sessions.withGapDuration(Duration.standardSeconds(configuration.windowSizeSec)))
+        .triggering(Repeatedly.forever(AfterPane.elementCountAtLeast(configuration.maxLogEvents)))
+        .discardingFiredPanes()
+        .withAllowedLateness(Duration.standardSeconds(configuration.occasionalDelaySec / 2)))
+        .apply(Count.<Long, Void>perKey())
+        .apply(name + ".ToResult",
+            ParDo.of(new DoFn<KV<Long, Long>, BidsPerSession>() {
+                  @ProcessElement
+                  public void processElement(ProcessContext c) {
+                    c.output(new BidsPerSession(c.element().getKey(), c.element().getValue()));
+                  }
+                }));
+  }
+
+  @Override
+  protected PCollection<KnownSize> applyPrim(PCollection<Event> events) {
+    return NexmarkUtils.castToKnownSize(name, applyTyped(events));
+  }
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/a7f9f7d0/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query12.java
----------------------------------------------------------------------
diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query12.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query12.java
new file mode 100644
index 0000000..c67401b
--- /dev/null
+++ b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query12.java
@@ -0,0 +1,84 @@
+/*
+ * 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.beam.integration.nexmark.queries;
+
+import org.apache.beam.integration.nexmark.NexmarkConfiguration;
+import org.apache.beam.integration.nexmark.NexmarkQuery;
+import org.apache.beam.integration.nexmark.NexmarkUtils;
+import org.apache.beam.integration.nexmark.model.Bid;
+import org.apache.beam.integration.nexmark.model.BidsPerSession;
+import org.apache.beam.integration.nexmark.model.Event;
+import org.apache.beam.integration.nexmark.model.KnownSize;
+import org.apache.beam.sdk.transforms.Count;
+import org.apache.beam.sdk.transforms.DoFn;
+import org.apache.beam.sdk.transforms.ParDo;
+import org.apache.beam.sdk.transforms.windowing.AfterProcessingTime;
+import org.apache.beam.sdk.transforms.windowing.GlobalWindows;
+import org.apache.beam.sdk.transforms.windowing.Repeatedly;
+import org.apache.beam.sdk.transforms.windowing.Window;
+import org.apache.beam.sdk.values.KV;
+import org.apache.beam.sdk.values.PCollection;
+import org.joda.time.Duration;
+
+/**
+ * Query "12", 'Processing time windows' (Not in original suite.)
+ *
+ * <p>Group bids by the same user into processing time windows of windowSize. Emit the count
+ * of bids per window.
+ */
+public class Query12 extends NexmarkQuery {
+  public Query12(NexmarkConfiguration configuration) {
+    super(configuration, "Query12");
+  }
+
+  private PCollection<BidsPerSession> applyTyped(PCollection<Event> events) {
+    return events
+        .apply(JUST_BIDS)
+        .apply(name + ".Rekey",
+          // TODO etienne: why not avoid this ParDo and do a Cont.perElement?
+            ParDo.of(new DoFn<Bid, KV<Long, Void>>() {
+                   @ProcessElement
+                   public void processElement(ProcessContext c) {
+                     Bid bid = c.element();
+                     c.output(KV.of(bid.bidder, (Void) null));
+                   }
+                 }))
+        .apply(Window.<KV<Long, Void>>into(new GlobalWindows())
+            .triggering(
+                Repeatedly.forever(
+                    AfterProcessingTime.pastFirstElementInPane()
+                                       .plusDelayOf(
+                                           Duration.standardSeconds(configuration.windowSizeSec))))
+            .discardingFiredPanes()
+            .withAllowedLateness(Duration.ZERO))
+        .apply(Count.<Long, Void>perKey())
+        .apply(name + ".ToResult",
+            ParDo.of(new DoFn<KV<Long, Long>, BidsPerSession>() {
+                   @ProcessElement
+                   public void processElement(ProcessContext c) {
+                     c.output(
+                         new BidsPerSession(c.element().getKey(), c.element().getValue()));
+                   }
+                 }));
+  }
+
+  @Override
+  protected PCollection<KnownSize> applyPrim(PCollection<Event> events) {
+    return NexmarkUtils.castToKnownSize(name, applyTyped(events));
+  }
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/a7f9f7d0/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query1Model.java
----------------------------------------------------------------------
diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query1Model.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query1Model.java
new file mode 100644
index 0000000..58037d3
--- /dev/null
+++ b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query1Model.java
@@ -0,0 +1,79 @@
+/*
+ * 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.beam.integration.nexmark.queries;
+
+import java.io.Serializable;
+import java.util.Collection;
+import java.util.Iterator;
+
+import org.apache.beam.integration.nexmark.AbstractSimulator;
+import org.apache.beam.integration.nexmark.NexmarkConfiguration;
+import org.apache.beam.integration.nexmark.NexmarkQueryModel;
+import org.apache.beam.integration.nexmark.NexmarkUtils;
+import org.apache.beam.integration.nexmark.model.Bid;
+import org.apache.beam.integration.nexmark.model.Event;
+import org.apache.beam.sdk.values.TimestampedValue;
+
+/**
+ * A direct implementation of {@link Query1}.
+ */
+public class Query1Model extends NexmarkQueryModel implements Serializable {
+  /**
+   * Simulator for query 1.
+   */
+  private class Simulator extends AbstractSimulator<Event, Bid> {
+    public Simulator(NexmarkConfiguration configuration) {
+      super(NexmarkUtils.standardEventIterator(configuration));
+    }
+
+    @Override
+    protected void run() {
+      TimestampedValue<Event> timestampedEvent = nextInput();
+      if (timestampedEvent == null) {
+        allDone();
+        return;
+      }
+      Event event = timestampedEvent.getValue();
+      if (event.bid == null) {
+        // Ignore non-bid events.
+        return;
+      }
+      Bid bid = event.bid;
+      Bid resultBid =
+          new Bid(bid.auction, bid.bidder, bid.price * 89 / 100, bid.dateTime, bid.extra);
+      TimestampedValue<Bid> result =
+          TimestampedValue.of(resultBid, timestampedEvent.getTimestamp());
+      addResult(result);
+      //TODO test fails because offset of some hundreds of ms beween expect and actual
+    }
+  }
+
+  public Query1Model(NexmarkConfiguration configuration) {
+    super(configuration);
+  }
+
+  @Override
+  public AbstractSimulator<?, ?> simulator() {
+    return new Simulator(configuration);
+  }
+
+  @Override
+  protected <T> Collection<String> toCollection(Iterator<TimestampedValue<T>> itr) {
+    return toValueTimestampOrder(itr);
+  }
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/a7f9f7d0/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query2.java
----------------------------------------------------------------------
diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query2.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query2.java
new file mode 100644
index 0000000..4c8f878
--- /dev/null
+++ b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query2.java
@@ -0,0 +1,80 @@
+/*
+ * 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.beam.integration.nexmark.queries;
+
+import org.apache.beam.integration.nexmark.NexmarkConfiguration;
+import org.apache.beam.integration.nexmark.NexmarkQuery;
+import org.apache.beam.integration.nexmark.NexmarkUtils;
+import org.apache.beam.integration.nexmark.model.AuctionPrice;
+import org.apache.beam.integration.nexmark.model.Bid;
+import org.apache.beam.integration.nexmark.model.Event;
+import org.apache.beam.integration.nexmark.model.KnownSize;
+import org.apache.beam.sdk.transforms.DoFn;
+import org.apache.beam.sdk.transforms.Filter;
+import org.apache.beam.sdk.transforms.ParDo;
+import org.apache.beam.sdk.transforms.SerializableFunction;
+import org.apache.beam.sdk.values.PCollection;
+
+/**
+ * Query 2, 'Filtering. Find bids with specific auction ids and show their bid price.
+ * In CQL syntax:
+ *
+ * <pre>
+ * SELECT Rstream(auction, price)
+ * FROM Bid [NOW]
+ * WHERE auction = 1007 OR auction = 1020 OR auction = 2001 OR auction = 2019 OR auction = 2087;
+ * </pre>
+ *
+ * <p>As written that query will only yield a few hundred results over event streams of
+ * arbitrary size. To make it more interesting we instead choose bids for every
+ * {@code auctionSkip}'th auction.
+ */
+public class Query2 extends NexmarkQuery {
+  public Query2(NexmarkConfiguration configuration) {
+    super(configuration, "Query2");
+  }
+
+  private PCollection<AuctionPrice> applyTyped(PCollection<Event> events) {
+    return events
+        // Only want the bid events.
+        .apply(JUST_BIDS)
+
+        // Select just the bids for the auctions we care about.
+        .apply(Filter.by(new SerializableFunction<Bid, Boolean>() {
+          @Override
+          public Boolean apply(Bid bid) {
+            return bid.auction % configuration.auctionSkip == 0;
+          }
+        }))
+
+        // Project just auction id and price.
+        .apply(name + ".Project",
+            ParDo.of(new DoFn<Bid, AuctionPrice>() {
+                  @ProcessElement
+                  public void processElement(ProcessContext c) {
+                    Bid bid = c.element();
+                    c.output(new AuctionPrice(bid.auction, bid.price));
+                  }
+                }));
+  }
+
+  @Override
+  protected PCollection<KnownSize> applyPrim(PCollection<Event> events) {
+    return NexmarkUtils.castToKnownSize(name, applyTyped(events));
+  }
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/a7f9f7d0/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query2Model.java
----------------------------------------------------------------------
diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query2Model.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query2Model.java
new file mode 100644
index 0000000..f578e4c
--- /dev/null
+++ b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query2Model.java
@@ -0,0 +1,82 @@
+/*
+ * 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.beam.integration.nexmark.queries;
+
+import java.io.Serializable;
+import java.util.Collection;
+import java.util.Iterator;
+
+import org.apache.beam.integration.nexmark.AbstractSimulator;
+import org.apache.beam.integration.nexmark.NexmarkConfiguration;
+import org.apache.beam.integration.nexmark.NexmarkQueryModel;
+import org.apache.beam.integration.nexmark.NexmarkUtils;
+import org.apache.beam.integration.nexmark.model.AuctionPrice;
+import org.apache.beam.integration.nexmark.model.Bid;
+import org.apache.beam.integration.nexmark.model.Event;
+import org.apache.beam.sdk.values.TimestampedValue;
+
+/**
+ * A direct implementation of {@link Query2}.
+ */
+public class Query2Model extends NexmarkQueryModel implements Serializable {
+  /**
+   * Simulator for query 2.
+   */
+  private class Simulator extends AbstractSimulator<Event, AuctionPrice> {
+    public Simulator(NexmarkConfiguration configuration) {
+      super(NexmarkUtils.standardEventIterator(configuration));
+    }
+
+    @Override
+    protected void run() {
+      TimestampedValue<Event> timestampedEvent = nextInput();
+      if (timestampedEvent == null) {
+        allDone();
+        return;
+      }
+      Event event = timestampedEvent.getValue();
+      if (event.bid == null) {
+        // Ignore non bid events.
+        return;
+      }
+      Bid bid = event.bid;
+      if (bid.auction % configuration.auctionSkip != 0) {
+        // Ignore bids for auctions we don't care about.
+        return;
+      }
+      AuctionPrice auctionPrice = new AuctionPrice(bid.auction, bid.price);
+      TimestampedValue<AuctionPrice> result =
+          TimestampedValue.of(auctionPrice, timestampedEvent.getTimestamp());
+      addResult(result);
+    }
+  }
+
+  public Query2Model(NexmarkConfiguration configuration) {
+    super(configuration);
+  }
+
+  @Override
+  public AbstractSimulator<?, ?> simulator() {
+    return new Simulator(configuration);
+  }
+
+  @Override
+  protected <T> Collection<String> toCollection(Iterator<TimestampedValue<T>> itr) {
+    return toValueTimestampOrder(itr);
+  }
+}


[19/55] [abbrv] beam git commit: Activate monitoring on NexmarkSparkRunner and on specific runners

Posted by ie...@apache.org.
Activate monitoring on NexmarkSparkRunner and on specific runners

issue #28

Fix compilation issue after rebase + make checkstyle happy again


Project: http://git-wip-us.apache.org/repos/asf/beam/repo
Commit: http://git-wip-us.apache.org/repos/asf/beam/commit/a1fe33bc
Tree: http://git-wip-us.apache.org/repos/asf/beam/tree/a1fe33bc
Diff: http://git-wip-us.apache.org/repos/asf/beam/diff/a1fe33bc

Branch: refs/heads/master
Commit: a1fe33bc122b26960697c87620ca0dc2ed522e56
Parents: a095e40
Author: Etienne Chauchot <ec...@gmail.com>
Authored: Wed Mar 15 15:25:41 2017 +0100
Committer: Ismaël Mejía <ie...@gmail.com>
Committed: Wed Aug 23 19:07:27 2017 +0200

----------------------------------------------------------------------
 integration/java/nexmark/pom.xml                | 32 ++++++++++----------
 .../integration/nexmark/NexmarkApexRunner.java  |  2 --
 .../nexmark/NexmarkDirectRunner.java            |  5 ---
 .../integration/nexmark/NexmarkFlinkRunner.java | 12 +-------
 .../nexmark/NexmarkGoogleDriver.java            |  2 --
 .../nexmark/NexmarkGoogleRunner.java            |  2 +-
 .../beam/integration/nexmark/NexmarkRunner.java | 13 ++++----
 .../integration/nexmark/NexmarkSparkDriver.java |  4 +--
 .../integration/nexmark/NexmarkSparkRunner.java | 11 +------
 .../beam/integration/nexmark/NexmarkUtils.java  |  3 +-
 .../apache/beam/integration/nexmark/Query5.java |  3 +-
 11 files changed, 31 insertions(+), 58 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/beam/blob/a1fe33bc/integration/java/nexmark/pom.xml
----------------------------------------------------------------------
diff --git a/integration/java/nexmark/pom.xml b/integration/java/nexmark/pom.xml
index 07d14c2..febd96d 100644
--- a/integration/java/nexmark/pom.xml
+++ b/integration/java/nexmark/pom.xml
@@ -179,28 +179,28 @@
       <artifactId>beam-runners-flink_2.10</artifactId>
     </dependency>
 
-    <!--<dependency>-->
-      <!--<groupId>org.apache.flink</groupId>-->
-      <!--<artifactId>flink-shaded-hadoop2</artifactId>-->
-      <!--<version>${flink.version}</version>-->
-      <!--<scope>provided</scope>-->
-    <!--</dependency>-->
+    <dependency>
+      <groupId>org.apache.flink</groupId>
+      <artifactId>flink-shaded-hadoop2</artifactId>
+      <version>${flink.version}</version>
+      <scope>provided</scope>
+    </dependency>
 
     <!-- Spark runner -->
     <dependency>
       <groupId>org.apache.beam</groupId>
       <artifactId>beam-runners-spark</artifactId>
     </dependency>
-    <!--<dependency>-->
-      <!--<groupId>org.apache.spark</groupId>-->
-      <!--<artifactId>spark-core_2.10</artifactId>-->
-      <!--<version>${spark.version}</version>-->
-    <!--</dependency>-->
-    <!--<dependency>-->
-      <!--<groupId>org.apache.spark</groupId>-->
-      <!--<artifactId>spark-streaming_2.10</artifactId>-->
-      <!--<version>${spark.version}</version>-->
-    <!--</dependency>-->
+    <dependency>
+      <groupId>org.apache.spark</groupId>
+      <artifactId>spark-core_2.10</artifactId>
+      <version>${spark.version}</version>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.spark</groupId>
+      <artifactId>spark-streaming_2.10</artifactId>
+      <version>${spark.version}</version>
+    </dependency>
 
     <!-- Apex runner -->
     <dependency>

http://git-wip-us.apache.org/repos/asf/beam/blob/a1fe33bc/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkApexRunner.java
----------------------------------------------------------------------
diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkApexRunner.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkApexRunner.java
index f2da1c7..ea46082 100644
--- a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkApexRunner.java
+++ b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkApexRunner.java
@@ -18,8 +18,6 @@
 package org.apache.beam.integration.nexmark;
 
 import javax.annotation.Nullable;
-import org.apache.beam.runners.apex.ApexRunnerResult;
-import org.apache.beam.sdk.PipelineResult;
 
 /**
  * Run a query using the Apex runner.

http://git-wip-us.apache.org/repos/asf/beam/blob/a1fe33bc/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkDirectRunner.java
----------------------------------------------------------------------
diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkDirectRunner.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkDirectRunner.java
index ee234b1..c70e41e 100644
--- a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkDirectRunner.java
+++ b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkDirectRunner.java
@@ -17,11 +17,6 @@
  */
 package org.apache.beam.integration.nexmark;
 
-import javax.annotation.Nullable;
-import org.apache.beam.runners.dataflow.DataflowPipelineJob;
-import org.apache.beam.runners.direct.DirectRunner;
-import org.apache.beam.sdk.PipelineResult;
-
 /**
  * Run a single query using the Direct Runner.
  */

http://git-wip-us.apache.org/repos/asf/beam/blob/a1fe33bc/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkFlinkRunner.java
----------------------------------------------------------------------
diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkFlinkRunner.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkFlinkRunner.java
index a8b4401..8e22917 100644
--- a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkFlinkRunner.java
+++ b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkFlinkRunner.java
@@ -17,10 +17,6 @@
  */
 package org.apache.beam.integration.nexmark;
 
-import javax.annotation.Nullable;
-import org.apache.beam.runners.flink.FlinkRunnerResult;
-import org.apache.beam.sdk.PipelineResult;
-
 /**
  * Run a query using the Flink runner.
  */
@@ -42,7 +38,7 @@ public class NexmarkFlinkRunner extends NexmarkRunner<NexmarkFlinkDriver.Nexmark
 
   @Override
   protected boolean canMonitor() {
-    return false;
+    return true;
   }
 
   @Override
@@ -56,12 +52,6 @@ public class NexmarkFlinkRunner extends NexmarkRunner<NexmarkFlinkDriver.Nexmark
     throw new UnsupportedOperationException();
   }
 
-  @Override
-  @Nullable
-  protected NexmarkPerf monitor(NexmarkQuery query) {
-    return null;
-  }
-
   public NexmarkFlinkRunner(NexmarkFlinkDriver.NexmarkFlinkOptions options) {
     super(options);
   }

http://git-wip-us.apache.org/repos/asf/beam/blob/a1fe33bc/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkGoogleDriver.java
----------------------------------------------------------------------
diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkGoogleDriver.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkGoogleDriver.java
index 67c4aeb..50c2a7c 100644
--- a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkGoogleDriver.java
+++ b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkGoogleDriver.java
@@ -17,10 +17,8 @@
  */
 package org.apache.beam.integration.nexmark;
 
-import javax.annotation.Nullable;
 import org.apache.beam.runners.dataflow.DataflowRunner;
 import org.apache.beam.runners.dataflow.options.DataflowPipelineOptions;
-import org.apache.beam.sdk.options.Description;
 import org.apache.beam.sdk.options.PipelineOptionsFactory;
 
 /**

http://git-wip-us.apache.org/repos/asf/beam/blob/a1fe33bc/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkGoogleRunner.java
----------------------------------------------------------------------
diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkGoogleRunner.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkGoogleRunner.java
index c78bb42..135d428 100644
--- a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkGoogleRunner.java
+++ b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkGoogleRunner.java
@@ -66,7 +66,7 @@ class NexmarkGoogleRunner extends NexmarkRunner<NexmarkGoogleDriver.NexmarkGoogl
 
   @Override
   protected String getJobId(PipelineResult job) {
-    return ((DataflowPipelineJob)job).getJobId();
+    return ((DataflowPipelineJob) job).getJobId();
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/beam/blob/a1fe33bc/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkRunner.java
----------------------------------------------------------------------
diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkRunner.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkRunner.java
index 5365dbe..8d4c1f1 100644
--- a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkRunner.java
+++ b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkRunner.java
@@ -53,8 +53,6 @@ import org.apache.beam.sdk.values.TupleTag;
 import org.apache.beam.sdk.values.TupleTagList;
 import org.joda.time.Duration;
 
-import static org.apache.beam.sdk.metrics.MetricUpdates.MetricUpdate;
-
 /**
  * Run a single Nexmark query using a given configuration.
  */
@@ -203,7 +201,8 @@ public abstract class NexmarkRunner<OptionT extends Options> {
    * Find a 'steady state' events/sec from {@code snapshots} and
    * store it in {@code perf} if found.
    */
-  protected void captureSteadyState(NexmarkPerf perf, List<NexmarkPerf.ProgressSnapshot> snapshots) {
+  protected void captureSteadyState(NexmarkPerf perf,
+                                    List<NexmarkPerf.ProgressSnapshot> snapshots) {
     if (!options.isStreaming()) {
       return;
     }
@@ -365,7 +364,9 @@ public abstract class NexmarkRunner<OptionT extends Options> {
     return perf;
   }
 
-  String getJobId(PipelineResult job){return "";}
+  String getJobId(PipelineResult job) {
+    return "";
+  }
 
   // TODO specific to dataflow, see if we can find an equivalent
 /*
@@ -926,8 +927,8 @@ public abstract class NexmarkRunner<OptionT extends Options> {
                                       new TableFieldSchema().setName("index").setType("INTEGER"),
                                       new TableFieldSchema().setName("value").setType("STRING")))));
     NexmarkUtils.console("Writing results to BigQuery table %s", tableSpec);
-    BigQueryIO.Write.Bound io =
-        BigQueryIO.Write.to(tableSpec)
+    BigQueryIO.Write io =
+        BigQueryIO.write().to(tableSpec)
                         .withSchema(tableSchema)
                         .withCreateDisposition(BigQueryIO.Write.CreateDisposition.CREATE_IF_NEEDED)
                         .withWriteDisposition(BigQueryIO.Write.WriteDisposition.WRITE_APPEND);

http://git-wip-us.apache.org/repos/asf/beam/blob/a1fe33bc/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkSparkDriver.java
----------------------------------------------------------------------
diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkSparkDriver.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkSparkDriver.java
index 1ea963d..a46d38a 100644
--- a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkSparkDriver.java
+++ b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkSparkDriver.java
@@ -18,7 +18,6 @@
 package org.apache.beam.integration.nexmark;
 
 import org.apache.beam.runners.spark.SparkPipelineOptions;
-import org.apache.beam.runners.spark.SparkRunner;
 import org.apache.beam.sdk.options.PipelineOptionsFactory;
 
 /**
@@ -39,7 +38,8 @@ class NexmarkSparkDriver extends NexmarkDriver<NexmarkSparkDriver.NexmarkSparkOp
                 PipelineOptionsFactory.fromArgs(args)
                         .withValidation()
                         .as(NexmarkSparkOptions.class);
-        options.setRunner(SparkRunner.class);
+//        options.setRunner(org.apache.beam.runners.spark.SparkRunner.class);
+        options.setRunner(org.apache.beam.runners.spark.SparkRunnerDebugger.class);
         NexmarkSparkRunner runner = new NexmarkSparkRunner(options);
         new NexmarkSparkDriver().runAll(options, runner);
     }

http://git-wip-us.apache.org/repos/asf/beam/blob/a1fe33bc/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkSparkRunner.java
----------------------------------------------------------------------
diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkSparkRunner.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkSparkRunner.java
index 109e8a0..32fee30 100644
--- a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkSparkRunner.java
+++ b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkSparkRunner.java
@@ -17,10 +17,6 @@
  */
 package org.apache.beam.integration.nexmark;
 
-import javax.annotation.Nullable;
-import org.apache.beam.runners.spark.SparkPipelineResult;
-import org.apache.beam.sdk.PipelineResult;
-
 /**
  * Run a query using the Spark runner.
  */
@@ -42,7 +38,7 @@ public class NexmarkSparkRunner extends NexmarkRunner<NexmarkSparkDriver.Nexmark
 
     @Override
     protected boolean canMonitor() {
-        return false;
+        return true;
     }
 
     @Override
@@ -56,11 +52,6 @@ public class NexmarkSparkRunner extends NexmarkRunner<NexmarkSparkDriver.Nexmark
         throw new UnsupportedOperationException();
     }
 
-    @Override
-    @Nullable
-    protected NexmarkPerf monitor(NexmarkQuery query) {
-        return null;
-    }
 
     public NexmarkSparkRunner(NexmarkSparkDriver.NexmarkSparkOptions options) {
         super(options);

http://git-wip-us.apache.org/repos/asf/beam/blob/a1fe33bc/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkUtils.java
----------------------------------------------------------------------
diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkUtils.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkUtils.java
index 6588f85..8f4cb22 100644
--- a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkUtils.java
+++ b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkUtils.java
@@ -37,7 +37,6 @@ import org.apache.beam.sdk.coders.CoderRegistry;
 import org.apache.beam.sdk.coders.CustomCoder;
 import org.apache.beam.sdk.coders.SerializableCoder;
 import org.apache.beam.sdk.io.Read;
-import org.apache.beam.sdk.runners.PipelineRunner;
 import org.apache.beam.sdk.transforms.Aggregator;
 import org.apache.beam.sdk.transforms.Combine;
 import org.apache.beam.sdk.transforms.DoFn;
@@ -325,8 +324,8 @@ public class NexmarkUtils {
    * Setup pipeline with codes and some other options.
    */
   public static void setupPipeline(CoderStrategy coderStrategy, Pipeline p) {
-    PipelineRunner<?> runner = p.getRunner();
     //TODO Ismael check
+//    PipelineRunner<?> runner = p.getRunner();
 //    if (runner instanceof DirectRunner) {
 //      // Disable randomization of output since we want to check batch and streaming match the
 //      // model both locally and on the cloud.

http://git-wip-us.apache.org/repos/asf/beam/blob/a1fe33bc/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/Query5.java
----------------------------------------------------------------------
diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/Query5.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/Query5.java
index 7001986..9020494 100644
--- a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/Query5.java
+++ b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/Query5.java
@@ -67,7 +67,8 @@ class Query5 extends NexmarkQuery {
         // Count the number of bids per auction id.
         .apply(Count.<Long>perElement())
 
-        // We'll want to keep all auctions with the maximal number of bids.
+      //TODO replace by simple key
+      // We'll want to keep all auctions with the maximal number of bids.
         // Start by lifting each into a singleton list.
         .apply(name + ".ToSingletons",
             ParDo.of(new DoFn<KV<Long, Long>, KV<List<Long>, Long>>() {


[30/55] [abbrv] beam git commit: Fix compile after Coders and Pubsub refactor

Posted by ie...@apache.org.
Fix compile after Coders and Pubsub refactor


Project: http://git-wip-us.apache.org/repos/asf/beam/repo
Commit: http://git-wip-us.apache.org/repos/asf/beam/commit/8b96949b
Tree: http://git-wip-us.apache.org/repos/asf/beam/tree/8b96949b
Diff: http://git-wip-us.apache.org/repos/asf/beam/diff/8b96949b

Branch: refs/heads/master
Commit: 8b96949b934be1df7102aeb24ef4b23d9dd28812
Parents: b438fa7
Author: Etienne Chauchot <ec...@gmail.com>
Authored: Fri Apr 28 10:29:38 2017 +0200
Committer: Ismaël Mejía <ie...@gmail.com>
Committed: Wed Aug 23 19:07:28 2017 +0200

----------------------------------------------------------------------
 .../integration/nexmark/NexmarkOptions.java     |  3 +--
 .../beam/integration/nexmark/WinningBids.java   | 23 +++++++++++---------
 .../integration/nexmark/io/PubsubClient.java    |  2 +-
 .../integration/nexmark/io/PubsubHelper.java    |  2 +-
 .../nexmark/io/PubsubJsonClient.java            |  2 +-
 .../nexmark/io/PubsubTestClient.java            |  2 +-
 .../beam/integration/nexmark/model/Auction.java |  6 ++---
 .../integration/nexmark/model/AuctionBid.java   |  6 ++---
 .../integration/nexmark/model/AuctionCount.java |  6 ++---
 .../integration/nexmark/model/AuctionPrice.java |  6 ++---
 .../beam/integration/nexmark/model/Bid.java     |  8 +++----
 .../nexmark/model/BidsPerSession.java           |  7 +++---
 .../nexmark/model/CategoryPrice.java            |  7 +++---
 .../beam/integration/nexmark/model/Done.java    |  7 +++---
 .../beam/integration/nexmark/model/Event.java   |  6 ++---
 .../nexmark/model/IdNameReserve.java            |  7 +++---
 .../nexmark/model/NameCityStateId.java          |  7 +++---
 .../beam/integration/nexmark/model/Person.java  |  7 +++---
 .../integration/nexmark/model/SellerPrice.java  |  7 +++---
 .../integration/nexmark/sources/Generator.java  |  6 ++---
 20 files changed, 57 insertions(+), 70 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/beam/blob/8b96949b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkOptions.java
----------------------------------------------------------------------
diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkOptions.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkOptions.java
index 5d093ae..e1c1af2 100644
--- a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkOptions.java
+++ b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkOptions.java
@@ -18,10 +18,9 @@
 package org.apache.beam.integration.nexmark;
 
 import javax.annotation.Nullable;
-
+import org.apache.beam.sdk.io.gcp.pubsub.PubsubOptions;
 import org.apache.beam.sdk.options.Default;
 import org.apache.beam.sdk.options.Description;
-import org.apache.beam.sdk.options.PubsubOptions;
 
 /**
  * Command line flags.

http://git-wip-us.apache.org/repos/asf/beam/blob/8b96949b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/WinningBids.java
----------------------------------------------------------------------
diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/WinningBids.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/WinningBids.java
index f2566b8..3815b9d 100644
--- a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/WinningBids.java
+++ b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/WinningBids.java
@@ -35,9 +35,9 @@ import org.apache.beam.integration.nexmark.model.AuctionBid;
 import org.apache.beam.integration.nexmark.model.Bid;
 import org.apache.beam.integration.nexmark.model.Event;
 import org.apache.beam.integration.nexmark.sources.GeneratorConfig;
-import org.apache.beam.sdk.coders.AtomicCoder;
 import org.apache.beam.sdk.coders.Coder;
 import org.apache.beam.sdk.coders.CoderException;
+import org.apache.beam.sdk.coders.CustomCoder;
 import org.apache.beam.sdk.coders.VarIntCoder;
 import org.apache.beam.sdk.coders.VarLongCoder;
 import org.apache.beam.sdk.metrics.Counter;
@@ -145,7 +145,7 @@ public class WinningBids extends PTransform<PCollection<Event>, PCollection<Auct
   /**
    * Encodes an {@link AuctionOrBidWindow} as an {@link IntervalWindow} and an auction id long.
    */
-  private static class AuctionOrBidWindowCoder extends AtomicCoder<AuctionOrBidWindow> {
+  private static class AuctionOrBidWindowCoder extends CustomCoder<AuctionOrBidWindow> {
     private static final AuctionOrBidWindowCoder INSTANCE = new AuctionOrBidWindowCoder();
     private static final Coder<IntervalWindow> SUPER_CODER = IntervalWindow.getCoder();
     private static final Coder<Long> ID_CODER = VarLongCoder.of();
@@ -157,22 +157,25 @@ public class WinningBids extends PTransform<PCollection<Event>, PCollection<Auct
     }
 
     @Override
-    public void encode(AuctionOrBidWindow window, OutputStream outStream, Context context)
+    public void encode(AuctionOrBidWindow window, OutputStream outStream, Coder.Context context)
         throws IOException, CoderException {
-      SUPER_CODER.encode(window, outStream, Context.NESTED);
-      ID_CODER.encode(window.auction, outStream, Context.NESTED);
-      INT_CODER.encode(window.isAuctionWindow ? 1 : 0, outStream, Context.NESTED);
+      SUPER_CODER.encode(window, outStream, Coder.Context.NESTED);
+      ID_CODER.encode(window.auction, outStream, Coder.Context.NESTED);
+      INT_CODER.encode(window.isAuctionWindow ? 1 : 0, outStream, Coder.Context.NESTED);
     }
 
     @Override
-    public AuctionOrBidWindow decode(InputStream inStream, Context context)
+    public AuctionOrBidWindow decode(InputStream inStream, Coder.Context context)
         throws IOException, CoderException {
-      IntervalWindow superWindow = SUPER_CODER.decode(inStream, Context.NESTED);
-      long auction = ID_CODER.decode(inStream, Context.NESTED);
-      boolean isAuctionWindow = INT_CODER.decode(inStream, Context.NESTED) == 0 ? false : true;
+      IntervalWindow superWindow = SUPER_CODER.decode(inStream, Coder.Context.NESTED);
+      long auction = ID_CODER.decode(inStream, Coder.Context.NESTED);
+      boolean isAuctionWindow =
+          INT_CODER.decode(inStream, Coder.Context.NESTED) == 0 ? false : true;
       return new AuctionOrBidWindow(
           superWindow.start(), superWindow.end(), auction, isAuctionWindow);
     }
+
+    @Override public void verifyDeterministic() throws NonDeterministicException {}
   }
 
   /** Assign events to auction windows and merges them intelligently. */

http://git-wip-us.apache.org/repos/asf/beam/blob/8b96949b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/io/PubsubClient.java
----------------------------------------------------------------------
diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/io/PubsubClient.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/io/PubsubClient.java
index 687aa35..931fe6e 100644
--- a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/io/PubsubClient.java
+++ b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/io/PubsubClient.java
@@ -32,7 +32,7 @@ import java.util.List;
 import java.util.Map;
 import java.util.concurrent.ThreadLocalRandom;
 import javax.annotation.Nullable;
-import org.apache.beam.sdk.options.PubsubOptions;
+import org.apache.beam.sdk.io.gcp.pubsub.PubsubOptions;
 
 /**
  * An (abstract) helper class for talking to Pubsub via an underlying transport.

http://git-wip-us.apache.org/repos/asf/beam/blob/8b96949b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/io/PubsubHelper.java
----------------------------------------------------------------------
diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/io/PubsubHelper.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/io/PubsubHelper.java
index 15401b7..bcc5b1c 100644
--- a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/io/PubsubHelper.java
+++ b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/io/PubsubHelper.java
@@ -23,7 +23,7 @@ import java.util.Collection;
 import java.util.List;
 
 import org.apache.beam.integration.nexmark.NexmarkUtils;
-import org.apache.beam.sdk.options.PubsubOptions;
+import org.apache.beam.sdk.io.gcp.pubsub.PubsubOptions;
 
 /**
  * Helper for working with pubsub.

http://git-wip-us.apache.org/repos/asf/beam/blob/8b96949b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/io/PubsubJsonClient.java
----------------------------------------------------------------------
diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/io/PubsubJsonClient.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/io/PubsubJsonClient.java
index b778a09..afddbd8 100644
--- a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/io/PubsubJsonClient.java
+++ b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/io/PubsubJsonClient.java
@@ -47,7 +47,7 @@ import java.util.List;
 import java.util.Map;
 import java.util.TreeMap;
 import javax.annotation.Nullable;
-import org.apache.beam.sdk.options.PubsubOptions;
+import org.apache.beam.sdk.io.gcp.pubsub.PubsubOptions;
 import org.apache.beam.sdk.util.RetryHttpRequestInitializer;
 import org.apache.beam.sdk.util.Transport;
 

http://git-wip-us.apache.org/repos/asf/beam/blob/8b96949b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/io/PubsubTestClient.java
----------------------------------------------------------------------
diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/io/PubsubTestClient.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/io/PubsubTestClient.java
index 125a8d6..69ba2b0 100644
--- a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/io/PubsubTestClient.java
+++ b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/io/PubsubTestClient.java
@@ -33,7 +33,7 @@ import java.util.List;
 import java.util.Map;
 import java.util.Set;
 import javax.annotation.Nullable;
-import org.apache.beam.sdk.options.PubsubOptions;
+import org.apache.beam.sdk.io.gcp.pubsub.PubsubOptions;
 
 /**
  * A (partial) implementation of {@link PubsubClient} for use by unit tests. Only suitable for

http://git-wip-us.apache.org/repos/asf/beam/blob/8b96949b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/model/Auction.java
----------------------------------------------------------------------
diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/model/Auction.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/model/Auction.java
index ac30568..4b1a848 100644
--- a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/model/Auction.java
+++ b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/model/Auction.java
@@ -19,16 +19,14 @@ package org.apache.beam.integration.nexmark.model;
 
 import com.fasterxml.jackson.annotation.JsonProperty;
 import com.fasterxml.jackson.core.JsonProcessingException;
-
 import java.io.IOException;
 import java.io.InputStream;
 import java.io.OutputStream;
 import java.io.Serializable;
-
 import org.apache.beam.integration.nexmark.NexmarkUtils;
-import org.apache.beam.sdk.coders.AtomicCoder;
 import org.apache.beam.sdk.coders.Coder;
 import org.apache.beam.sdk.coders.CoderException;
+import org.apache.beam.sdk.coders.CustomCoder;
 import org.apache.beam.sdk.coders.StringUtf8Coder;
 import org.apache.beam.sdk.coders.VarLongCoder;
 
@@ -39,7 +37,7 @@ public class Auction implements KnownSize, Serializable {
   private static final Coder<Long> LONG_CODER = VarLongCoder.of();
   private static final Coder<String> STRING_CODER = StringUtf8Coder.of();
 
-  public static final Coder<Auction> CODER = new AtomicCoder<Auction>() {
+  public static final Coder<Auction> CODER = new CustomCoder<Auction>() {
     @Override
     public void encode(Auction value, OutputStream outStream,
         Coder.Context context)

http://git-wip-us.apache.org/repos/asf/beam/blob/8b96949b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/model/AuctionBid.java
----------------------------------------------------------------------
diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/model/AuctionBid.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/model/AuctionBid.java
index c014257..7f6b7c9 100644
--- a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/model/AuctionBid.java
+++ b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/model/AuctionBid.java
@@ -19,23 +19,21 @@ package org.apache.beam.integration.nexmark.model;
 
 import com.fasterxml.jackson.annotation.JsonProperty;
 import com.fasterxml.jackson.core.JsonProcessingException;
-
 import java.io.IOException;
 import java.io.InputStream;
 import java.io.OutputStream;
 import java.io.Serializable;
-
 import org.apache.beam.integration.nexmark.NexmarkUtils;
 import org.apache.beam.integration.nexmark.WinningBids;
-import org.apache.beam.sdk.coders.AtomicCoder;
 import org.apache.beam.sdk.coders.Coder;
 import org.apache.beam.sdk.coders.CoderException;
+import org.apache.beam.sdk.coders.CustomCoder;
 
 /**
  * Result of {@link WinningBids} transform.
  */
 public class AuctionBid implements KnownSize, Serializable {
-  public static final Coder<AuctionBid> CODER = new AtomicCoder<AuctionBid>() {
+  public static final Coder<AuctionBid> CODER = new CustomCoder<AuctionBid>() {
     @Override
     public void encode(AuctionBid value, OutputStream outStream,
         Coder.Context context)

http://git-wip-us.apache.org/repos/asf/beam/blob/8b96949b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/model/AuctionCount.java
----------------------------------------------------------------------
diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/model/AuctionCount.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/model/AuctionCount.java
index aa16629..e6d3450 100644
--- a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/model/AuctionCount.java
+++ b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/model/AuctionCount.java
@@ -19,16 +19,14 @@ package org.apache.beam.integration.nexmark.model;
 
 import com.fasterxml.jackson.annotation.JsonProperty;
 import com.fasterxml.jackson.core.JsonProcessingException;
-
 import java.io.IOException;
 import java.io.InputStream;
 import java.io.OutputStream;
 import java.io.Serializable;
-
 import org.apache.beam.integration.nexmark.NexmarkUtils;
-import org.apache.beam.sdk.coders.AtomicCoder;
 import org.apache.beam.sdk.coders.Coder;
 import org.apache.beam.sdk.coders.CoderException;
+import org.apache.beam.sdk.coders.CustomCoder;
 import org.apache.beam.sdk.coders.VarLongCoder;
 
 /**
@@ -37,7 +35,7 @@ import org.apache.beam.sdk.coders.VarLongCoder;
 public class AuctionCount implements KnownSize, Serializable {
   private static final Coder<Long> LONG_CODER = VarLongCoder.of();
 
-  public static final Coder<AuctionCount> CODER = new AtomicCoder<AuctionCount>() {
+  public static final Coder<AuctionCount> CODER = new CustomCoder<AuctionCount>() {
     @Override
     public void encode(AuctionCount value, OutputStream outStream,
         Coder.Context context)

http://git-wip-us.apache.org/repos/asf/beam/blob/8b96949b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/model/AuctionPrice.java
----------------------------------------------------------------------
diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/model/AuctionPrice.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/model/AuctionPrice.java
index f365cc8..cb971e2 100644
--- a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/model/AuctionPrice.java
+++ b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/model/AuctionPrice.java
@@ -19,16 +19,14 @@ package org.apache.beam.integration.nexmark.model;
 
 import com.fasterxml.jackson.annotation.JsonProperty;
 import com.fasterxml.jackson.core.JsonProcessingException;
-
 import java.io.IOException;
 import java.io.InputStream;
 import java.io.OutputStream;
 import java.io.Serializable;
-
 import org.apache.beam.integration.nexmark.NexmarkUtils;
-import org.apache.beam.sdk.coders.AtomicCoder;
 import org.apache.beam.sdk.coders.Coder;
 import org.apache.beam.sdk.coders.CoderException;
+import org.apache.beam.sdk.coders.CustomCoder;
 import org.apache.beam.sdk.coders.VarLongCoder;
 
 /**
@@ -37,7 +35,7 @@ import org.apache.beam.sdk.coders.VarLongCoder;
 public class AuctionPrice implements KnownSize, Serializable {
   private static final Coder<Long> LONG_CODER = VarLongCoder.of();
 
-  public static final Coder<AuctionPrice> CODER = new AtomicCoder<AuctionPrice>() {
+  public static final Coder<AuctionPrice> CODER = new CustomCoder<AuctionPrice>() {
     @Override
     public void encode(AuctionPrice value, OutputStream outStream,
         Coder.Context context)

http://git-wip-us.apache.org/repos/asf/beam/blob/8b96949b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/model/Bid.java
----------------------------------------------------------------------
diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/model/Bid.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/model/Bid.java
index 59a33c1..faeb928 100644
--- a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/model/Bid.java
+++ b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/model/Bid.java
@@ -19,17 +19,15 @@ package org.apache.beam.integration.nexmark.model;
 
 import com.fasterxml.jackson.annotation.JsonProperty;
 import com.fasterxml.jackson.core.JsonProcessingException;
-
 import java.io.IOException;
 import java.io.InputStream;
 import java.io.OutputStream;
 import java.io.Serializable;
 import java.util.Comparator;
-
 import org.apache.beam.integration.nexmark.NexmarkUtils;
-import org.apache.beam.sdk.coders.AtomicCoder;
 import org.apache.beam.sdk.coders.Coder;
 import org.apache.beam.sdk.coders.CoderException;
+import org.apache.beam.sdk.coders.CustomCoder;
 import org.apache.beam.sdk.coders.StringUtf8Coder;
 import org.apache.beam.sdk.coders.VarLongCoder;
 
@@ -40,7 +38,7 @@ public class Bid implements KnownSize, Serializable {
   private static final Coder<Long> LONG_CODER = VarLongCoder.of();
   private static final Coder<String> STRING_CODER = StringUtf8Coder.of();
 
-  public static final Coder<Bid> CODER = new AtomicCoder<Bid>() {
+  public static final Coder<Bid> CODER = new CustomCoder<Bid>() {
     @Override
     public void encode(Bid value, OutputStream outStream,
         Coder.Context context)
@@ -63,6 +61,8 @@ public class Bid implements KnownSize, Serializable {
       String extra = STRING_CODER.decode(inStream, Context.NESTED);
       return new Bid(auction, bidder, price, dateTime, extra);
     }
+
+    @Override public void verifyDeterministic() throws NonDeterministicException {}
   };
 
   /**

http://git-wip-us.apache.org/repos/asf/beam/blob/8b96949b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/model/BidsPerSession.java
----------------------------------------------------------------------
diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/model/BidsPerSession.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/model/BidsPerSession.java
index 7c4dfae..26b6a41 100644
--- a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/model/BidsPerSession.java
+++ b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/model/BidsPerSession.java
@@ -19,16 +19,14 @@ package org.apache.beam.integration.nexmark.model;
 
 import com.fasterxml.jackson.annotation.JsonProperty;
 import com.fasterxml.jackson.core.JsonProcessingException;
-
 import java.io.IOException;
 import java.io.InputStream;
 import java.io.OutputStream;
 import java.io.Serializable;
-
 import org.apache.beam.integration.nexmark.NexmarkUtils;
-import org.apache.beam.sdk.coders.AtomicCoder;
 import org.apache.beam.sdk.coders.Coder;
 import org.apache.beam.sdk.coders.CoderException;
+import org.apache.beam.sdk.coders.CustomCoder;
 import org.apache.beam.sdk.coders.VarLongCoder;
 
 /**
@@ -37,7 +35,7 @@ import org.apache.beam.sdk.coders.VarLongCoder;
 public class BidsPerSession implements KnownSize, Serializable {
   private static final Coder<Long> LONG_CODER = VarLongCoder.of();
 
-  public static final Coder<BidsPerSession> CODER = new AtomicCoder<BidsPerSession>() {
+  public static final Coder<BidsPerSession> CODER = new CustomCoder<BidsPerSession>() {
     @Override
     public void encode(BidsPerSession value, OutputStream outStream,
         Coder.Context context)
@@ -54,6 +52,7 @@ public class BidsPerSession implements KnownSize, Serializable {
       long bidsPerSession = LONG_CODER.decode(inStream, Context.NESTED);
       return new BidsPerSession(personId, bidsPerSession);
     }
+    @Override public void verifyDeterministic() throws NonDeterministicException {}
   };
 
   @JsonProperty

http://git-wip-us.apache.org/repos/asf/beam/blob/8b96949b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/model/CategoryPrice.java
----------------------------------------------------------------------
diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/model/CategoryPrice.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/model/CategoryPrice.java
index 6512cc1..ccb2bc7 100644
--- a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/model/CategoryPrice.java
+++ b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/model/CategoryPrice.java
@@ -19,16 +19,14 @@ package org.apache.beam.integration.nexmark.model;
 
 import com.fasterxml.jackson.annotation.JsonProperty;
 import com.fasterxml.jackson.core.JsonProcessingException;
-
 import java.io.IOException;
 import java.io.InputStream;
 import java.io.OutputStream;
 import java.io.Serializable;
-
 import org.apache.beam.integration.nexmark.NexmarkUtils;
-import org.apache.beam.sdk.coders.AtomicCoder;
 import org.apache.beam.sdk.coders.Coder;
 import org.apache.beam.sdk.coders.CoderException;
+import org.apache.beam.sdk.coders.CustomCoder;
 import org.apache.beam.sdk.coders.VarIntCoder;
 import org.apache.beam.sdk.coders.VarLongCoder;
 
@@ -39,7 +37,7 @@ public class CategoryPrice implements KnownSize, Serializable {
   private static final Coder<Long> LONG_CODER = VarLongCoder.of();
   private static final Coder<Integer> INT_CODER = VarIntCoder.of();
 
-  public static final Coder<CategoryPrice> CODER = new AtomicCoder<CategoryPrice>() {
+  public static final Coder<CategoryPrice> CODER = new CustomCoder<CategoryPrice>() {
     @Override
     public void encode(CategoryPrice value, OutputStream outStream,
         Coder.Context context)
@@ -58,6 +56,7 @@ public class CategoryPrice implements KnownSize, Serializable {
       boolean isLast = INT_CODER.decode(inStream, context) != 0;
       return new CategoryPrice(category, price, isLast);
     }
+    @Override public void verifyDeterministic() throws NonDeterministicException {}
   };
 
   @JsonProperty

http://git-wip-us.apache.org/repos/asf/beam/blob/8b96949b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/model/Done.java
----------------------------------------------------------------------
diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/model/Done.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/model/Done.java
index 6009463..42999cd 100644
--- a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/model/Done.java
+++ b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/model/Done.java
@@ -19,16 +19,14 @@ package org.apache.beam.integration.nexmark.model;
 
 import com.fasterxml.jackson.annotation.JsonProperty;
 import com.fasterxml.jackson.core.JsonProcessingException;
-
 import java.io.IOException;
 import java.io.InputStream;
 import java.io.OutputStream;
 import java.io.Serializable;
-
 import org.apache.beam.integration.nexmark.NexmarkUtils;
-import org.apache.beam.sdk.coders.AtomicCoder;
 import org.apache.beam.sdk.coders.Coder;
 import org.apache.beam.sdk.coders.CoderException;
+import org.apache.beam.sdk.coders.CustomCoder;
 import org.apache.beam.sdk.coders.StringUtf8Coder;
 
 /**
@@ -37,7 +35,7 @@ import org.apache.beam.sdk.coders.StringUtf8Coder;
 public class Done implements KnownSize, Serializable {
   private static final Coder<String> STRING_CODER = StringUtf8Coder.of();
 
-  public static final Coder<Done> CODER = new AtomicCoder<Done>() {
+  public static final Coder<Done> CODER = new CustomCoder<Done>() {
     @Override
     public void encode(Done value, OutputStream outStream,
         Coder.Context context)
@@ -52,6 +50,7 @@ public class Done implements KnownSize, Serializable {
       String message = STRING_CODER.decode(inStream, Context.NESTED);
       return new Done(message);
     }
+    @Override public void verifyDeterministic() throws NonDeterministicException {}
   };
 
   @JsonProperty

http://git-wip-us.apache.org/repos/asf/beam/blob/8b96949b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/model/Event.java
----------------------------------------------------------------------
diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/model/Event.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/model/Event.java
index 8a278bf..e2130c9 100644
--- a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/model/Event.java
+++ b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/model/Event.java
@@ -22,10 +22,9 @@ import java.io.InputStream;
 import java.io.OutputStream;
 import java.io.Serializable;
 import javax.annotation.Nullable;
-
-import org.apache.beam.sdk.coders.AtomicCoder;
 import org.apache.beam.sdk.coders.Coder;
 import org.apache.beam.sdk.coders.CoderException;
+import org.apache.beam.sdk.coders.CustomCoder;
 import org.apache.beam.sdk.coders.VarIntCoder;
 
 /**
@@ -35,7 +34,7 @@ import org.apache.beam.sdk.coders.VarIntCoder;
 public class Event implements KnownSize, Serializable {
   private static final Coder<Integer> INT_CODER = VarIntCoder.of();
 
-  public static final Coder<Event> CODER = new AtomicCoder<Event>() {
+  public static final Coder<Event> CODER = new CustomCoder<Event>() {
     @Override
     public void encode(Event value, OutputStream outStream, Coder.Context context)
         throws CoderException, IOException {
@@ -71,6 +70,7 @@ public class Event implements KnownSize, Serializable {
         throw new RuntimeException("invalid event encoding");
       }
     }
+    @Override public void verifyDeterministic() throws NonDeterministicException {}
   };
 
   @Nullable

http://git-wip-us.apache.org/repos/asf/beam/blob/8b96949b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/model/IdNameReserve.java
----------------------------------------------------------------------
diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/model/IdNameReserve.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/model/IdNameReserve.java
index 5d22651..cf1e571 100644
--- a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/model/IdNameReserve.java
+++ b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/model/IdNameReserve.java
@@ -19,16 +19,14 @@ package org.apache.beam.integration.nexmark.model;
 
 import com.fasterxml.jackson.annotation.JsonProperty;
 import com.fasterxml.jackson.core.JsonProcessingException;
-
 import java.io.IOException;
 import java.io.InputStream;
 import java.io.OutputStream;
 import java.io.Serializable;
-
 import org.apache.beam.integration.nexmark.NexmarkUtils;
-import org.apache.beam.sdk.coders.AtomicCoder;
 import org.apache.beam.sdk.coders.Coder;
 import org.apache.beam.sdk.coders.CoderException;
+import org.apache.beam.sdk.coders.CustomCoder;
 import org.apache.beam.sdk.coders.StringUtf8Coder;
 import org.apache.beam.sdk.coders.VarLongCoder;
 
@@ -39,7 +37,7 @@ public class IdNameReserve implements KnownSize, Serializable {
   private static final Coder<Long> LONG_CODER = VarLongCoder.of();
   private static final Coder<String> STRING_CODER = StringUtf8Coder.of();
 
-  public static final Coder<IdNameReserve> CODER = new AtomicCoder<IdNameReserve>() {
+  public static final Coder<IdNameReserve> CODER = new CustomCoder<IdNameReserve>() {
     @Override
     public void encode(IdNameReserve value, OutputStream outStream,
         Coder.Context context)
@@ -58,6 +56,7 @@ public class IdNameReserve implements KnownSize, Serializable {
       long reserve = LONG_CODER.decode(inStream, Context.NESTED);
       return new IdNameReserve(id, name, reserve);
     }
+    @Override public void verifyDeterministic() throws NonDeterministicException {}
   };
 
   @JsonProperty

http://git-wip-us.apache.org/repos/asf/beam/blob/8b96949b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/model/NameCityStateId.java
----------------------------------------------------------------------
diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/model/NameCityStateId.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/model/NameCityStateId.java
index ac22879..86d1738 100644
--- a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/model/NameCityStateId.java
+++ b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/model/NameCityStateId.java
@@ -19,16 +19,14 @@ package org.apache.beam.integration.nexmark.model;
 
 import com.fasterxml.jackson.annotation.JsonProperty;
 import com.fasterxml.jackson.core.JsonProcessingException;
-
 import java.io.IOException;
 import java.io.InputStream;
 import java.io.OutputStream;
 import java.io.Serializable;
-
 import org.apache.beam.integration.nexmark.NexmarkUtils;
-import org.apache.beam.sdk.coders.AtomicCoder;
 import org.apache.beam.sdk.coders.Coder;
 import org.apache.beam.sdk.coders.CoderException;
+import org.apache.beam.sdk.coders.CustomCoder;
 import org.apache.beam.sdk.coders.StringUtf8Coder;
 import org.apache.beam.sdk.coders.VarLongCoder;
 
@@ -39,7 +37,7 @@ public class NameCityStateId implements KnownSize, Serializable {
   private static final Coder<Long> LONG_CODER = VarLongCoder.of();
   private static final Coder<String> STRING_CODER = StringUtf8Coder.of();
 
-  public static final Coder<NameCityStateId> CODER = new AtomicCoder<NameCityStateId>() {
+  public static final Coder<NameCityStateId> CODER = new CustomCoder<NameCityStateId>() {
     @Override
     public void encode(NameCityStateId value, OutputStream outStream,
         Coder.Context context)
@@ -60,6 +58,7 @@ public class NameCityStateId implements KnownSize, Serializable {
       long id = LONG_CODER.decode(inStream, Context.NESTED);
       return new NameCityStateId(name, city, state, id);
     }
+    @Override public void verifyDeterministic() throws NonDeterministicException {}
   };
 
   @JsonProperty

http://git-wip-us.apache.org/repos/asf/beam/blob/8b96949b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/model/Person.java
----------------------------------------------------------------------
diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/model/Person.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/model/Person.java
index 85c7183..906df94 100644
--- a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/model/Person.java
+++ b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/model/Person.java
@@ -19,16 +19,14 @@ package org.apache.beam.integration.nexmark.model;
 
 import com.fasterxml.jackson.annotation.JsonProperty;
 import com.fasterxml.jackson.core.JsonProcessingException;
-
 import java.io.IOException;
 import java.io.InputStream;
 import java.io.OutputStream;
 import java.io.Serializable;
-
 import org.apache.beam.integration.nexmark.NexmarkUtils;
-import org.apache.beam.sdk.coders.AtomicCoder;
 import org.apache.beam.sdk.coders.Coder;
 import org.apache.beam.sdk.coders.CoderException;
+import org.apache.beam.sdk.coders.CustomCoder;
 import org.apache.beam.sdk.coders.StringUtf8Coder;
 import org.apache.beam.sdk.coders.VarLongCoder;
 
@@ -38,7 +36,7 @@ import org.apache.beam.sdk.coders.VarLongCoder;
 public class Person implements KnownSize, Serializable {
   private static final Coder<Long> LONG_CODER = VarLongCoder.of();
   private static final Coder<String> STRING_CODER = StringUtf8Coder.of();
-  public static final Coder<Person> CODER = new AtomicCoder<Person>() {
+  public static final Coder<Person> CODER = new CustomCoder<Person>() {
     @Override
     public void encode(Person value, OutputStream outStream,
         Coder.Context context)
@@ -67,6 +65,7 @@ public class Person implements KnownSize, Serializable {
       String extra = STRING_CODER.decode(inStream, Context.NESTED);
       return new Person(id, name, emailAddress, creditCard, city, state, dateTime, extra);
     }
+    @Override public void verifyDeterministic() throws NonDeterministicException {}
   };
 
   /** Id of person. */

http://git-wip-us.apache.org/repos/asf/beam/blob/8b96949b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/model/SellerPrice.java
----------------------------------------------------------------------
diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/model/SellerPrice.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/model/SellerPrice.java
index b7c2b14..68f2697 100644
--- a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/model/SellerPrice.java
+++ b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/model/SellerPrice.java
@@ -19,16 +19,14 @@ package org.apache.beam.integration.nexmark.model;
 
 import com.fasterxml.jackson.annotation.JsonProperty;
 import com.fasterxml.jackson.core.JsonProcessingException;
-
 import java.io.IOException;
 import java.io.InputStream;
 import java.io.OutputStream;
 import java.io.Serializable;
-
 import org.apache.beam.integration.nexmark.NexmarkUtils;
-import org.apache.beam.sdk.coders.AtomicCoder;
 import org.apache.beam.sdk.coders.Coder;
 import org.apache.beam.sdk.coders.CoderException;
+import org.apache.beam.sdk.coders.CustomCoder;
 import org.apache.beam.sdk.coders.VarLongCoder;
 
 /**
@@ -37,7 +35,7 @@ import org.apache.beam.sdk.coders.VarLongCoder;
 public class SellerPrice implements KnownSize, Serializable {
   private static final Coder<Long> LONG_CODER = VarLongCoder.of();
 
-  public static final Coder<SellerPrice> CODER = new AtomicCoder<SellerPrice>() {
+  public static final Coder<SellerPrice> CODER = new CustomCoder<SellerPrice>() {
     @Override
     public void encode(SellerPrice value, OutputStream outStream,
         Coder.Context context)
@@ -54,6 +52,7 @@ public class SellerPrice implements KnownSize, Serializable {
       long price = LONG_CODER.decode(inStream, Context.NESTED);
       return new SellerPrice(seller, price);
     }
+    @Override public void verifyDeterministic() throws NonDeterministicException {}
   };
 
   @JsonProperty

http://git-wip-us.apache.org/repos/asf/beam/blob/8b96949b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/sources/Generator.java
----------------------------------------------------------------------
diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/sources/Generator.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/sources/Generator.java
index cffc7a5..012d4e6 100644
--- a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/sources/Generator.java
+++ b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/sources/Generator.java
@@ -27,14 +27,13 @@ import java.util.Arrays;
 import java.util.Iterator;
 import java.util.List;
 import java.util.Random;
-
 import org.apache.beam.integration.nexmark.model.Auction;
 import org.apache.beam.integration.nexmark.model.Bid;
 import org.apache.beam.integration.nexmark.model.Event;
 import org.apache.beam.integration.nexmark.model.Person;
-import org.apache.beam.sdk.coders.AtomicCoder;
 import org.apache.beam.sdk.coders.Coder;
 import org.apache.beam.sdk.coders.CoderException;
+import org.apache.beam.sdk.coders.CustomCoder;
 import org.apache.beam.sdk.coders.VarLongCoder;
 import org.apache.beam.sdk.io.UnboundedSource;
 import org.apache.beam.sdk.values.TimestampedValue;
@@ -102,7 +101,7 @@ public class Generator implements Iterator<TimestampedValue<Event>>, Serializabl
 
     /** Coder for this class. */
     public static final Coder<Checkpoint> CODER_INSTANCE =
-        new AtomicCoder<Checkpoint>() {
+        new CustomCoder<Checkpoint>() {
           @Override
           public void encode(
               Checkpoint value,
@@ -121,6 +120,7 @@ public class Generator implements Iterator<TimestampedValue<Event>>, Serializabl
             long wallclockBaseTime = LONG_CODER.decode(inStream, Context.NESTED);
             return new Checkpoint(numEvents, wallclockBaseTime);
           }
+          @Override public void verifyDeterministic() throws NonDeterministicException {}
         };
 
     private long numEvents;


[22/55] [abbrv] beam git commit: Fix compile after ParDo refactor

Posted by ie...@apache.org.
Fix compile after ParDo refactor


Project: http://git-wip-us.apache.org/repos/asf/beam/repo
Commit: http://git-wip-us.apache.org/repos/asf/beam/commit/bd93c8b5
Tree: http://git-wip-us.apache.org/repos/asf/beam/tree/bd93c8b5
Diff: http://git-wip-us.apache.org/repos/asf/beam/diff/bd93c8b5

Branch: refs/heads/master
Commit: bd93c8b55ba6f81c87b74364b26d64e0f8c1103f
Parents: 7bfc982
Author: Ismaël Mejía <ie...@apache.org>
Authored: Wed Mar 29 10:10:13 2017 +0200
Committer: Ismaël Mejía <ie...@gmail.com>
Committed: Wed Aug 23 19:07:27 2017 +0200

----------------------------------------------------------------------
 .../beam/integration/nexmark/NexmarkQuery.java  | 14 ++++++-------
 .../beam/integration/nexmark/NexmarkRunner.java |  3 +--
 .../beam/integration/nexmark/NexmarkUtils.java  | 16 +++++++-------
 .../integration/nexmark/queries/Query7.java     | 22 ++++++++++----------
 4 files changed, 27 insertions(+), 28 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/beam/blob/bd93c8b5/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkQuery.java
----------------------------------------------------------------------
diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkQuery.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkQuery.java
index c268a3b..e1cd493 100644
--- a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkQuery.java
+++ b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkQuery.java
@@ -97,7 +97,7 @@ public abstract class NexmarkQuery
   };
 
   /** Transform to key each person by their id. */
-  protected static final ParDo.Bound<Person, KV<Long, Person>> PERSON_BY_ID =
+  protected static final ParDo.SingleOutput<Person, KV<Long, Person>> PERSON_BY_ID =
       ParDo.of(new DoFn<Person, KV<Long, Person>>() {
              @ProcessElement
              public void processElement(ProcessContext c) {
@@ -106,7 +106,7 @@ public abstract class NexmarkQuery
            });
 
   /** Transform to key each auction by its id. */
-  protected static final ParDo.Bound<Auction, KV<Long, Auction>> AUCTION_BY_ID =
+  protected static final ParDo.SingleOutput<Auction, KV<Long, Auction>> AUCTION_BY_ID =
       ParDo.of(new DoFn<Auction, KV<Long, Auction>>() {
              @ProcessElement
              public void processElement(ProcessContext c) {
@@ -115,7 +115,7 @@ public abstract class NexmarkQuery
            });
 
   /** Transform to key each auction by its seller id. */
-  protected static final ParDo.Bound<Auction, KV<Long, Auction>> AUCTION_BY_SELLER =
+  protected static final ParDo.SingleOutput<Auction, KV<Long, Auction>> AUCTION_BY_SELLER =
       ParDo.of(new DoFn<Auction, KV<Long, Auction>>() {
              @ProcessElement
              public void processElement(ProcessContext c) {
@@ -124,7 +124,7 @@ public abstract class NexmarkQuery
            });
 
   /** Transform to key each bid by it's auction id. */
-  protected static final ParDo.Bound<Bid, KV<Long, Bid>> BID_BY_AUCTION =
+  protected static final ParDo.SingleOutput<Bid, KV<Long, Bid>> BID_BY_AUCTION =
       ParDo.of(new DoFn<Bid, KV<Long, Bid>>() {
              @ProcessElement
              public void processElement(ProcessContext c) {
@@ -133,7 +133,7 @@ public abstract class NexmarkQuery
            });
 
   /** Transform to project the auction id from each bid. */
-  protected static final ParDo.Bound<Bid, Long> BID_TO_AUCTION =
+  protected static final ParDo.SingleOutput<Bid, Long> BID_TO_AUCTION =
       ParDo.of(new DoFn<Bid, Long>() {
              @ProcessElement
              public void processElement(ProcessContext c) {
@@ -142,7 +142,7 @@ public abstract class NexmarkQuery
            });
 
   /** Transform to project the price from each bid. */
-  protected static final ParDo.Bound<Bid, Long> BID_TO_PRICE =
+  protected static final ParDo.SingleOutput<Bid, Long> BID_TO_PRICE =
       ParDo.of(new DoFn<Bid, Long>() {
              @ProcessElement
              public void processElement(ProcessContext c) {
@@ -151,7 +151,7 @@ public abstract class NexmarkQuery
            });
 
   /** Transform to emit each event with the timestamp embedded within it. */
-  public static final ParDo.Bound<Event, Event> EVENT_TIMESTAMP_FROM_DATA =
+  public static final ParDo.SingleOutput<Event, Event> EVENT_TIMESTAMP_FROM_DATA =
       ParDo.of(new DoFn<Event, Event>() {
              @ProcessElement
              public void processElement(ProcessContext c) {

http://git-wip-us.apache.org/repos/asf/beam/blob/bd93c8b5/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkRunner.java
----------------------------------------------------------------------
diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkRunner.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkRunner.java
index e8d791f..df1000a 100644
--- a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkRunner.java
+++ b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkRunner.java
@@ -1073,8 +1073,7 @@ public abstract class NexmarkRunner<OptionT extends NexmarkOptions> {
       case BIGQUERY:
         // Multiple BigQuery backends to mimic what most customers do.
         PCollectionTuple res = formattedResults.apply(queryName + ".Partition",
-            ParDo.withOutputTags(MAIN, TupleTagList.of(SIDE))
-                 .of(new PartitionDoFn()));
+            ParDo.of(new PartitionDoFn()).withOutputTags(MAIN, TupleTagList.of(SIDE)));
         sinkResultsToBigQuery(res.get(MAIN), now, "main");
         sinkResultsToBigQuery(res.get(SIDE), now, "side");
         sinkResultsToBigQuery(formattedResults, now, "copy");

http://git-wip-us.apache.org/repos/asf/beam/blob/bd93c8b5/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkUtils.java
----------------------------------------------------------------------
diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkUtils.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkUtils.java
index b0421a4..a47ebcc 100644
--- a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkUtils.java
+++ b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkUtils.java
@@ -417,7 +417,7 @@ public class NexmarkUtils {
   /**
    * Return a transform to pass-through events, but count them as they go by.
    */
-  public static ParDo.Bound<Event, Event> snoop(final String name) {
+  public static ParDo.SingleOutput<Event, Event> snoop(final String name) {
     return ParDo.of(new DoFn<Event, Event>() {
                   final Aggregator<Long, Long> eventCounter =
                       createAggregator("events", Sum.ofLongs());
@@ -451,7 +451,7 @@ public class NexmarkUtils {
   /**
    * Return a transform to count and discard each element.
    */
-  public static <T> ParDo.Bound<T, Void> devNull(String name) {
+  public static <T> ParDo.SingleOutput<T, Void> devNull(String name) {
     return ParDo.of(new DoFn<T, Void>() {
                   final Aggregator<Long, Long> discardCounter =
                       createAggregator("discarded", Sum.ofLongs());
@@ -466,7 +466,7 @@ public class NexmarkUtils {
   /**
    * Return a transform to log each element, passing it through unchanged.
    */
-  public static <T> ParDo.Bound<T, T> log(final String name) {
+  public static <T> ParDo.SingleOutput<T, T> log(final String name) {
     return ParDo.of(new DoFn<T, T>() {
                   @ProcessElement
                   public void processElement(ProcessContext c) {
@@ -479,7 +479,7 @@ public class NexmarkUtils {
   /**
    * Return a transform to format each element as a string.
    */
-  public static <T> ParDo.Bound<T, String> format(String name) {
+  public static <T> ParDo.SingleOutput<T, String> format(String name) {
     return ParDo.of(new DoFn<T, String>() {
                   final Aggregator<Long, Long> recordCounter =
                       createAggregator("records", Sum.ofLongs());
@@ -495,7 +495,7 @@ public class NexmarkUtils {
   /**
    * Return a transform to make explicit the timestamp of each element.
    */
-  public static <T> ParDo.Bound<T, TimestampedValue<T>> stamp(String name) {
+  public static <T> ParDo.SingleOutput<T, TimestampedValue<T>> stamp(String name) {
     return ParDo.of(new DoFn<T, TimestampedValue<T>>() {
                   @ProcessElement
                   public void processElement(ProcessContext c) {
@@ -548,7 +548,7 @@ public class NexmarkUtils {
   /**
    * Return a transform to keep the CPU busy for given milliseconds on every record.
    */
-  public static <T> ParDo.Bound<T, T> cpuDelay(String name, final long delayMs) {
+  public static <T> ParDo.SingleOutput<T, T> cpuDelay(String name, final long delayMs) {
     return ParDo.of(new DoFn<T, T>() {
                   @ProcessElement
                   public void processElement(ProcessContext c) {
@@ -580,7 +580,7 @@ public class NexmarkUtils {
   /**
    * Return a transform to write given number of bytes to durable store on every record.
    */
-  public static <T> ParDo.Bound<T, T> diskBusy(String name, final long bytes) {
+  public static <T> ParDo.SingleOutput<T, T> diskBusy(String name, final long bytes) {
     return ParDo.of(new DoFn<T, T>() {
                   @ProcessElement
                   public void processElement(ProcessContext c) {
@@ -608,7 +608,7 @@ public class NexmarkUtils {
   /**
    * Return a transform to cast each element to {@link KnownSize}.
    */
-  private static <T extends KnownSize> ParDo.Bound<T, KnownSize> castToKnownSize() {
+  private static <T extends KnownSize> ParDo.SingleOutput<T, KnownSize> castToKnownSize() {
     return ParDo.of(new DoFn<T, KnownSize>() {
                   @ProcessElement
                   public void processElement(ProcessContext c) {

http://git-wip-us.apache.org/repos/asf/beam/blob/bd93c8b5/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query7.java
----------------------------------------------------------------------
diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query7.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query7.java
index 2835737..f3d1ba4 100644
--- a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query7.java
+++ b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query7.java
@@ -70,18 +70,18 @@ public class Query7 extends NexmarkQuery {
 
     return slidingBids
         // Select all bids which have that maximum price (there may be more than one).
-        .apply(name + ".Select",
-            ParDo.withSideInputs(maxPriceView)
-                .of(new DoFn<Bid, Bid>() {
-                  @ProcessElement
-                  public void processElement(ProcessContext c) {
-                    long maxPrice = c.sideInput(maxPriceView);
-                    Bid bid = c.element();
-                    if (bid.price == maxPrice) {
-                      c.output(bid);
-                    }
+        .apply(name + ".Select", ParDo
+          .of(new DoFn<Bid, Bid>() {
+                @ProcessElement
+                public void processElement(ProcessContext c) {
+                  long maxPrice = c.sideInput(maxPriceView);
+                  Bid bid = c.element();
+                  if (bid.price == maxPrice) {
+                    c.output(bid);
                   }
-                }));
+                }
+              })
+          .withSideInputs(maxPriceView));
   }
 
   @Override


[43/55] [abbrv] beam git commit: Move module beam-integration-java-nexmark to beam-sdks-java-nexmark

Posted by ie...@apache.org.
http://git-wip-us.apache.org/repos/asf/beam/blob/f4333df7/sdks/java/nexmark/src/main/java/org/apache/beam/sdk/nexmark/queries/AbstractSimulator.java
----------------------------------------------------------------------
diff --git a/sdks/java/nexmark/src/main/java/org/apache/beam/sdk/nexmark/queries/AbstractSimulator.java b/sdks/java/nexmark/src/main/java/org/apache/beam/sdk/nexmark/queries/AbstractSimulator.java
new file mode 100644
index 0000000..6f4ad56
--- /dev/null
+++ b/sdks/java/nexmark/src/main/java/org/apache/beam/sdk/nexmark/queries/AbstractSimulator.java
@@ -0,0 +1,211 @@
+/*
+ * 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.beam.sdk.nexmark.queries;
+
+import java.util.ArrayList;
+import java.util.Iterator;
+import java.util.List;
+import javax.annotation.Nullable;
+
+import org.apache.beam.sdk.nexmark.NexmarkUtils;
+import org.apache.beam.sdk.transforms.windowing.BoundedWindow;
+import org.apache.beam.sdk.values.TimestampedValue;
+import org.joda.time.Duration;
+import org.joda.time.Instant;
+
+/**
+ * Abstract base class for simulator of a query.
+ *
+ * @param <InputT> Type of input elements.
+ * @param <OutputT> Type of output elements.
+ */
+public abstract class AbstractSimulator<InputT, OutputT> {
+  /** Window size for action bucket sampling. */
+  private static final Duration WINDOW_SIZE = Duration.standardMinutes(1);
+
+  /** Input event stream we should draw from. */
+  private final Iterator<TimestampedValue<InputT>> input;
+
+  /** Set to true when no more results. */
+  private boolean isDone;
+
+  /**
+   * Results which have not yet been returned by the {@link #results} iterator.
+   */
+  private final List<TimestampedValue<OutputT>> pendingResults;
+
+  /**
+   * Current window timestamp (ms since epoch).
+   */
+  private long currentWindow;
+
+  /**
+   * Number of (possibly intermediate) results for the current window.
+   */
+  private long currentCount;
+
+  /**
+   * Result counts per window which have not yet been returned by the {@link #resultsPerWindow}
+   * iterator.
+   */
+  private final List<Long> pendingCounts;
+
+  public AbstractSimulator(Iterator<TimestampedValue<InputT>> input) {
+    this.input = input;
+    isDone = false;
+    pendingResults = new ArrayList<>();
+    currentWindow = BoundedWindow.TIMESTAMP_MIN_VALUE.getMillis();
+    currentCount = 0;
+    pendingCounts = new ArrayList<>();
+  }
+
+  /** Called by implementors of {@link #run}: Fetch the next input element. */
+  @Nullable
+  TimestampedValue<InputT> nextInput() {
+    if (!input.hasNext()) {
+      return null;
+    }
+    TimestampedValue<InputT> timestampedInput = input.next();
+    NexmarkUtils.info("input: %s", timestampedInput);
+    return timestampedInput;
+  }
+
+  /**
+   * Called by implementors of {@link #run}:  Capture an intermediate result, for the purpose of
+   * recording the expected activity of the query over time.
+   */
+  void addIntermediateResult(TimestampedValue<OutputT> result) {
+    NexmarkUtils.info("intermediate result: %s", result);
+    updateCounts(result.getTimestamp());
+  }
+
+  /**
+   * Called by implementors of {@link #run}: Capture a final result, for the purpose of checking
+   * semantic correctness.
+   */
+  void addResult(TimestampedValue<OutputT> result) {
+    NexmarkUtils.info("result: %s", result);
+    pendingResults.add(result);
+    updateCounts(result.getTimestamp());
+  }
+
+  /**
+   * Update window and counts.
+   */
+  private void updateCounts(Instant timestamp) {
+    long window = timestamp.getMillis() - timestamp.getMillis() % WINDOW_SIZE.getMillis();
+    if (window > currentWindow) {
+      if (currentWindow > BoundedWindow.TIMESTAMP_MIN_VALUE.getMillis()) {
+        pendingCounts.add(currentCount);
+      }
+      currentCount = 0;
+      currentWindow = window;
+    }
+    currentCount++;
+  }
+
+  /** Called by implementors of {@link #run}: Record that no more results will be emitted. */
+  void allDone() {
+    isDone = true;
+  }
+
+  /**
+   * Overridden by derived classes to do the next increment of work. Each call should
+   * call one or more of {@link #nextInput}, {@link #addIntermediateResult}, {@link #addResult}
+   * or {@link #allDone}. It is ok for a single call to emit more than one result via
+   * {@link #addResult}. It is ok for a single call to run the entire simulation, though
+   * this will prevent the {@link #results} and {@link #resultsPerWindow} iterators to
+   * stall.
+   */
+  protected abstract void run();
+
+  /**
+   * Return iterator over all expected timestamped results. The underlying simulator state is
+   * changed. Only one of {@link #results} or {@link #resultsPerWindow} can be called.
+   */
+  public Iterator<TimestampedValue<OutputT>> results() {
+    return new Iterator<TimestampedValue<OutputT>>() {
+      @Override
+      public boolean hasNext() {
+        while (true) {
+          if (!pendingResults.isEmpty()) {
+            return true;
+          }
+          if (isDone) {
+            return false;
+          }
+          run();
+        }
+      }
+
+      @Override
+      public TimestampedValue<OutputT> next() {
+        TimestampedValue<OutputT> result = pendingResults.get(0);
+        pendingResults.remove(0);
+        return result;
+      }
+
+      @Override
+      public void remove() {
+        throw new UnsupportedOperationException();
+      }
+    };
+  }
+
+  /**
+   * Return an iterator over the number of results per {@link #WINDOW_SIZE} period. The underlying
+   * simulator state is changed.  Only one of {@link #results} or {@link #resultsPerWindow} can be
+   * called.
+   */
+  public Iterator<Long> resultsPerWindow() {
+    return new Iterator<Long>() {
+      @Override
+      public boolean hasNext() {
+        while (true) {
+          if (!pendingCounts.isEmpty()) {
+            return true;
+          }
+          if (isDone) {
+            if (currentCount > 0) {
+              pendingCounts.add(currentCount);
+              currentCount = 0;
+              currentWindow = BoundedWindow.TIMESTAMP_MAX_VALUE.getMillis();
+              return true;
+            } else {
+              return false;
+            }
+          }
+          run();
+        }
+      }
+
+      @Override
+      public Long next() {
+        Long result = pendingCounts.get(0);
+        pendingCounts.remove(0);
+        return result;
+      }
+
+      @Override
+      public void remove() {
+        throw new UnsupportedOperationException();
+      }
+    };
+  }
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/f4333df7/sdks/java/nexmark/src/main/java/org/apache/beam/sdk/nexmark/queries/NexmarkQuery.java
----------------------------------------------------------------------
diff --git a/sdks/java/nexmark/src/main/java/org/apache/beam/sdk/nexmark/queries/NexmarkQuery.java b/sdks/java/nexmark/src/main/java/org/apache/beam/sdk/nexmark/queries/NexmarkQuery.java
new file mode 100644
index 0000000..d070058
--- /dev/null
+++ b/sdks/java/nexmark/src/main/java/org/apache/beam/sdk/nexmark/queries/NexmarkQuery.java
@@ -0,0 +1,270 @@
+/*
+ * 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.beam.sdk.nexmark.queries;
+
+import org.apache.beam.sdk.metrics.Counter;
+import org.apache.beam.sdk.metrics.Metrics;
+import org.apache.beam.sdk.nexmark.Monitor;
+import org.apache.beam.sdk.nexmark.NexmarkConfiguration;
+import org.apache.beam.sdk.nexmark.NexmarkUtils;
+import org.apache.beam.sdk.nexmark.model.Auction;
+import org.apache.beam.sdk.nexmark.model.Bid;
+import org.apache.beam.sdk.nexmark.model.Event;
+import org.apache.beam.sdk.nexmark.model.KnownSize;
+import org.apache.beam.sdk.nexmark.model.Person;
+import org.apache.beam.sdk.transforms.DoFn;
+import org.apache.beam.sdk.transforms.Filter;
+import org.apache.beam.sdk.transforms.PTransform;
+import org.apache.beam.sdk.transforms.ParDo;
+import org.apache.beam.sdk.transforms.SerializableFunction;
+import org.apache.beam.sdk.values.KV;
+import org.apache.beam.sdk.values.PCollection;
+import org.apache.beam.sdk.values.TimestampedValue;
+import org.apache.beam.sdk.values.TupleTag;
+import org.joda.time.Instant;
+
+/**
+ * Base class for the eight 'NEXMark' queries. Supplies some fragments common to
+ * multiple queries.
+ */
+public abstract class NexmarkQuery
+    extends PTransform<PCollection<Event>, PCollection<TimestampedValue<KnownSize>>> {
+  public static final TupleTag<Auction> AUCTION_TAG = new TupleTag<>("auctions");
+  public static final TupleTag<Bid> BID_TAG = new TupleTag<>("bids");
+  static final TupleTag<Person> PERSON_TAG = new TupleTag<>("person");
+
+  /** Predicate to detect a new person event. */
+  private static final SerializableFunction<Event, Boolean> IS_NEW_PERSON =
+      new SerializableFunction<Event, Boolean>() {
+        @Override
+        public Boolean apply(Event event) {
+          return event.newPerson != null;
+        }
+      };
+
+  /** DoFn to convert a new person event to a person. */
+  private static final DoFn<Event, Person> AS_PERSON = new DoFn<Event, Person>() {
+    @ProcessElement
+    public void processElement(ProcessContext c) {
+      c.output(c.element().newPerson);
+    }
+  };
+
+  /** Predicate to detect a new auction event. */
+  private static final SerializableFunction<Event, Boolean> IS_NEW_AUCTION =
+      new SerializableFunction<Event, Boolean>() {
+        @Override
+        public Boolean apply(Event event) {
+          return event.newAuction != null;
+        }
+      };
+
+  /** DoFn to convert a new auction event to an auction. */
+  private static final DoFn<Event, Auction> AS_AUCTION = new DoFn<Event, Auction>() {
+    @ProcessElement
+    public void processElement(ProcessContext c) {
+      c.output(c.element().newAuction);
+    }
+  };
+
+  /** Predicate to detect a new bid event. */
+  private static final SerializableFunction<Event, Boolean> IS_BID =
+      new SerializableFunction<Event, Boolean>() {
+        @Override
+        public Boolean apply(Event event) {
+          return event.bid != null;
+        }
+      };
+
+  /** DoFn to convert a bid event to a bid. */
+  private static final DoFn<Event, Bid> AS_BID = new DoFn<Event, Bid>() {
+    @ProcessElement
+    public void processElement(ProcessContext c) {
+      c.output(c.element().bid);
+    }
+  };
+
+  /** Transform to key each person by their id. */
+  static final ParDo.SingleOutput<Person, KV<Long, Person>> PERSON_BY_ID =
+      ParDo.of(new DoFn<Person, KV<Long, Person>>() {
+             @ProcessElement
+             public void processElement(ProcessContext c) {
+               c.output(KV.of(c.element().id, c.element()));
+             }
+           });
+
+  /** Transform to key each auction by its id. */
+  static final ParDo.SingleOutput<Auction, KV<Long, Auction>> AUCTION_BY_ID =
+      ParDo.of(new DoFn<Auction, KV<Long, Auction>>() {
+             @ProcessElement
+             public void processElement(ProcessContext c) {
+               c.output(KV.of(c.element().id, c.element()));
+             }
+           });
+
+  /** Transform to key each auction by its seller id. */
+  static final ParDo.SingleOutput<Auction, KV<Long, Auction>> AUCTION_BY_SELLER =
+      ParDo.of(new DoFn<Auction, KV<Long, Auction>>() {
+             @ProcessElement
+             public void processElement(ProcessContext c) {
+               c.output(KV.of(c.element().seller, c.element()));
+             }
+           });
+
+  /** Transform to key each bid by it's auction id. */
+  static final ParDo.SingleOutput<Bid, KV<Long, Bid>> BID_BY_AUCTION =
+      ParDo.of(new DoFn<Bid, KV<Long, Bid>>() {
+             @ProcessElement
+             public void processElement(ProcessContext c) {
+               c.output(KV.of(c.element().auction, c.element()));
+             }
+           });
+
+  /** Transform to project the auction id from each bid. */
+  static final ParDo.SingleOutput<Bid, Long> BID_TO_AUCTION =
+      ParDo.of(new DoFn<Bid, Long>() {
+             @ProcessElement
+             public void processElement(ProcessContext c) {
+               c.output(c.element().auction);
+             }
+           });
+
+  /** Transform to project the price from each bid. */
+  static final ParDo.SingleOutput<Bid, Long> BID_TO_PRICE =
+      ParDo.of(new DoFn<Bid, Long>() {
+             @ProcessElement
+             public void processElement(ProcessContext c) {
+               c.output(c.element().price);
+             }
+           });
+
+  /** Transform to emit each event with the timestamp embedded within it. */
+  public static final ParDo.SingleOutput<Event, Event> EVENT_TIMESTAMP_FROM_DATA =
+      ParDo.of(new DoFn<Event, Event>() {
+             @ProcessElement
+             public void processElement(ProcessContext c) {
+               Event e = c.element();
+               if (e.bid != null) {
+                 c.outputWithTimestamp(e, new Instant(e.bid.dateTime));
+               } else if (e.newPerson != null) {
+                 c.outputWithTimestamp(e, new Instant(e.newPerson.dateTime));
+               } else if (e.newAuction != null) {
+                 c.outputWithTimestamp(e, new Instant(e.newAuction.dateTime));
+               }
+             }
+           });
+
+  /**
+   * Transform to filter for just the new auction events.
+   */
+  public static final PTransform<PCollection<Event>, PCollection<Auction>> JUST_NEW_AUCTIONS =
+      new PTransform<PCollection<Event>, PCollection<Auction>>("justNewAuctions") {
+        @Override
+        public PCollection<Auction> expand(PCollection<Event> input) {
+          return input.apply("IsNewAuction", Filter.by(IS_NEW_AUCTION))
+                      .apply("AsAuction", ParDo.of(AS_AUCTION));
+        }
+      };
+
+  /**
+   * Transform to filter for just the new person events.
+   */
+  public static final PTransform<PCollection<Event>, PCollection<Person>> JUST_NEW_PERSONS =
+      new PTransform<PCollection<Event>, PCollection<Person>>("justNewPersons") {
+        @Override
+        public PCollection<Person> expand(PCollection<Event> input) {
+          return input.apply("IsNewPerson", Filter.by(IS_NEW_PERSON))
+                      .apply("AsPerson", ParDo.of(AS_PERSON));
+        }
+      };
+
+  /**
+   * Transform to filter for just the bid events.
+   */
+  public static final PTransform<PCollection<Event>, PCollection<Bid>> JUST_BIDS =
+      new PTransform<PCollection<Event>, PCollection<Bid>>("justBids") {
+        @Override
+        public PCollection<Bid> expand(PCollection<Event> input) {
+          return input.apply("IsBid", Filter.by(IS_BID))
+                      .apply("AsBid", ParDo.of(AS_BID));
+        }
+      };
+
+  final NexmarkConfiguration configuration;
+  public final Monitor<Event> eventMonitor;
+  public final Monitor<KnownSize> resultMonitor;
+  private final Monitor<Event> endOfStreamMonitor;
+  private final Counter fatalCounter;
+
+  NexmarkQuery(NexmarkConfiguration configuration, String name) {
+    super(name);
+    this.configuration = configuration;
+    if (configuration.debug) {
+      eventMonitor = new Monitor<>(name + ".Events", "event");
+      resultMonitor = new Monitor<>(name + ".Results", "result");
+      endOfStreamMonitor = new Monitor<>(name + ".EndOfStream", "end");
+      fatalCounter = Metrics.counter(name , "fatal");
+    } else {
+      eventMonitor = null;
+      resultMonitor = null;
+      endOfStreamMonitor = null;
+      fatalCounter = null;
+    }
+  }
+
+  /**
+   * Implement the actual query. All we know about the result is it has a known encoded size.
+   */
+  protected abstract PCollection<KnownSize> applyPrim(PCollection<Event> events);
+
+  @Override
+  public PCollection<TimestampedValue<KnownSize>> expand(PCollection<Event> events) {
+
+    if (configuration.debug) {
+      events =
+          events
+              // Monitor events as they go by.
+              .apply(name + ".Monitor", eventMonitor.getTransform())
+              // Count each type of event.
+              .apply(name + ".Snoop", NexmarkUtils.snoop(name));
+    }
+
+    if (configuration.cpuDelayMs > 0) {
+      // Slow down by pegging one core at 100%.
+      events = events.apply(name + ".CpuDelay",
+              NexmarkUtils.<Event>cpuDelay(name, configuration.cpuDelayMs));
+    }
+
+    if (configuration.diskBusyBytes > 0) {
+      // Slow down by forcing bytes to durable store.
+      events = events.apply(name + ".DiskBusy",
+              NexmarkUtils.<Event>diskBusy(configuration.diskBusyBytes));
+    }
+
+    // Run the query.
+    PCollection<KnownSize> queryResults = applyPrim(events);
+
+    if (configuration.debug) {
+      // Monitor results as they go by.
+      queryResults = queryResults.apply(name + ".Debug", resultMonitor.getTransform());
+    }
+
+    // Timestamp the query results.
+    return queryResults.apply(name + ".Stamp", NexmarkUtils.<KnownSize>stamp(name));
+  }
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/f4333df7/sdks/java/nexmark/src/main/java/org/apache/beam/sdk/nexmark/queries/NexmarkQueryModel.java
----------------------------------------------------------------------
diff --git a/sdks/java/nexmark/src/main/java/org/apache/beam/sdk/nexmark/queries/NexmarkQueryModel.java b/sdks/java/nexmark/src/main/java/org/apache/beam/sdk/nexmark/queries/NexmarkQueryModel.java
new file mode 100644
index 0000000..1f093a0
--- /dev/null
+++ b/sdks/java/nexmark/src/main/java/org/apache/beam/sdk/nexmark/queries/NexmarkQueryModel.java
@@ -0,0 +1,118 @@
+/*
+ * 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.beam.sdk.nexmark.queries;
+
+import java.io.Serializable;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Set;
+
+import org.apache.beam.sdk.nexmark.NexmarkConfiguration;
+import org.apache.beam.sdk.nexmark.model.KnownSize;
+import org.apache.beam.sdk.transforms.SerializableFunction;
+import org.apache.beam.sdk.values.TimestampedValue;
+
+
+import org.hamcrest.core.IsEqual;
+import org.joda.time.Duration;
+import org.joda.time.Instant;
+import org.junit.Assert;
+
+/**
+ * Base class for models of the eight NEXMark queries. Provides an assertion function which can be
+ * applied against the actual query results to check their consistency with the model.
+ */
+public abstract class NexmarkQueryModel implements Serializable {
+  public final NexmarkConfiguration configuration;
+
+  NexmarkQueryModel(NexmarkConfiguration configuration) {
+    this.configuration = configuration;
+  }
+
+  /**
+   * Return the start of the most recent window of {@code size} and {@code period} which ends
+   * strictly before {@code timestamp}.
+   */
+  static Instant windowStart(Duration size, Duration period, Instant timestamp) {
+    long ts = timestamp.getMillis();
+    long p = period.getMillis();
+    long lim = ts - ts % p;
+    long s = size.getMillis();
+    return new Instant(lim - s);
+  }
+
+  /** Convert {@code itr} to strings capturing values, timestamps and order. */
+  static <T> List<String> toValueTimestampOrder(Iterator<TimestampedValue<T>> itr) {
+    List<String> strings = new ArrayList<>();
+    while (itr.hasNext()) {
+      strings.add(itr.next().toString());
+    }
+    return strings;
+  }
+
+  /** Convert {@code itr} to strings capturing values and order. */
+  static <T> List<String> toValueOrder(Iterator<TimestampedValue<T>> itr) {
+    List<String> strings = new ArrayList<>();
+    while (itr.hasNext()) {
+      strings.add(itr.next().getValue().toString());
+    }
+    return strings;
+  }
+
+  /** Convert {@code itr} to strings capturing values only. */
+  static <T> Set<String> toValue(Iterator<TimestampedValue<T>> itr) {
+    Set<String> strings = new HashSet<>();
+    while (itr.hasNext()) {
+      strings.add(itr.next().getValue().toString());
+    }
+    return strings;
+  }
+
+  /** Return simulator for query. */
+  public abstract AbstractSimulator<?, ?> simulator();
+
+  /** Return sub-sequence of results which are significant for model. */
+  Iterable<TimestampedValue<KnownSize>> relevantResults(
+      Iterable<TimestampedValue<KnownSize>> results) {
+    return results;
+  }
+
+  /**
+   * Convert iterator of elements to collection of strings to use when testing coherence of model
+   * against actual query results.
+   */
+  protected abstract <T> Collection<String> toCollection(Iterator<TimestampedValue<T>> itr);
+
+  /** Return assertion to use on results of pipeline for this query. */
+  public SerializableFunction<Iterable<TimestampedValue<KnownSize>>, Void> assertionFor() {
+    final Collection<String> expectedStrings = toCollection(simulator().results());
+
+    return new SerializableFunction<Iterable<TimestampedValue<KnownSize>>, Void>() {
+      @Override
+      public Void apply(Iterable<TimestampedValue<KnownSize>> actual) {
+      Collection<String> actualStrings = toCollection(relevantResults(actual).iterator());
+        Assert.assertThat("wrong pipeline output", actualStrings,
+          IsEqual.equalTo(expectedStrings));
+        return null;
+      }
+    };
+  }
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/f4333df7/sdks/java/nexmark/src/main/java/org/apache/beam/sdk/nexmark/queries/Query0.java
----------------------------------------------------------------------
diff --git a/sdks/java/nexmark/src/main/java/org/apache/beam/sdk/nexmark/queries/Query0.java b/sdks/java/nexmark/src/main/java/org/apache/beam/sdk/nexmark/queries/Query0.java
new file mode 100644
index 0000000..68bf78e
--- /dev/null
+++ b/sdks/java/nexmark/src/main/java/org/apache/beam/sdk/nexmark/queries/Query0.java
@@ -0,0 +1,70 @@
+/*
+ * 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.beam.sdk.nexmark.queries;
+
+import java.io.ByteArrayInputStream;
+import java.io.ByteArrayOutputStream;
+import java.io.IOException;
+import org.apache.beam.sdk.coders.Coder;
+import org.apache.beam.sdk.coders.CoderException;
+import org.apache.beam.sdk.metrics.Counter;
+import org.apache.beam.sdk.metrics.Metrics;
+import org.apache.beam.sdk.nexmark.NexmarkConfiguration;
+import org.apache.beam.sdk.nexmark.NexmarkUtils;
+import org.apache.beam.sdk.nexmark.model.Event;
+import org.apache.beam.sdk.nexmark.model.KnownSize;
+import org.apache.beam.sdk.transforms.DoFn;
+import org.apache.beam.sdk.transforms.ParDo;
+import org.apache.beam.sdk.values.PCollection;
+
+/**
+ * Query 0: Pass events through unchanged. However, force them to do a round trip through
+ * serialization so that we measure the impact of the choice of coders.
+ */
+public class Query0 extends NexmarkQuery {
+  public Query0(NexmarkConfiguration configuration) {
+    super(configuration, "Query0");
+  }
+
+  private PCollection<Event> applyTyped(PCollection<Event> events) {
+    final Coder<Event> coder = events.getCoder();
+    return events
+        // Force round trip through coder.
+        .apply(name + ".Serialize",
+            ParDo.of(new DoFn<Event, Event>() {
+                  private final Counter bytesMetric =
+                    Metrics.counter(name , "bytes");
+
+                  @ProcessElement
+                  public void processElement(ProcessContext c) throws CoderException, IOException {
+                    ByteArrayOutputStream outStream = new ByteArrayOutputStream();
+                    coder.encode(c.element(), outStream, Coder.Context.OUTER);
+                    byte[] byteArray = outStream.toByteArray();
+                    bytesMetric.inc((long) byteArray.length);
+                    ByteArrayInputStream inStream = new ByteArrayInputStream(byteArray);
+                    Event event = coder.decode(inStream, Coder.Context.OUTER);
+                    c.output(event);
+                  }
+                }));
+  }
+
+  @Override
+  protected PCollection<KnownSize> applyPrim(PCollection<Event> events) {
+    return NexmarkUtils.castToKnownSize(name, applyTyped(events));
+  }
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/f4333df7/sdks/java/nexmark/src/main/java/org/apache/beam/sdk/nexmark/queries/Query0Model.java
----------------------------------------------------------------------
diff --git a/sdks/java/nexmark/src/main/java/org/apache/beam/sdk/nexmark/queries/Query0Model.java b/sdks/java/nexmark/src/main/java/org/apache/beam/sdk/nexmark/queries/Query0Model.java
new file mode 100644
index 0000000..0e73a21
--- /dev/null
+++ b/sdks/java/nexmark/src/main/java/org/apache/beam/sdk/nexmark/queries/Query0Model.java
@@ -0,0 +1,64 @@
+/*
+ * 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.beam.sdk.nexmark.queries;
+
+import java.util.Collection;
+import java.util.Iterator;
+
+import org.apache.beam.sdk.nexmark.NexmarkConfiguration;
+import org.apache.beam.sdk.nexmark.NexmarkUtils;
+import org.apache.beam.sdk.nexmark.model.Event;
+import org.apache.beam.sdk.values.TimestampedValue;
+
+/**
+ * A direct implementation of {@link Query0}.
+ */
+public class Query0Model extends NexmarkQueryModel {
+  /**
+   * Simulator for query 0.
+   */
+  private static class Simulator extends AbstractSimulator<Event, Event> {
+    public Simulator(NexmarkConfiguration configuration) {
+      super(NexmarkUtils.standardEventIterator(configuration));
+    }
+
+    @Override
+    protected void run() {
+      TimestampedValue<Event> timestampedEvent = nextInput();
+      if (timestampedEvent == null) {
+        allDone();
+        return;
+      }
+      addResult(timestampedEvent);
+    }
+  }
+
+  public Query0Model(NexmarkConfiguration configuration) {
+    super(configuration);
+  }
+
+  @Override
+  public AbstractSimulator<?, ?> simulator() {
+    return new Simulator(configuration);
+  }
+
+  @Override
+  protected <T> Collection<String> toCollection(Iterator<TimestampedValue<T>> itr) {
+    return toValueTimestampOrder(itr);
+  }
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/f4333df7/sdks/java/nexmark/src/main/java/org/apache/beam/sdk/nexmark/queries/Query1.java
----------------------------------------------------------------------
diff --git a/sdks/java/nexmark/src/main/java/org/apache/beam/sdk/nexmark/queries/Query1.java b/sdks/java/nexmark/src/main/java/org/apache/beam/sdk/nexmark/queries/Query1.java
new file mode 100644
index 0000000..810cd87
--- /dev/null
+++ b/sdks/java/nexmark/src/main/java/org/apache/beam/sdk/nexmark/queries/Query1.java
@@ -0,0 +1,67 @@
+/*
+ * 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.beam.sdk.nexmark.queries;
+
+import org.apache.beam.sdk.nexmark.NexmarkConfiguration;
+import org.apache.beam.sdk.nexmark.NexmarkUtils;
+import org.apache.beam.sdk.nexmark.model.Bid;
+import org.apache.beam.sdk.nexmark.model.Event;
+import org.apache.beam.sdk.nexmark.model.KnownSize;
+import org.apache.beam.sdk.transforms.DoFn;
+import org.apache.beam.sdk.transforms.ParDo;
+import org.apache.beam.sdk.values.PCollection;
+
+/**
+ * Query 1, 'Currency Conversion'. Convert each bid value from dollars to euros.
+ * In CQL syntax:
+ *
+ * <pre>
+ * SELECT Istream(auction, DOLTOEUR(price), bidder, datetime)
+ * FROM bid [ROWS UNBOUNDED];
+ * </pre>
+ *
+ * <p>To make things more interesting, allow the 'currency conversion' to be arbitrarily
+ * slowed down.
+ */
+public class Query1 extends NexmarkQuery {
+  public Query1(NexmarkConfiguration configuration) {
+    super(configuration, "Query1");
+  }
+
+  private PCollection<Bid> applyTyped(PCollection<Event> events) {
+    return events
+        // Only want the bid events.
+        .apply(JUST_BIDS)
+
+        // Map the conversion function over all bids.
+        .apply(name + ".ToEuros",
+            ParDo.of(new DoFn<Bid, Bid>() {
+                  @ProcessElement
+                  public void processElement(ProcessContext c) {
+                    Bid bid = c.element();
+                    c.output(new Bid(
+                        bid.auction, bid.bidder, (bid.price * 89) / 100, bid.dateTime, bid.extra));
+                  }
+                }));
+  }
+
+  @Override
+  protected PCollection<KnownSize> applyPrim(PCollection<Event> events) {
+    return NexmarkUtils.castToKnownSize(name, applyTyped(events));
+  }
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/f4333df7/sdks/java/nexmark/src/main/java/org/apache/beam/sdk/nexmark/queries/Query10.java
----------------------------------------------------------------------
diff --git a/sdks/java/nexmark/src/main/java/org/apache/beam/sdk/nexmark/queries/Query10.java b/sdks/java/nexmark/src/main/java/org/apache/beam/sdk/nexmark/queries/Query10.java
new file mode 100644
index 0000000..1c4e443
--- /dev/null
+++ b/sdks/java/nexmark/src/main/java/org/apache/beam/sdk/nexmark/queries/Query10.java
@@ -0,0 +1,367 @@
+/*
+ * 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.beam.sdk.nexmark.queries;
+
+import java.io.IOException;
+import java.io.OutputStream;
+import java.io.Serializable;
+import java.nio.channels.Channels;
+import java.nio.channels.WritableByteChannel;
+import java.util.concurrent.ThreadLocalRandom;
+import javax.annotation.Nullable;
+import org.apache.beam.sdk.coders.Coder;
+import org.apache.beam.sdk.extensions.gcp.options.GcsOptions;
+import org.apache.beam.sdk.metrics.Counter;
+import org.apache.beam.sdk.metrics.Metrics;
+import org.apache.beam.sdk.nexmark.NexmarkConfiguration;
+import org.apache.beam.sdk.nexmark.NexmarkUtils;
+import org.apache.beam.sdk.nexmark.model.Done;
+import org.apache.beam.sdk.nexmark.model.Event;
+import org.apache.beam.sdk.nexmark.model.KnownSize;
+import org.apache.beam.sdk.transforms.DoFn;
+import org.apache.beam.sdk.transforms.GroupByKey;
+import org.apache.beam.sdk.transforms.ParDo;
+import org.apache.beam.sdk.transforms.windowing.AfterEach;
+import org.apache.beam.sdk.transforms.windowing.AfterFirst;
+import org.apache.beam.sdk.transforms.windowing.AfterPane;
+import org.apache.beam.sdk.transforms.windowing.AfterProcessingTime;
+import org.apache.beam.sdk.transforms.windowing.AfterWatermark;
+import org.apache.beam.sdk.transforms.windowing.BoundedWindow;
+import org.apache.beam.sdk.transforms.windowing.FixedWindows;
+import org.apache.beam.sdk.transforms.windowing.PaneInfo;
+import org.apache.beam.sdk.transforms.windowing.PaneInfo.Timing;
+import org.apache.beam.sdk.transforms.windowing.Repeatedly;
+import org.apache.beam.sdk.transforms.windowing.Window;
+import org.apache.beam.sdk.values.KV;
+import org.apache.beam.sdk.values.PCollection;
+import org.joda.time.Duration;
+import org.joda.time.Instant;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Query "10", 'Log to sharded files' (Not in original suite.)
+ *
+ * <p>Every windowSizeSec, save all events from the last period into 2*maxWorkers log files.
+ */
+public class Query10 extends NexmarkQuery {
+  private static final Logger LOG = LoggerFactory.getLogger(Query10.class);
+  private static final int CHANNEL_BUFFER = 8 << 20; // 8MB
+  private static final int NUM_SHARDS_PER_WORKER = 5;
+  private static final Duration LATE_BATCHING_PERIOD = Duration.standardSeconds(10);
+
+  /**
+   * Capture everything we need to know about the records in a single output file.
+   */
+  private static class OutputFile implements Serializable {
+    /** Maximum possible timestamp of records in file. */
+    private final Instant maxTimestamp;
+    /** Shard within window. */
+    private final String shard;
+    /** Index of file in all files in shard. */
+    private final long index;
+    /** Timing of records in this file. */
+    private final PaneInfo.Timing timing;
+    /** Path to file containing records, or {@literal null} if no output required. */
+    @Nullable
+    private final String filename;
+
+    public OutputFile(
+        Instant maxTimestamp,
+        String shard,
+        long index,
+        PaneInfo.Timing timing,
+        @Nullable String filename) {
+      this.maxTimestamp = maxTimestamp;
+      this.shard = shard;
+      this.index = index;
+      this.timing = timing;
+      this.filename = filename;
+    }
+
+    @Override
+    public String toString() {
+      return String.format("%s %s %d %s %s%n", maxTimestamp, shard, index, timing, filename);
+    }
+  }
+
+  /**
+   * GCS uri prefix for all log and 'finished' files. If null they won't be written.
+   */
+  @Nullable
+  private String outputPath;
+
+  /**
+   * Maximum number of workers, used to determine log sharding factor.
+   */
+  private int maxNumWorkers;
+
+  public Query10(NexmarkConfiguration configuration) {
+    super(configuration, "Query10");
+  }
+
+  public void setOutputPath(@Nullable String outputPath) {
+    this.outputPath = outputPath;
+  }
+
+  public void setMaxNumWorkers(int maxNumWorkers) {
+    this.maxNumWorkers = maxNumWorkers;
+  }
+
+  /**
+   * Return channel for writing bytes to GCS.
+   */
+  private WritableByteChannel openWritableGcsFile(GcsOptions options, String filename)
+      throws IOException {
+    //TODO
+    // Fix after PR: right now this is a specific Google added use case
+    // Discuss it on ML: shall we keep GCS or use HDFS or use a generic beam filesystem way.
+    throw new UnsupportedOperationException("Disabled after removal of GcsIOChannelFactory");
+  }
+
+  /** Return a short string to describe {@code timing}. */
+  private String timingToString(PaneInfo.Timing timing) {
+    switch (timing) {
+      case EARLY:
+        return "E";
+      case ON_TIME:
+        return "O";
+      case LATE:
+        return "L";
+    }
+    throw new RuntimeException(); // cases are exhaustive
+  }
+
+  /** Construct an {@link OutputFile} for {@code pane} in {@code window} for {@code shard}. */
+  private OutputFile outputFileFor(BoundedWindow window, String shard, PaneInfo pane) {
+    @Nullable String filename =
+        outputPath == null
+        ? null
+        : String.format("%s/LOG-%s-%s-%03d-%s-%x",
+            outputPath, window.maxTimestamp(), shard, pane.getIndex(),
+            timingToString(pane.getTiming()),
+            ThreadLocalRandom.current().nextLong());
+    return new OutputFile(window.maxTimestamp(), shard, pane.getIndex(),
+        pane.getTiming(), filename);
+  }
+
+  /**
+   * Return path to which we should write the index for {@code window}, or {@literal null}
+   * if no output required.
+   */
+  @Nullable
+  private String indexPathFor(BoundedWindow window) {
+    if (outputPath == null) {
+      return null;
+    }
+    return String.format("%s/INDEX-%s", outputPath, window.maxTimestamp());
+  }
+
+  private PCollection<Done> applyTyped(PCollection<Event> events) {
+    final int numLogShards = maxNumWorkers * NUM_SHARDS_PER_WORKER;
+
+    return events
+      .apply(name + ".ShardEvents",
+        ParDo.of(new DoFn<Event, KV<String, Event>>() {
+          private final Counter lateCounter = Metrics.counter(name , "actuallyLateEvent");
+          private final Counter onTimeCounter = Metrics.counter(name , "onTimeCounter");
+
+          @ProcessElement
+          public void processElement(ProcessContext c) {
+            if (c.element().hasAnnotation("LATE")) {
+              lateCounter.inc();
+              LOG.info("Observed late: %s", c.element());
+            } else {
+              onTimeCounter.inc();
+            }
+            int shardNum = (int) Math.abs((long) c.element().hashCode() % numLogShards);
+            String shard = String.format("shard-%05d-of-%05d", shardNum, numLogShards);
+            c.output(KV.of(shard, c.element()));
+          }
+        }))
+      .apply(name + ".WindowEvents",
+        Window.<KV<String, Event>>into(
+          FixedWindows.of(Duration.standardSeconds(configuration.windowSizeSec)))
+          .triggering(AfterEach.inOrder(
+              Repeatedly
+                  .forever(AfterPane.elementCountAtLeast(configuration.maxLogEvents))
+                  .orFinally(AfterWatermark.pastEndOfWindow()),
+              Repeatedly.forever(
+                  AfterFirst.of(AfterPane.elementCountAtLeast(configuration.maxLogEvents),
+                      AfterProcessingTime.pastFirstElementInPane()
+                                         .plusDelayOf(LATE_BATCHING_PERIOD)))))
+          .discardingFiredPanes()
+          // Use a 1 day allowed lateness so that any forgotten hold will stall the
+          // pipeline for that period and be very noticeable.
+          .withAllowedLateness(Duration.standardDays(1)))
+      .apply(name + ".GroupByKey", GroupByKey.<String, Event>create())
+      .apply(name + ".CheckForLateEvents",
+        ParDo.of(new DoFn<KV<String, Iterable<Event>>,
+                 KV<String, Iterable<Event>>>() {
+          private final Counter earlyCounter = Metrics.counter(name , "earlyShard");
+          private final Counter onTimeCounter = Metrics.counter(name , "onTimeShard");
+          private final Counter lateCounter = Metrics.counter(name , "lateShard");
+          private final Counter unexpectedLatePaneCounter =
+            Metrics.counter(name , "ERROR_unexpectedLatePane");
+          private final Counter unexpectedOnTimeElementCounter =
+            Metrics.counter(name , "ERROR_unexpectedOnTimeElement");
+
+          @ProcessElement
+          public void processElement(ProcessContext c, BoundedWindow window) {
+            int numLate = 0;
+            int numOnTime = 0;
+            for (Event event : c.element().getValue()) {
+              if (event.hasAnnotation("LATE")) {
+                numLate++;
+              } else {
+                numOnTime++;
+              }
+            }
+            String shard = c.element().getKey();
+            LOG.info(String.format(
+                "%s with timestamp %s has %d actually late and %d on-time "
+                    + "elements in pane %s for window %s",
+                shard, c.timestamp(), numLate, numOnTime, c.pane(),
+                window.maxTimestamp()));
+            if (c.pane().getTiming() == PaneInfo.Timing.LATE) {
+              if (numLate == 0) {
+                LOG.error(
+                    "ERROR! No late events in late pane for %s", shard);
+                unexpectedLatePaneCounter.inc();
+              }
+              if (numOnTime > 0) {
+                LOG.error(
+                    "ERROR! Have %d on-time events in late pane for %s",
+                    numOnTime, shard);
+                unexpectedOnTimeElementCounter.inc();
+              }
+              lateCounter.inc();
+            } else if (c.pane().getTiming() == PaneInfo.Timing.EARLY) {
+              if (numOnTime + numLate < configuration.maxLogEvents) {
+                LOG.error(
+                    "ERROR! Only have %d events in early pane for %s",
+                    numOnTime + numLate, shard);
+              }
+              earlyCounter.inc();
+            } else {
+              onTimeCounter.inc();
+            }
+            c.output(c.element());
+          }
+        }))
+      .apply(name + ".UploadEvents",
+        ParDo.of(new DoFn<KV<String, Iterable<Event>>,
+                 KV<Void, OutputFile>>() {
+          private final Counter savedFileCounter = Metrics.counter(name , "savedFile");
+          private final Counter writtenRecordsCounter = Metrics.counter(name , "writtenRecords");
+
+            @ProcessElement
+            public void processElement(ProcessContext c, BoundedWindow window)
+                    throws IOException {
+              String shard = c.element().getKey();
+              GcsOptions options = c.getPipelineOptions().as(GcsOptions.class);
+              OutputFile outputFile = outputFileFor(window, shard, c.pane());
+              LOG.info(String.format(
+                  "Writing %s with record timestamp %s, window timestamp %s, pane %s",
+                  shard, c.timestamp(), window.maxTimestamp(), c.pane()));
+              if (outputFile.filename != null) {
+                LOG.info("Beginning write to '%s'", outputFile.filename);
+                int n = 0;
+                try (OutputStream output =
+                         Channels.newOutputStream(openWritableGcsFile(options, outputFile
+                             .filename))) {
+                  for (Event event : c.element().getValue()) {
+                    Event.CODER.encode(event, output, Coder.Context.OUTER);
+                    writtenRecordsCounter.inc();
+                    if (++n % 10000 == 0) {
+                      LOG.info("So far written %d records to '%s'", n,
+                          outputFile.filename);
+                    }
+                  }
+                }
+                LOG.info("Written all %d records to '%s'", n, outputFile.filename);
+              }
+              savedFileCounter.inc();
+              c.output(KV.<Void, OutputFile>of(null, outputFile));
+            }
+          }))
+      // Clear fancy triggering from above.
+      .apply(name + ".WindowLogFiles", Window.<KV<Void, OutputFile>>into(
+        FixedWindows.of(Duration.standardSeconds(configuration.windowSizeSec)))
+        .triggering(AfterWatermark.pastEndOfWindow())
+        // We expect no late data here, but we'll assume the worst so we can detect any.
+        .withAllowedLateness(Duration.standardDays(1))
+        .discardingFiredPanes())
+      // this GroupByKey allows to have one file per window
+      .apply(name + ".GroupByKey2", GroupByKey.<Void, OutputFile>create())
+      .apply(name + ".Index",
+        ParDo.of(new DoFn<KV<Void, Iterable<OutputFile>>, Done>() {
+          private final Counter unexpectedLateCounter =
+            Metrics.counter(name , "ERROR_unexpectedLate");
+          private final Counter unexpectedEarlyCounter =
+              Metrics.counter(name , "ERROR_unexpectedEarly");
+          private final Counter unexpectedIndexCounter =
+              Metrics.counter(name , "ERROR_unexpectedIndex");
+          private final Counter finalizedCounter = Metrics.counter(name , "indexed");
+
+          @ProcessElement
+          public void processElement(ProcessContext c, BoundedWindow window)
+                  throws IOException {
+            if (c.pane().getTiming() == Timing.LATE) {
+              unexpectedLateCounter.inc();
+              LOG.error("ERROR! Unexpected LATE pane: %s", c.pane());
+            } else if (c.pane().getTiming() == Timing.EARLY) {
+              unexpectedEarlyCounter.inc();
+              LOG.error("ERROR! Unexpected EARLY pane: %s", c.pane());
+            } else if (c.pane().getTiming() == Timing.ON_TIME
+                && c.pane().getIndex() != 0) {
+              unexpectedIndexCounter.inc();
+              LOG.error("ERROR! Unexpected ON_TIME pane index: %s", c.pane());
+            } else {
+              GcsOptions options = c.getPipelineOptions().as(GcsOptions.class);
+              LOG.info(
+                  "Index with record timestamp %s, window timestamp %s, pane %s",
+                  c.timestamp(), window.maxTimestamp(), c.pane());
+
+              @Nullable String filename = indexPathFor(window);
+              if (filename != null) {
+                LOG.info("Beginning write to '%s'", filename);
+                int n = 0;
+                try (OutputStream output =
+                         Channels.newOutputStream(
+                             openWritableGcsFile(options, filename))) {
+                  for (OutputFile outputFile : c.element().getValue()) {
+                    output.write(outputFile.toString().getBytes("UTF-8"));
+                    n++;
+                  }
+                }
+                LOG.info("Written all %d lines to '%s'", n, filename);
+              }
+              c.output(
+                  new Done("written for timestamp " + window.maxTimestamp()));
+              finalizedCounter.inc();
+            }
+          }
+        }));
+  }
+
+  @Override
+  protected PCollection<KnownSize> applyPrim(PCollection<Event> events) {
+    return NexmarkUtils.castToKnownSize(name, applyTyped(events));
+  }
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/f4333df7/sdks/java/nexmark/src/main/java/org/apache/beam/sdk/nexmark/queries/Query11.java
----------------------------------------------------------------------
diff --git a/sdks/java/nexmark/src/main/java/org/apache/beam/sdk/nexmark/queries/Query11.java b/sdks/java/nexmark/src/main/java/org/apache/beam/sdk/nexmark/queries/Query11.java
new file mode 100644
index 0000000..47e7c00
--- /dev/null
+++ b/sdks/java/nexmark/src/main/java/org/apache/beam/sdk/nexmark/queries/Query11.java
@@ -0,0 +1,79 @@
+/*
+ * 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.beam.sdk.nexmark.queries;
+
+import org.apache.beam.sdk.nexmark.NexmarkConfiguration;
+import org.apache.beam.sdk.nexmark.NexmarkUtils;
+import org.apache.beam.sdk.nexmark.model.Bid;
+import org.apache.beam.sdk.nexmark.model.BidsPerSession;
+import org.apache.beam.sdk.nexmark.model.Event;
+import org.apache.beam.sdk.nexmark.model.KnownSize;
+import org.apache.beam.sdk.transforms.Count;
+import org.apache.beam.sdk.transforms.DoFn;
+import org.apache.beam.sdk.transforms.ParDo;
+import org.apache.beam.sdk.transforms.windowing.AfterPane;
+import org.apache.beam.sdk.transforms.windowing.Repeatedly;
+import org.apache.beam.sdk.transforms.windowing.Sessions;
+import org.apache.beam.sdk.transforms.windowing.Window;
+import org.apache.beam.sdk.values.KV;
+import org.apache.beam.sdk.values.PCollection;
+import org.joda.time.Duration;
+
+/**
+ * Query "11", 'User sessions' (Not in original suite.)
+ *
+ * <p>Group bids by the same user into sessions with {@code windowSizeSec} max gap.
+ * However limit the session to at most {@code maxLogEvents}. Emit the number of
+ * bids per session.
+ */
+public class Query11 extends NexmarkQuery {
+  public Query11(NexmarkConfiguration configuration) {
+    super(configuration, "Query11");
+  }
+
+  private PCollection<BidsPerSession> applyTyped(PCollection<Event> events) {
+    PCollection<Long> bidders = events.apply(JUST_BIDS).apply(name + ".Rekey",
+        ParDo.of(new DoFn<Bid, Long>() {
+
+          @ProcessElement public void processElement(ProcessContext c) {
+            Bid bid = c.element();
+            c.output(bid.bidder);
+          }
+        }));
+
+    PCollection<Long> biddersWindowed = bidders.apply(
+        Window.<Long>into(
+          Sessions.withGapDuration(Duration.standardSeconds(configuration.windowSizeSec)))
+            .triggering(
+                Repeatedly.forever(AfterPane.elementCountAtLeast(configuration.maxLogEvents)))
+            .discardingFiredPanes()
+            .withAllowedLateness(Duration.standardSeconds(configuration.occasionalDelaySec / 2)));
+    return biddersWindowed.apply(Count.<Long>perElement())
+        .apply(name + ".ToResult", ParDo.of(new DoFn<KV<Long, Long>, BidsPerSession>() {
+
+          @ProcessElement public void processElement(ProcessContext c) {
+            c.output(new BidsPerSession(c.element().getKey(), c.element().getValue()));
+          }
+        }));
+  }
+
+  @Override
+  protected PCollection<KnownSize> applyPrim(PCollection<Event> events) {
+    return NexmarkUtils.castToKnownSize(name, applyTyped(events));
+  }
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/f4333df7/sdks/java/nexmark/src/main/java/org/apache/beam/sdk/nexmark/queries/Query12.java
----------------------------------------------------------------------
diff --git a/sdks/java/nexmark/src/main/java/org/apache/beam/sdk/nexmark/queries/Query12.java b/sdks/java/nexmark/src/main/java/org/apache/beam/sdk/nexmark/queries/Query12.java
new file mode 100644
index 0000000..0f4b232
--- /dev/null
+++ b/sdks/java/nexmark/src/main/java/org/apache/beam/sdk/nexmark/queries/Query12.java
@@ -0,0 +1,80 @@
+/*
+ * 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.beam.sdk.nexmark.queries;
+
+import org.apache.beam.sdk.nexmark.NexmarkConfiguration;
+import org.apache.beam.sdk.nexmark.NexmarkUtils;
+import org.apache.beam.sdk.nexmark.model.Bid;
+import org.apache.beam.sdk.nexmark.model.BidsPerSession;
+import org.apache.beam.sdk.nexmark.model.Event;
+import org.apache.beam.sdk.nexmark.model.KnownSize;
+import org.apache.beam.sdk.transforms.Count;
+import org.apache.beam.sdk.transforms.DoFn;
+import org.apache.beam.sdk.transforms.ParDo;
+import org.apache.beam.sdk.transforms.windowing.AfterProcessingTime;
+import org.apache.beam.sdk.transforms.windowing.GlobalWindows;
+import org.apache.beam.sdk.transforms.windowing.Repeatedly;
+import org.apache.beam.sdk.transforms.windowing.Window;
+import org.apache.beam.sdk.values.KV;
+import org.apache.beam.sdk.values.PCollection;
+import org.joda.time.Duration;
+
+/**
+ * Query "12", 'Processing time windows' (Not in original suite.)
+ *
+ * <p>Group bids by the same user into processing time windows of windowSize. Emit the count
+ * of bids per window.
+ */
+public class Query12 extends NexmarkQuery {
+  public Query12(NexmarkConfiguration configuration) {
+    super(configuration, "Query12");
+  }
+
+  private PCollection<BidsPerSession> applyTyped(PCollection<Event> events) {
+    return events
+        .apply(JUST_BIDS)
+        .apply(ParDo.of(new DoFn<Bid, Long>() {
+          @ProcessElement
+          public void processElement(ProcessContext c){
+            c.output(c.element().bidder);
+          }
+        }))
+        .apply(Window.<Long>into(new GlobalWindows())
+            .triggering(
+                Repeatedly.forever(
+                    AfterProcessingTime.pastFirstElementInPane()
+                                       .plusDelayOf(
+                                           Duration.standardSeconds(configuration.windowSizeSec))))
+            .discardingFiredPanes()
+            .withAllowedLateness(Duration.ZERO))
+        .apply(Count.<Long>perElement())
+        .apply(name + ".ToResult",
+            ParDo.of(new DoFn<KV<Long, Long>, BidsPerSession>() {
+                   @ProcessElement
+                   public void processElement(ProcessContext c) {
+                     c.output(
+                         new BidsPerSession(c.element().getKey(), c.element().getValue()));
+                   }
+                 }));
+  }
+
+  @Override
+  protected PCollection<KnownSize> applyPrim(PCollection<Event> events) {
+    return NexmarkUtils.castToKnownSize(name, applyTyped(events));
+  }
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/f4333df7/sdks/java/nexmark/src/main/java/org/apache/beam/sdk/nexmark/queries/Query1Model.java
----------------------------------------------------------------------
diff --git a/sdks/java/nexmark/src/main/java/org/apache/beam/sdk/nexmark/queries/Query1Model.java b/sdks/java/nexmark/src/main/java/org/apache/beam/sdk/nexmark/queries/Query1Model.java
new file mode 100644
index 0000000..76c182a
--- /dev/null
+++ b/sdks/java/nexmark/src/main/java/org/apache/beam/sdk/nexmark/queries/Query1Model.java
@@ -0,0 +1,76 @@
+/*
+ * 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.beam.sdk.nexmark.queries;
+
+import java.io.Serializable;
+import java.util.Collection;
+import java.util.Iterator;
+
+import org.apache.beam.sdk.nexmark.NexmarkConfiguration;
+import org.apache.beam.sdk.nexmark.NexmarkUtils;
+import org.apache.beam.sdk.nexmark.model.Bid;
+import org.apache.beam.sdk.nexmark.model.Event;
+import org.apache.beam.sdk.values.TimestampedValue;
+
+/**
+ * A direct implementation of {@link Query1}.
+ */
+public class Query1Model extends NexmarkQueryModel implements Serializable {
+  /**
+   * Simulator for query 1.
+   */
+  private static class Simulator extends AbstractSimulator<Event, Bid> {
+    public Simulator(NexmarkConfiguration configuration) {
+      super(NexmarkUtils.standardEventIterator(configuration));
+    }
+
+    @Override
+    protected void run() {
+      TimestampedValue<Event> timestampedEvent = nextInput();
+      if (timestampedEvent == null) {
+        allDone();
+        return;
+      }
+      Event event = timestampedEvent.getValue();
+      if (event.bid == null) {
+        // Ignore non-bid events.
+        return;
+      }
+      Bid bid = event.bid;
+      Bid resultBid =
+          new Bid(bid.auction, bid.bidder, bid.price * 89 / 100, bid.dateTime, bid.extra);
+      TimestampedValue<Bid> result =
+          TimestampedValue.of(resultBid, timestampedEvent.getTimestamp());
+      addResult(result);
+    }
+  }
+
+  public Query1Model(NexmarkConfiguration configuration) {
+    super(configuration);
+  }
+
+  @Override
+  public AbstractSimulator<?, ?> simulator() {
+    return new Simulator(configuration);
+  }
+
+  @Override
+  protected <T> Collection<String> toCollection(Iterator<TimestampedValue<T>> itr) {
+    return toValueTimestampOrder(itr);
+  }
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/f4333df7/sdks/java/nexmark/src/main/java/org/apache/beam/sdk/nexmark/queries/Query2.java
----------------------------------------------------------------------
diff --git a/sdks/java/nexmark/src/main/java/org/apache/beam/sdk/nexmark/queries/Query2.java b/sdks/java/nexmark/src/main/java/org/apache/beam/sdk/nexmark/queries/Query2.java
new file mode 100644
index 0000000..c5ab992
--- /dev/null
+++ b/sdks/java/nexmark/src/main/java/org/apache/beam/sdk/nexmark/queries/Query2.java
@@ -0,0 +1,79 @@
+/*
+ * 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.beam.sdk.nexmark.queries;
+
+import org.apache.beam.sdk.nexmark.NexmarkConfiguration;
+import org.apache.beam.sdk.nexmark.NexmarkUtils;
+import org.apache.beam.sdk.nexmark.model.AuctionPrice;
+import org.apache.beam.sdk.nexmark.model.Bid;
+import org.apache.beam.sdk.nexmark.model.Event;
+import org.apache.beam.sdk.nexmark.model.KnownSize;
+import org.apache.beam.sdk.transforms.DoFn;
+import org.apache.beam.sdk.transforms.Filter;
+import org.apache.beam.sdk.transforms.ParDo;
+import org.apache.beam.sdk.transforms.SerializableFunction;
+import org.apache.beam.sdk.values.PCollection;
+
+/**
+ * Query 2, 'Filtering. Find bids with specific auction ids and show their bid price.
+ * In CQL syntax:
+ *
+ * <pre>
+ * SELECT Rstream(auction, price)
+ * FROM Bid [NOW]
+ * WHERE auction = 1007 OR auction = 1020 OR auction = 2001 OR auction = 2019 OR auction = 2087;
+ * </pre>
+ *
+ * <p>As written that query will only yield a few hundred results over event streams of
+ * arbitrary size. To make it more interesting we instead choose bids for every
+ * {@code auctionSkip}'th auction.
+ */
+public class Query2 extends NexmarkQuery {
+  public Query2(NexmarkConfiguration configuration) {
+    super(configuration, "Query2");
+  }
+
+  private PCollection<AuctionPrice> applyTyped(PCollection<Event> events) {
+    return events
+        // Only want the bid events.
+        .apply(JUST_BIDS)
+
+        // Select just the bids for the auctions we care about.
+        .apply(Filter.by(new SerializableFunction<Bid, Boolean>() {
+          @Override
+          public Boolean apply(Bid bid) {
+            return bid.auction % configuration.auctionSkip == 0;
+          }
+        }))
+
+        // Project just auction id and price.
+        .apply(name + ".Project",
+            ParDo.of(new DoFn<Bid, AuctionPrice>() {
+                  @ProcessElement
+                  public void processElement(ProcessContext c) {
+                    Bid bid = c.element();
+                    c.output(new AuctionPrice(bid.auction, bid.price));
+                  }
+                }));
+  }
+
+  @Override
+  protected PCollection<KnownSize> applyPrim(PCollection<Event> events) {
+    return NexmarkUtils.castToKnownSize(name, applyTyped(events));
+  }
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/f4333df7/sdks/java/nexmark/src/main/java/org/apache/beam/sdk/nexmark/queries/Query2Model.java
----------------------------------------------------------------------
diff --git a/sdks/java/nexmark/src/main/java/org/apache/beam/sdk/nexmark/queries/Query2Model.java b/sdks/java/nexmark/src/main/java/org/apache/beam/sdk/nexmark/queries/Query2Model.java
new file mode 100644
index 0000000..33a1f8d
--- /dev/null
+++ b/sdks/java/nexmark/src/main/java/org/apache/beam/sdk/nexmark/queries/Query2Model.java
@@ -0,0 +1,80 @@
+/*
+ * 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.beam.sdk.nexmark.queries;
+
+import java.io.Serializable;
+import java.util.Collection;
+import java.util.Iterator;
+
+import org.apache.beam.sdk.nexmark.NexmarkConfiguration;
+import org.apache.beam.sdk.nexmark.NexmarkUtils;
+import org.apache.beam.sdk.nexmark.model.AuctionPrice;
+import org.apache.beam.sdk.nexmark.model.Bid;
+import org.apache.beam.sdk.nexmark.model.Event;
+import org.apache.beam.sdk.values.TimestampedValue;
+
+/**
+ * A direct implementation of {@link Query2}.
+ */
+public class Query2Model extends NexmarkQueryModel implements Serializable {
+  /**
+   * Simulator for query 2.
+   */
+  private class Simulator extends AbstractSimulator<Event, AuctionPrice> {
+    public Simulator(NexmarkConfiguration configuration) {
+      super(NexmarkUtils.standardEventIterator(configuration));
+    }
+
+    @Override
+    protected void run() {
+      TimestampedValue<Event> timestampedEvent = nextInput();
+      if (timestampedEvent == null) {
+        allDone();
+        return;
+      }
+      Event event = timestampedEvent.getValue();
+      if (event.bid == null) {
+        // Ignore non bid events.
+        return;
+      }
+      Bid bid = event.bid;
+      if (bid.auction % configuration.auctionSkip != 0) {
+        // Ignore bids for auctions we don't care about.
+        return;
+      }
+      AuctionPrice auctionPrice = new AuctionPrice(bid.auction, bid.price);
+      TimestampedValue<AuctionPrice> result =
+          TimestampedValue.of(auctionPrice, timestampedEvent.getTimestamp());
+      addResult(result);
+    }
+  }
+
+  public Query2Model(NexmarkConfiguration configuration) {
+    super(configuration);
+  }
+
+  @Override
+  public AbstractSimulator<?, ?> simulator() {
+    return new Simulator(configuration);
+  }
+
+  @Override
+  protected <T> Collection<String> toCollection(Iterator<TimestampedValue<T>> itr) {
+    return toValueTimestampOrder(itr);
+  }
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/f4333df7/sdks/java/nexmark/src/main/java/org/apache/beam/sdk/nexmark/queries/Query3.java
----------------------------------------------------------------------
diff --git a/sdks/java/nexmark/src/main/java/org/apache/beam/sdk/nexmark/queries/Query3.java b/sdks/java/nexmark/src/main/java/org/apache/beam/sdk/nexmark/queries/Query3.java
new file mode 100644
index 0000000..6f8d72d
--- /dev/null
+++ b/sdks/java/nexmark/src/main/java/org/apache/beam/sdk/nexmark/queries/Query3.java
@@ -0,0 +1,301 @@
+/*
+ * 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.beam.sdk.nexmark.queries;
+
+import java.util.ArrayList;
+import java.util.List;
+import org.apache.beam.sdk.coders.ListCoder;
+import org.apache.beam.sdk.metrics.Counter;
+import org.apache.beam.sdk.metrics.Metrics;
+import org.apache.beam.sdk.nexmark.NexmarkConfiguration;
+import org.apache.beam.sdk.nexmark.NexmarkUtils;
+import org.apache.beam.sdk.nexmark.model.Auction;
+import org.apache.beam.sdk.nexmark.model.Event;
+import org.apache.beam.sdk.nexmark.model.KnownSize;
+import org.apache.beam.sdk.nexmark.model.NameCityStateId;
+import org.apache.beam.sdk.nexmark.model.Person;
+import org.apache.beam.sdk.state.StateSpec;
+import org.apache.beam.sdk.state.StateSpecs;
+import org.apache.beam.sdk.state.TimeDomain;
+import org.apache.beam.sdk.state.Timer;
+import org.apache.beam.sdk.state.TimerSpec;
+import org.apache.beam.sdk.state.TimerSpecs;
+import org.apache.beam.sdk.state.ValueState;
+import org.apache.beam.sdk.transforms.DoFn;
+import org.apache.beam.sdk.transforms.Filter;
+import org.apache.beam.sdk.transforms.ParDo;
+import org.apache.beam.sdk.transforms.SerializableFunction;
+import org.apache.beam.sdk.transforms.join.CoGbkResult;
+import org.apache.beam.sdk.transforms.join.CoGroupByKey;
+import org.apache.beam.sdk.transforms.join.KeyedPCollectionTuple;
+import org.apache.beam.sdk.transforms.windowing.AfterPane;
+import org.apache.beam.sdk.transforms.windowing.GlobalWindows;
+import org.apache.beam.sdk.transforms.windowing.Repeatedly;
+import org.apache.beam.sdk.transforms.windowing.Window;
+import org.apache.beam.sdk.values.KV;
+import org.apache.beam.sdk.values.PCollection;
+import org.joda.time.Duration;
+import org.joda.time.Instant;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Query 3, 'Local Item Suggestion'. Who is selling in OR, ID or CA in category 10, and for what
+ * auction ids? In CQL syntax:
+ *
+ * <pre>
+ * SELECT Istream(P.name, P.city, P.state, A.id)
+ * FROM Auction A [ROWS UNBOUNDED], Person P [ROWS UNBOUNDED]
+ * WHERE A.seller = P.id AND (P.state = `OR' OR P.state = `ID' OR P.state = `CA') AND A.category
+ * = 10;
+ * </pre>
+ *
+ * <p>We'll implement this query to allow 'new auction' events to come before the 'new person'
+ * events for the auction seller. Those auctions will be stored until the matching person is seen.
+ * Then all subsequent auctions for a person will use the stored person record.
+ *
+ * <p>A real system would use an external system to maintain the id-to-person association.
+ */
+public class Query3 extends NexmarkQuery {
+
+  private static final Logger LOG = LoggerFactory.getLogger(Query3.class);
+  private final JoinDoFn joinDoFn;
+
+  public Query3(NexmarkConfiguration configuration) {
+    super(configuration, "Query3");
+    joinDoFn = new JoinDoFn(name, configuration.maxAuctionsWaitingTime);
+  }
+
+  private PCollection<NameCityStateId> applyTyped(PCollection<Event> events) {
+    int numEventsInPane = 30;
+
+    PCollection<Event> eventsWindowed =
+        events.apply(
+            Window.<Event>into(new GlobalWindows())
+                .triggering(Repeatedly.forever((AfterPane.elementCountAtLeast(numEventsInPane))))
+                .discardingFiredPanes()
+                .withAllowedLateness(Duration.ZERO));
+    PCollection<KV<Long, Auction>> auctionsBySellerId =
+        eventsWindowed
+            // Only want the new auction events.
+            .apply(JUST_NEW_AUCTIONS)
+
+            // We only want auctions in category 10.
+            .apply(
+                name + ".InCategory",
+                Filter.by(
+                    new SerializableFunction<Auction, Boolean>() {
+
+                      @Override
+                      public Boolean apply(Auction auction) {
+                        return auction.category == 10;
+                      }
+                    }))
+
+            // Key auctions by their seller id.
+            .apply("AuctionBySeller", AUCTION_BY_SELLER);
+
+    PCollection<KV<Long, Person>> personsById =
+        eventsWindowed
+            // Only want the new people events.
+            .apply(JUST_NEW_PERSONS)
+
+            // We only want people in OR, ID, CA.
+            .apply(
+                name + ".InState",
+                Filter.by(
+                    new SerializableFunction<Person, Boolean>() {
+
+                      @Override
+                      public Boolean apply(Person person) {
+                        return person.state.equals("OR")
+                            || person.state.equals("ID")
+                            || person.state.equals("CA");
+                      }
+                    }))
+
+            // Key people by their id.
+            .apply("PersonById", PERSON_BY_ID);
+
+    return
+    // Join auctions and people.
+    // concatenate KeyedPCollections
+    KeyedPCollectionTuple.of(AUCTION_TAG, auctionsBySellerId)
+        .and(PERSON_TAG, personsById)
+        // group auctions and persons by personId
+        .apply(CoGroupByKey.<Long>create())
+        .apply(name + ".Join", ParDo.of(joinDoFn))
+
+        // Project what we want.
+        .apply(
+            name + ".Project",
+            ParDo.of(
+                new DoFn<KV<Auction, Person>, NameCityStateId>() {
+
+                  @ProcessElement
+                  public void processElement(ProcessContext c) {
+                    Auction auction = c.element().getKey();
+                    Person person = c.element().getValue();
+                    c.output(
+                        new NameCityStateId(person.name, person.city, person.state, auction.id));
+                  }
+                }));
+  }
+
+  @Override
+  protected PCollection<KnownSize> applyPrim(PCollection<Event> events) {
+    return NexmarkUtils.castToKnownSize(name, applyTyped(events));
+  }
+
+  /**
+   * Join {@code auctions} and {@code people} by person id and emit their cross-product one pair at
+   * a time.
+   *
+   * <p>We know a person may submit any number of auctions. Thus new person event must have the
+   * person record stored in persistent state in order to match future auctions by that person.
+   *
+   * <p>However we know that each auction is associated with at most one person, so only need to
+   * store auction records in persistent state until we have seen the corresponding person record.
+   * And of course may have already seen that record.
+   */
+  private static class JoinDoFn extends DoFn<KV<Long, CoGbkResult>, KV<Auction, Person>> {
+
+    private final int maxAuctionsWaitingTime;
+    private static final String AUCTIONS = "auctions";
+    private static final String PERSON = "person";
+
+    @StateId(PERSON)
+    private static final StateSpec<ValueState<Person>> personSpec =
+        StateSpecs.value(Person.CODER);
+
+    private static final String PERSON_STATE_EXPIRING = "personStateExpiring";
+
+    @StateId(AUCTIONS)
+    private final StateSpec<ValueState<List<Auction>>> auctionsSpec =
+        StateSpecs.value(ListCoder.of(Auction.CODER));
+
+    @TimerId(PERSON_STATE_EXPIRING)
+    private final TimerSpec timerSpec = TimerSpecs.timer(TimeDomain.EVENT_TIME);
+
+    // Used to refer the metrics namespace
+    private final String name;
+
+    private final Counter newAuctionCounter;
+    private final Counter newPersonCounter;
+    private final Counter newNewOutputCounter;
+    private final Counter newOldOutputCounter;
+    private final Counter oldNewOutputCounter;
+    private final Counter fatalCounter;
+
+    private JoinDoFn(String name, int maxAuctionsWaitingTime) {
+      this.name = name;
+      this.maxAuctionsWaitingTime = maxAuctionsWaitingTime;
+      newAuctionCounter = Metrics.counter(name, "newAuction");
+      newPersonCounter = Metrics.counter(name, "newPerson");
+      newNewOutputCounter = Metrics.counter(name, "newNewOutput");
+      newOldOutputCounter = Metrics.counter(name, "newOldOutput");
+      oldNewOutputCounter = Metrics.counter(name, "oldNewOutput");
+      fatalCounter = Metrics.counter(name , "fatal");
+    }
+
+    @ProcessElement
+    public void processElement(
+        ProcessContext c,
+        @TimerId(PERSON_STATE_EXPIRING) Timer timer,
+        @StateId(PERSON) ValueState<Person> personState,
+        @StateId(AUCTIONS) ValueState<List<Auction>> auctionsState) {
+      // We would *almost* implement this by  rewindowing into the global window and
+      // running a combiner over the result. The combiner's accumulator would be the
+      // state we use below. However, combiners cannot emit intermediate results, thus
+      // we need to wait for the pending ReduceFn API.
+
+      Person existingPerson = personState.read();
+      if (existingPerson != null) {
+        // We've already seen the new person event for this person id.
+        // We can join with any new auctions on-the-fly without needing any
+        // additional persistent state.
+        for (Auction newAuction : c.element().getValue().getAll(AUCTION_TAG)) {
+          newAuctionCounter.inc();
+          newOldOutputCounter.inc();
+          c.output(KV.of(newAuction, existingPerson));
+        }
+        return;
+      }
+
+      Person theNewPerson = null;
+      for (Person newPerson : c.element().getValue().getAll(PERSON_TAG)) {
+        if (theNewPerson == null) {
+          theNewPerson = newPerson;
+        } else {
+          if (theNewPerson.equals(newPerson)) {
+            LOG.error("Duplicate person {}", theNewPerson);
+          } else {
+            LOG.error("Conflicting persons {} and {}", theNewPerson, newPerson);
+          }
+          fatalCounter.inc();
+          continue;
+        }
+        newPersonCounter.inc();
+        // We've now seen the person for this person id so can flush any
+        // pending auctions for the same seller id (an auction is done by only one seller).
+        List<Auction> pendingAuctions = auctionsState.read();
+        if (pendingAuctions != null) {
+          for (Auction pendingAuction : pendingAuctions) {
+            oldNewOutputCounter.inc();
+            c.output(KV.of(pendingAuction, newPerson));
+          }
+          auctionsState.clear();
+        }
+        // Also deal with any new auctions.
+        for (Auction newAuction : c.element().getValue().getAll(AUCTION_TAG)) {
+          newAuctionCounter.inc();
+          newNewOutputCounter.inc();
+          c.output(KV.of(newAuction, newPerson));
+        }
+        // Remember this person for any future auctions.
+        personState.write(newPerson);
+        //set a time out to clear this state
+        Instant firingTime = new Instant(newPerson.dateTime)
+                                  .plus(Duration.standardSeconds(maxAuctionsWaitingTime));
+        timer.set(firingTime);
+      }
+      if (theNewPerson != null) {
+        return;
+      }
+
+      // We'll need to remember the auctions until we see the corresponding
+      // new person event.
+      List<Auction> pendingAuctions = auctionsState.read();
+      if (pendingAuctions == null) {
+        pendingAuctions = new ArrayList<>();
+      }
+      for (Auction newAuction : c.element().getValue().getAll(AUCTION_TAG)) {
+        newAuctionCounter.inc();
+        pendingAuctions.add(newAuction);
+      }
+      auctionsState.write(pendingAuctions);
+    }
+
+    @OnTimer(PERSON_STATE_EXPIRING)
+    public void onTimerCallback(
+        OnTimerContext context,
+        @StateId(PERSON) ValueState<Person> personState) {
+        personState.clear();
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/f4333df7/sdks/java/nexmark/src/main/java/org/apache/beam/sdk/nexmark/queries/Query3Model.java
----------------------------------------------------------------------
diff --git a/sdks/java/nexmark/src/main/java/org/apache/beam/sdk/nexmark/queries/Query3Model.java b/sdks/java/nexmark/src/main/java/org/apache/beam/sdk/nexmark/queries/Query3Model.java
new file mode 100644
index 0000000..94f24cb
--- /dev/null
+++ b/sdks/java/nexmark/src/main/java/org/apache/beam/sdk/nexmark/queries/Query3Model.java
@@ -0,0 +1,124 @@
+/*
+ * 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.beam.sdk.nexmark.queries;
+
+import com.google.common.collect.ArrayListMultimap;
+import com.google.common.collect.Multimap;
+
+import java.io.Serializable;
+import java.util.Collection;
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.Map;
+
+import org.apache.beam.sdk.nexmark.NexmarkConfiguration;
+import org.apache.beam.sdk.nexmark.NexmarkUtils;
+import org.apache.beam.sdk.nexmark.model.Auction;
+import org.apache.beam.sdk.nexmark.model.Event;
+import org.apache.beam.sdk.nexmark.model.NameCityStateId;
+import org.apache.beam.sdk.nexmark.model.Person;
+import org.apache.beam.sdk.values.TimestampedValue;
+import org.joda.time.Instant;
+
+/**
+ * A direct implementation of {@link Query3}.
+ */
+public class Query3Model extends NexmarkQueryModel implements Serializable {
+  /**
+   * Simulator for query 3.
+   */
+  private static class Simulator extends AbstractSimulator<Event, NameCityStateId> {
+    /** Auctions, indexed by seller id. */
+    private final Multimap<Long, Auction> newAuctions;
+
+    /** Persons, indexed by id. */
+    private final Map<Long, Person> newPersons;
+
+    public Simulator(NexmarkConfiguration configuration) {
+      super(NexmarkUtils.standardEventIterator(configuration));
+      newPersons = new HashMap<>();
+      newAuctions = ArrayListMultimap.create();
+    }
+
+    /**
+     * Capture new result.
+     */
+    private void addResult(Auction auction, Person person, Instant timestamp) {
+      TimestampedValue<NameCityStateId> result = TimestampedValue.of(
+          new NameCityStateId(person.name, person.city, person.state, auction.id), timestamp);
+      addResult(result);
+    }
+
+    @Override
+    protected void run() {
+      TimestampedValue<Event> timestampedEvent = nextInput();
+      if (timestampedEvent == null) {
+        allDone();
+        return;
+      }
+      Event event = timestampedEvent.getValue();
+      if (event.bid != null) {
+        // Ignore bid events.
+        return;
+      }
+
+      Instant timestamp = timestampedEvent.getTimestamp();
+
+      if (event.newAuction != null) {
+        // Only want auctions in category 10.
+        if (event.newAuction.category == 10) {
+          // Join new auction with existing person, if any.
+          Person person = newPersons.get(event.newAuction.seller);
+          if (person != null) {
+            addResult(event.newAuction, person, timestamp);
+          } else {
+            // Remember auction for future new person event.
+            newAuctions.put(event.newAuction.seller, event.newAuction);
+          }
+        }
+      } else {
+        // Only want people in OR, ID or CA.
+        if (event.newPerson.state.equals("OR") || event.newPerson.state.equals("ID")
+            || event.newPerson.state.equals("CA")) {
+          // Join new person with existing auctions.
+          for (Auction auction : newAuctions.get(event.newPerson.id)) {
+            addResult(auction, event.newPerson, timestamp);
+          }
+          // We'll never need these auctions again.
+          newAuctions.removeAll(event.newPerson.id);
+          // Remember person for future auctions.
+          newPersons.put(event.newPerson.id, event.newPerson);
+        }
+      }
+    }
+  }
+
+  public Query3Model(NexmarkConfiguration configuration) {
+    super(configuration);
+  }
+
+  @Override
+  public AbstractSimulator<?, ?> simulator() {
+    return new Simulator(configuration);
+  }
+
+  @Override
+  protected <T> Collection<String> toCollection(Iterator<TimestampedValue<T>> itr) {
+    return toValue(itr);
+  }
+}


[04/55] [abbrv] beam git commit: NexMark

Posted by ie...@apache.org.
http://git-wip-us.apache.org/repos/asf/beam/blob/1f08970a/integration/java/src/main/java/org/apache/beam/integration/nexmark/NexmarkGoogleDriver.java
----------------------------------------------------------------------
diff --git a/integration/java/src/main/java/org/apache/beam/integration/nexmark/NexmarkGoogleDriver.java b/integration/java/src/main/java/org/apache/beam/integration/nexmark/NexmarkGoogleDriver.java
new file mode 100644
index 0000000..2534155
--- /dev/null
+++ b/integration/java/src/main/java/org/apache/beam/integration/nexmark/NexmarkGoogleDriver.java
@@ -0,0 +1,90 @@
+/*
+ * 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.beam.integration.nexmark;
+
+import org.apache.beam.runners.dataflow.DataflowPipelineRunner;
+import org.apache.beam.runners.dataflow.options.DataflowPipelineOptions;
+import org.apache.beam.sdk.options.Description;
+import org.apache.beam.sdk.options.PipelineOptionsFactory;
+import javax.annotation.Nullable;
+
+/**
+ * An implementation of the 'NEXMark queries' for Google Dataflow.
+ * These are 11 queries over a three table schema representing on online auction system:
+ * <ul>
+ * <li>{@link Person} represents a person submitting an item for auction and/or making a bid
+ * on an auction.
+ * <li>{@link Auction} represents an item under auction.
+ * <li>{@link Bid} represents a bid for an item under auction.
+ * </ul>
+ * The queries exercise many aspects of streaming dataflow.
+ * <p>
+ * <p>We synthesize the creation of people, auctions and bids in real-time. The data is not
+ * particularly sensible.
+ * <p>
+ * <p>See
+ * <a href="http://datalab.cs.pdx.edu/niagaraST/NEXMark/">
+ * http://datalab.cs.pdx.edu/niagaraST/NEXMark/</a>
+ */
+class NexmarkGoogleDriver extends NexmarkDriver<NexmarkGoogleDriver.NexmarkGoogleOptions> {
+  /**
+   * Command line flags.
+   */
+  public interface NexmarkGoogleOptions extends Options, DataflowPipelineOptions {
+    @Description("If set, cancel running pipelines after this long")
+    @Nullable
+    Long getRunningTimeMinutes();
+
+    void setRunningTimeMinutes(Long value);
+
+    @Description("If set and --monitorJobs is true, check that the system watermark is never more "
+                 + "than this far behind real time")
+    @Nullable
+    Long getMaxSystemLagSeconds();
+
+    void setMaxSystemLagSeconds(Long value);
+
+    @Description("If set and --monitorJobs is true, check that the data watermark is never more "
+                 + "than this far behind real time")
+    @Nullable
+    Long getMaxDataLagSeconds();
+
+    void setMaxDataLagSeconds(Long value);
+
+    @Description("Only start validating watermarks after this many seconds")
+    @Nullable
+    Long getWatermarkValidationDelaySeconds();
+
+    void setWatermarkValidationDelaySeconds(Long value);
+  }
+
+  /**
+   * Entry point.
+   */
+  public static void main(String[] args) {
+    // Gather command line args, baseline, configurations, etc.
+    NexmarkGoogleOptions options = PipelineOptionsFactory.fromArgs(args)
+                                                         .withValidation()
+                                                         .as(NexmarkGoogleOptions.class);
+    options.setRunner(DataflowPipelineRunner.class);
+    NexmarkGoogleRunner runner = new NexmarkGoogleRunner(options);
+    new NexmarkGoogleDriver().runAll(options, runner);
+  }
+}
+

http://git-wip-us.apache.org/repos/asf/beam/blob/1f08970a/integration/java/src/main/java/org/apache/beam/integration/nexmark/NexmarkGoogleRunner.java
----------------------------------------------------------------------
diff --git a/integration/java/src/main/java/org/apache/beam/integration/nexmark/NexmarkGoogleRunner.java b/integration/java/src/main/java/org/apache/beam/integration/nexmark/NexmarkGoogleRunner.java
new file mode 100644
index 0000000..4b73592
--- /dev/null
+++ b/integration/java/src/main/java/org/apache/beam/integration/nexmark/NexmarkGoogleRunner.java
@@ -0,0 +1,660 @@
+/*
+ * 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.beam.integration.nexmark;
+
+import org.apache.beam.runners.dataflow.DataflowPipelineJob;
+import org.apache.beam.sdk.PipelineResult;
+import org.apache.beam.sdk.runners.AggregatorRetrievalException;
+import org.apache.beam.sdk.transforms.Aggregator;
+
+import com.google.api.services.dataflow.model.JobMetrics;
+import com.google.api.services.dataflow.model.MetricUpdate;
+import com.google.common.base.Preconditions;
+import com.google.common.collect.Iterables;
+
+import org.joda.time.Duration;
+import org.joda.time.Instant;
+
+import java.io.IOException;
+import java.math.BigDecimal;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.List;
+import java.util.concurrent.TimeUnit;
+import javax.annotation.Nullable;
+
+/**
+ * Run a singe Nexmark query using a given configuration on Google Dataflow.
+ */
+class NexmarkGoogleRunner extends NexmarkRunner<NexmarkGoogleDriver.NexmarkGoogleOptions> {
+  /**
+   * How long to let streaming pipeline run after all events have been generated and we've
+   * seen no activity.
+   */
+  private static final Duration DONE_DELAY = Duration.standardMinutes(1);
+
+  /**
+   * How long to allow no activity without warning.
+   */
+  private static final Duration STUCK_WARNING_DELAY = Duration.standardMinutes(10);
+
+  /**
+   * How long to let streaming pipeline run after we've
+   * seen no activity, even if all events have not been generated.
+   */
+  private static final Duration STUCK_TERMINATE_DELAY = Duration.standardDays(3);
+
+  /**
+   * Delay between perf samples.
+   */
+  private static final Duration PERF_DELAY = Duration.standardSeconds(15);
+
+  /**
+   * Minimum number of samples needed for 'stead-state' rate calculation.
+   */
+  private static final int MIN_SAMPLES = 9;
+
+  /**
+   * Minimum length of time over which to consider samples for 'steady-state' rate calculation.
+   */
+  private static final Duration MIN_WINDOW = Duration.standardMinutes(2);
+
+  public NexmarkGoogleRunner(NexmarkGoogleDriver.NexmarkGoogleOptions options) {
+    super(options);
+  }
+
+  @Override
+  protected boolean isStreaming() {
+    return options.isStreaming();
+  }
+
+  @Override
+  protected int coresPerWorker() {
+    String machineType = options.getWorkerMachineType();
+    if (machineType == null || machineType.isEmpty()) {
+      return 1;
+    }
+    String[] split = machineType.split("-");
+    if (split.length != 3) {
+      return 1;
+    }
+    try {
+      return Integer.parseInt(split[2]);
+    } catch (NumberFormatException ex) {
+      return 1;
+    }
+  }
+
+  @Override
+  protected int maxNumWorkers() {
+    return Math.max(options.getNumWorkers(), options.getMaxNumWorkers());
+  }
+
+  @Override
+  protected boolean canMonitor() {
+    return true;
+  }
+
+  @Override
+  protected void invokeBuilderForPublishOnlyPipeline(PipelineBuilder builder) {
+    String jobName = options.getJobName();
+    String appName = options.getAppName();
+    options.setJobName("p-" + jobName);
+    options.setAppName("p-" + appName);
+    int coresPerWorker = coresPerWorker();
+    int eventGeneratorWorkers = (configuration.numEventGenerators + coresPerWorker - 1)
+                                / coresPerWorker;
+    options.setMaxNumWorkers(Math.min(options.getMaxNumWorkers(), eventGeneratorWorkers));
+    options.setNumWorkers(Math.min(options.getNumWorkers(), eventGeneratorWorkers));
+    publisherMonitor = new Monitor<Event>(queryName, "publisher");
+    try {
+      builder.build(options);
+    } finally {
+      options.setJobName(jobName);
+      options.setAppName(appName);
+      options.setMaxNumWorkers(options.getMaxNumWorkers());
+      options.setNumWorkers(options.getNumWorkers());
+    }
+  }
+
+  /**
+   * Monitor the progress of the publisher job. Return when it has been generating events for
+   * at least {@code configuration.preloadSeconds}.
+   */
+  @Override
+  protected void waitForPublisherPreload() {
+    Preconditions.checkNotNull(publisherMonitor);
+    Preconditions.checkNotNull(publisherResult);
+    if (!options.getMonitorJobs()) {
+      return;
+    }
+    if (!(publisherResult instanceof DataflowPipelineJob)) {
+      return;
+    }
+    if (configuration.preloadSeconds <= 0) {
+      return;
+    }
+
+    NexmarkUtils.console("waiting for publisher to pre-load");
+
+    DataflowPipelineJob job = (DataflowPipelineJob) publisherResult;
+
+    long numEvents = 0;
+    long startMsSinceEpoch = -1;
+    long endMsSinceEpoch = -1;
+    while (true) {
+      PipelineResult.State state = job.getState();
+      switch (state) {
+        case UNKNOWN:
+          // Keep waiting.
+          NexmarkUtils.console("%s publisher (%d events)", state, numEvents);
+          break;
+        case STOPPED:
+        case DONE:
+        case CANCELLED:
+        case FAILED:
+        case UPDATED:
+          NexmarkUtils.console("%s publisher (%d events)", state, numEvents);
+          return;
+        case RUNNING:
+          numEvents = getLong(job, publisherMonitor.getElementCounter());
+          if (startMsSinceEpoch < 0 && numEvents > 0) {
+            startMsSinceEpoch = System.currentTimeMillis();
+            endMsSinceEpoch = startMsSinceEpoch
+                              + Duration.standardSeconds(configuration.preloadSeconds).getMillis();
+          }
+          if (endMsSinceEpoch < 0) {
+            NexmarkUtils.console("%s publisher (%d events)", state, numEvents);
+          } else {
+            long remainMs = endMsSinceEpoch - System.currentTimeMillis();
+            if (remainMs > 0) {
+              NexmarkUtils.console("%s publisher (%d events, waiting for %ds)", state, numEvents,
+                  remainMs / 1000);
+            } else {
+              NexmarkUtils.console("publisher preloaded %d events", numEvents);
+              return;
+            }
+          }
+          break;
+      }
+
+      try {
+        Thread.sleep(PERF_DELAY.getMillis());
+      } catch (InterruptedException e) {
+        Thread.interrupted();
+        throw new RuntimeException("Interrupted: publisher still running.");
+      }
+    }
+  }
+
+  /**
+   * Monitor the performance and progress of a running job. Return final performance if
+   * it was measured.
+   */
+  @Override
+  @Nullable
+  protected NexmarkPerf monitor(NexmarkQuery query) {
+    if (!options.getMonitorJobs()) {
+      return null;
+    }
+    if (!(mainResult instanceof DataflowPipelineJob)) {
+      return null;
+    }
+
+    if (configuration.debug) {
+      NexmarkUtils.console("Waiting for main pipeline to 'finish'");
+    } else {
+      NexmarkUtils.console("--debug=false, so job will not self-cancel");
+    }
+
+    DataflowPipelineJob job = (DataflowPipelineJob) mainResult;
+    DataflowPipelineJob publisherJob = (DataflowPipelineJob) publisherResult;
+    List<NexmarkPerf.ProgressSnapshot> snapshots = new ArrayList<>();
+    long startMsSinceEpoch = System.currentTimeMillis();
+    long endMsSinceEpoch = -1;
+    if (options.getRunningTimeMinutes() != null) {
+      endMsSinceEpoch = startMsSinceEpoch
+                        + Duration.standardMinutes(options.getRunningTimeMinutes()).getMillis()
+                        - Duration.standardSeconds(configuration.preloadSeconds).getMillis();
+    }
+    long lastActivityMsSinceEpoch = -1;
+    NexmarkPerf perf = null;
+    boolean waitingForShutdown = false;
+    boolean publisherCancelled = false;
+    List<String> errors = new ArrayList<>();
+
+    while (true) {
+      long now = System.currentTimeMillis();
+      if (endMsSinceEpoch >= 0 && now > endMsSinceEpoch && !waitingForShutdown) {
+        NexmarkUtils.console("Reached end of test, cancelling job");
+        try {
+          job.cancel();
+        } catch (IOException e) {
+          throw new RuntimeException("Unable to cancel main job: ", e);
+        }
+        if (publisherResult != null) {
+          try {
+            publisherJob.cancel();
+          } catch (IOException e) {
+            throw new RuntimeException("Unable to cancel publisher job: ", e);
+          }
+          publisherCancelled = true;
+        }
+        waitingForShutdown = true;
+      }
+
+      PipelineResult.State state = job.getState();
+      NexmarkUtils.console("%s %s%s", state, queryName,
+          waitingForShutdown ? " (waiting for shutdown)" : "");
+
+      NexmarkPerf currPerf;
+      if (configuration.debug) {
+        currPerf = currentPerf(startMsSinceEpoch, now, job, snapshots,
+                               query.eventMonitor, query.resultMonitor);
+      } else {
+        currPerf = null;
+      }
+
+      if (perf == null || perf.anyActivity(currPerf)) {
+        lastActivityMsSinceEpoch = now;
+      }
+
+      if (options.isStreaming() && !waitingForShutdown) {
+        Duration quietFor = new Duration(lastActivityMsSinceEpoch, now);
+        if (query.getFatalCount() != null && getLong(job, query.getFatalCount()) > 0) {
+          NexmarkUtils.console("job has fatal errors, cancelling.");
+          errors.add(String.format("Pipeline reported %s fatal errors", query.getFatalCount()));
+          waitingForShutdown = true;
+        } else if (configuration.debug && configuration.numEvents > 0
+                   && currPerf.numEvents == configuration.numEvents
+                   && currPerf.numResults >= 0 && quietFor.isLongerThan(DONE_DELAY)) {
+          NexmarkUtils.console("streaming query appears to have finished, cancelling job.");
+          waitingForShutdown = true;
+        } else if (quietFor.isLongerThan(STUCK_TERMINATE_DELAY)) {
+          NexmarkUtils.console("streaming query appears to have gotten stuck, cancelling job.");
+          errors.add("Streaming job was cancelled since appeared stuck");
+          waitingForShutdown = true;
+        } else if (quietFor.isLongerThan(STUCK_WARNING_DELAY)) {
+          NexmarkUtils.console("WARNING: streaming query appears to have been stuck for %d min.",
+              quietFor.getStandardMinutes());
+          errors.add(
+              String.format("Streaming query was stuck for %d min", quietFor.getStandardMinutes()));
+        }
+
+        errors.addAll(checkWatermarks(job, startMsSinceEpoch));
+
+        if (waitingForShutdown) {
+          try {
+            job.cancel();
+          } catch (IOException e) {
+            throw new RuntimeException("Unable to cancel main job: ", e);
+          }
+        }
+      }
+
+      perf = currPerf;
+
+      boolean running = true;
+      switch (state) {
+        case UNKNOWN:
+        case STOPPED:
+        case RUNNING:
+          // Keep going.
+          break;
+        case DONE:
+          // All done.
+          running = false;
+          break;
+        case CANCELLED:
+          running = false;
+          if (!waitingForShutdown) {
+            errors.add("Job was unexpectedly cancelled");
+          }
+          break;
+        case FAILED:
+        case UPDATED:
+          // Abnormal termination.
+          running = false;
+          errors.add("Job was unexpectedly updated");
+          break;
+      }
+
+      if (!running) {
+        break;
+      }
+
+      if (lastActivityMsSinceEpoch == now) {
+        NexmarkUtils.console("new perf %s", perf);
+      } else {
+        NexmarkUtils.console("no activity");
+      }
+
+      try {
+        Thread.sleep(PERF_DELAY.getMillis());
+      } catch (InterruptedException e) {
+        Thread.interrupted();
+        NexmarkUtils.console("Interrupted: pipeline is still running");
+      }
+    }
+
+    perf.errors = errors;
+    perf.snapshots = snapshots;
+
+    if (publisherResult != null) {
+      NexmarkUtils.console("Shutting down publisher pipeline.");
+      try {
+        if (!publisherCancelled) {
+          publisherJob.cancel();
+        }
+        publisherJob.waitToFinish(5, TimeUnit.MINUTES, null);
+      } catch (IOException e) {
+        throw new RuntimeException("Unable to cancel publisher job: ", e);
+      } catch (InterruptedException e) {
+        Thread.interrupted();
+        throw new RuntimeException("Interrupted: publish job still running.", e);
+      }
+    }
+
+    return perf;
+  }
+
+  enum MetricType {
+    SYSTEM_WATERMARK,
+    DATA_WATERMARK,
+    OTHER
+  }
+
+  private MetricType getMetricType(MetricUpdate metric) {
+    String metricName = metric.getName().getName();
+    if (metricName.endsWith("windmill-system-watermark")) {
+      return MetricType.SYSTEM_WATERMARK;
+    } else if (metricName.endsWith("windmill-data-watermark")) {
+      return MetricType.DATA_WATERMARK;
+    } else {
+      return MetricType.OTHER;
+    }
+  }
+
+  /**
+   * Check that watermarks are not too far behind.
+   * <p>
+   * <p>Returns a list of errors detected.
+   */
+  private List<String> checkWatermarks(DataflowPipelineJob job, long startMsSinceEpoch) {
+    long now = System.currentTimeMillis();
+    List<String> errors = new ArrayList<>();
+    try {
+      JobMetrics metricResponse = job.getDataflowClient()
+                                     .projects()
+                                     .jobs()
+                                     .getMetrics(job.getProjectId(), job.getJobId())
+                                     .execute();
+      List<MetricUpdate> metrics = metricResponse.getMetrics();
+      if (metrics != null) {
+        boolean foundWatermarks = false;
+        for (MetricUpdate metric : metrics) {
+          MetricType type = getMetricType(metric);
+          if (type == MetricType.OTHER) {
+            continue;
+          }
+          foundWatermarks = true;
+          @SuppressWarnings("unchecked")
+          BigDecimal scalar = (BigDecimal) metric.getScalar();
+          if (scalar.signum() < 0) {
+            continue;
+          }
+          Instant value =
+              new Instant(scalar.divideToIntegralValue(new BigDecimal(1000)).longValueExact());
+          Instant updateTime = Instant.parse(metric.getUpdateTime());
+
+          if (options.getWatermarkValidationDelaySeconds() == null
+              || now > startMsSinceEpoch
+                       + Duration.standardSeconds(options.getWatermarkValidationDelaySeconds())
+                                 .getMillis()) {
+            Duration threshold = null;
+            if (type == MetricType.SYSTEM_WATERMARK && options.getMaxSystemLagSeconds() != null) {
+              threshold = Duration.standardSeconds(options.getMaxSystemLagSeconds());
+            } else if (type == MetricType.DATA_WATERMARK
+                       && options.getMaxDataLagSeconds() != null) {
+              threshold = Duration.standardSeconds(options.getMaxDataLagSeconds());
+            }
+
+            if (threshold != null && value.isBefore(updateTime.minus(threshold))) {
+              String msg = String.format("High lag for %s: %s vs %s (allowed lag of %s)",
+                  metric.getName().getName(), value, updateTime, threshold);
+              errors.add(msg);
+              NexmarkUtils.console(msg);
+            }
+          }
+        }
+        if (!foundWatermarks) {
+          NexmarkUtils.console("No known watermarks in update: " + metrics);
+          if (now > startMsSinceEpoch + Duration.standardMinutes(5).getMillis()) {
+            errors.add("No known watermarks found.  Metrics were " + metrics);
+          }
+        }
+      }
+    } catch (IOException e) {
+      NexmarkUtils.console("Warning: failed to get JobMetrics: " + e);
+    }
+
+    return errors;
+  }
+
+  /**
+   * Return the current performance given {@code eventMonitor} and {@code resultMonitor}.
+   */
+  private NexmarkPerf currentPerf(
+      long startMsSinceEpoch, long now, DataflowPipelineJob job,
+      List<NexmarkPerf.ProgressSnapshot> snapshots, Monitor<?> eventMonitor,
+      Monitor<?> resultMonitor) {
+    NexmarkPerf perf = new NexmarkPerf();
+
+    long numEvents = getLong(job, eventMonitor.getElementCounter());
+    long numEventBytes = getLong(job, eventMonitor.getBytesCounter());
+    long eventStart = getTimestamp(now, job, eventMonitor.getStartTime());
+    long eventEnd = getTimestamp(now, job, eventMonitor.getEndTime());
+    long numResults = getLong(job, resultMonitor.getElementCounter());
+    long numResultBytes = getLong(job, resultMonitor.getBytesCounter());
+    long resultStart = getTimestamp(now, job, resultMonitor.getStartTime());
+    long resultEnd = getTimestamp(now, job, resultMonitor.getEndTime());
+    long timestampStart = getTimestamp(now, job, resultMonitor.getStartTimestamp());
+    long timestampEnd = getTimestamp(now, job, resultMonitor.getEndTimestamp());
+
+    long effectiveEnd = -1;
+    if (eventEnd >= 0 && resultEnd >= 0) {
+      // It is possible for events to be generated after the last result was emitted.
+      // (Eg Query 2, which only yields results for a small prefix of the event stream.)
+      // So use the max of last event and last result times.
+      effectiveEnd = Math.max(eventEnd, resultEnd);
+    } else if (resultEnd >= 0) {
+      effectiveEnd = resultEnd;
+    } else if (eventEnd >= 0) {
+      // During startup we may have no result yet, but we would still like to track how
+      // long the pipeline has been running.
+      effectiveEnd = eventEnd;
+    }
+
+    if (effectiveEnd >= 0 && eventStart >= 0 && effectiveEnd >= eventStart) {
+      perf.runtimeSec = (effectiveEnd - eventStart) / 1000.0;
+    }
+
+    if (numEvents >= 0) {
+      perf.numEvents = numEvents;
+    }
+
+    if (numEvents >= 0 && perf.runtimeSec > 0.0) {
+      // For streaming we may later replace this with a 'steady-state' value calculated
+      // from the progress snapshots.
+      perf.eventsPerSec = numEvents / perf.runtimeSec;
+    }
+
+    if (numEventBytes >= 0 && perf.runtimeSec > 0.0) {
+      perf.eventBytesPerSec = numEventBytes / perf.runtimeSec;
+    }
+
+    if (numResults >= 0) {
+      perf.numResults = numResults;
+    }
+
+    if (numResults >= 0 && perf.runtimeSec > 0.0) {
+      perf.resultsPerSec = numResults / perf.runtimeSec;
+    }
+
+    if (numResultBytes >= 0 && perf.runtimeSec > 0.0) {
+      perf.resultBytesPerSec = numResultBytes / perf.runtimeSec;
+    }
+
+    if (eventStart >= 0) {
+      perf.startupDelaySec = (eventStart - startMsSinceEpoch) / 1000.0;
+    }
+
+    if (resultStart >= 0 && eventStart >= 0 && resultStart >= eventStart) {
+      perf.processingDelaySec = (resultStart - eventStart) / 1000.0;
+    }
+
+    if (timestampStart >= 0 && timestampEnd >= 0 && perf.runtimeSec > 0.0) {
+      double eventRuntimeSec = (timestampEnd - timestampStart) / 1000.0;
+      perf.timeDilation = eventRuntimeSec / perf.runtimeSec;
+    }
+
+    if (resultEnd >= 0) {
+      // Fill in the shutdown delay assuming the job has now finished.
+      perf.shutdownDelaySec = (now - resultEnd) / 1000.0;
+    }
+
+    perf.jobId = job.getJobId();
+    // As soon as available, try to capture cumulative cost at this point too.
+
+    NexmarkPerf.ProgressSnapshot snapshot = new NexmarkPerf.ProgressSnapshot();
+    snapshot.secSinceStart = (now - startMsSinceEpoch) / 1000.0;
+    snapshot.runtimeSec = perf.runtimeSec;
+    snapshot.numEvents = numEvents;
+    snapshot.numResults = numResults;
+    snapshots.add(snapshot);
+
+    captureSteadyState(perf, snapshots);
+
+    return perf;
+  }
+
+  /**
+   * Find a 'steady state' events/sec from {@code snapshots} and
+   * store it in {@code perf} if found.
+   */
+  private void captureSteadyState(NexmarkPerf perf, List<NexmarkPerf.ProgressSnapshot> snapshots) {
+    if (!options.isStreaming()) {
+      return;
+    }
+
+    // Find the first sample with actual event and result counts.
+    int dataStart = 0;
+    for (; dataStart < snapshots.size(); dataStart++) {
+      if (snapshots.get(dataStart).numEvents >= 0 && snapshots.get(dataStart).numResults >= 0) {
+        break;
+      }
+    }
+
+    // Find the last sample which demonstrated progress.
+    int dataEnd = snapshots.size() - 1;
+    for (; dataEnd > dataStart; dataEnd--) {
+      if (snapshots.get(dataEnd).anyActivity(snapshots.get(dataEnd - 1))) {
+        break;
+      }
+    }
+
+    int numSamples = dataEnd - dataStart + 1;
+    if (numSamples < MIN_SAMPLES) {
+      // Not enough samples.
+      NexmarkUtils.console("%d samples not enough to calculate steady-state event rate",
+          numSamples);
+      return;
+    }
+
+    // We'll look at only the middle third samples.
+    int sampleStart = dataStart + numSamples / 3;
+    int sampleEnd = dataEnd - numSamples / 3;
+
+    double sampleSec =
+        snapshots.get(sampleEnd).secSinceStart - snapshots.get(sampleStart).secSinceStart;
+    if (sampleSec < MIN_WINDOW.getStandardSeconds()) {
+      // Not sampled over enough time.
+      NexmarkUtils.console(
+          "sample of %.1f sec not long enough to calculate steady-state event rate",
+          sampleSec);
+      return;
+    }
+
+    // Find rate with least squares error.
+    double sumxx = 0.0;
+    double sumxy = 0.0;
+    long prevNumEvents = -1;
+    for (int i = sampleStart; i <= sampleEnd; i++) {
+      if (prevNumEvents == snapshots.get(i).numEvents) {
+        // Skip samples with no change in number of events since they contribute no data.
+        continue;
+      }
+      // Use the effective runtime instead of wallclock time so we can
+      // insulate ourselves from delays and stutters in the query manager.
+      double x = snapshots.get(i).runtimeSec;
+      prevNumEvents = snapshots.get(i).numEvents;
+      double y = prevNumEvents;
+      sumxx += x * x;
+      sumxy += x * y;
+    }
+    double eventsPerSec = sumxy / sumxx;
+    NexmarkUtils.console("revising events/sec from %.1f to %.1f", perf.eventsPerSec, eventsPerSec);
+    perf.eventsPerSec = eventsPerSec;
+  }
+
+  /**
+   * Return the current value for a long counter, or -1 if can't be retrieved.
+   */
+  private long getLong(DataflowPipelineJob job, Aggregator<Long, Long> aggregator) {
+    try {
+      Collection<Long> values = job.getAggregatorValues(aggregator).getValues();
+      if (values.size() != 1) {
+        return -1;
+      }
+      return Iterables.getOnlyElement(values);
+    } catch (AggregatorRetrievalException e) {
+      return -1;
+    }
+  }
+
+  /**
+   * Return the current value for a time counter, or -1 if can't be retrieved.
+   */
+  private long getTimestamp(
+      long now, DataflowPipelineJob job, Aggregator<Long, Long> aggregator) {
+    try {
+      Collection<Long> values = job.getAggregatorValues(aggregator).getValues();
+      if (values.size() != 1) {
+        return -1;
+      }
+      long value = Iterables.getOnlyElement(values);
+      if (Math.abs(value - now) > Duration.standardDays(10000).getMillis()) {
+        return -1;
+      }
+      return value;
+    } catch (AggregatorRetrievalException e) {
+      return -1;
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/1f08970a/integration/java/src/main/java/org/apache/beam/integration/nexmark/NexmarkInProcessDriver.java
----------------------------------------------------------------------
diff --git a/integration/java/src/main/java/org/apache/beam/integration/nexmark/NexmarkInProcessDriver.java b/integration/java/src/main/java/org/apache/beam/integration/nexmark/NexmarkInProcessDriver.java
new file mode 100644
index 0000000..fe279c0
--- /dev/null
+++ b/integration/java/src/main/java/org/apache/beam/integration/nexmark/NexmarkInProcessDriver.java
@@ -0,0 +1,48 @@
+/*
+ * 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.beam.integration.nexmark;
+
+import org.apache.beam.runners.dataflow.options.DataflowPipelineOptions;
+import org.apache.beam.runners.direct.InProcessPipelineRunner;
+import org.apache.beam.sdk.options.PipelineOptionsFactory;
+
+/**
+ * An implementation of the 'NEXMark queries' using the in-process runner.
+ */
+class NexmarkInProcessDriver extends NexmarkDriver<NexmarkInProcessDriver.NexmarkInProcessOptions> {
+  /**
+   * Command line flags.
+   */
+  public interface NexmarkInProcessOptions extends Options, DataflowPipelineOptions {
+  }
+
+  /**
+   * Entry point.
+   */
+  public static void main(String[] args) {
+    NexmarkInProcessOptions options =
+        PipelineOptionsFactory.fromArgs(args)
+                              .withValidation()
+                              .as(NexmarkInProcessOptions.class);
+    options.setRunner(InProcessPipelineRunner.class);
+    NexmarkInProcessRunner runner = new NexmarkInProcessRunner(options);
+    new NexmarkInProcessDriver().runAll(options, runner);
+  }
+}
+

http://git-wip-us.apache.org/repos/asf/beam/blob/1f08970a/integration/java/src/main/java/org/apache/beam/integration/nexmark/NexmarkInProcessRunner.java
----------------------------------------------------------------------
diff --git a/integration/java/src/main/java/org/apache/beam/integration/nexmark/NexmarkInProcessRunner.java b/integration/java/src/main/java/org/apache/beam/integration/nexmark/NexmarkInProcessRunner.java
new file mode 100644
index 0000000..ba141f9
--- /dev/null
+++ b/integration/java/src/main/java/org/apache/beam/integration/nexmark/NexmarkInProcessRunner.java
@@ -0,0 +1,77 @@
+/*
+ * 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.beam.integration.nexmark;
+
+import javax.annotation.Nullable;
+
+/**
+ * Run a singe Nexmark query using a given configuration on Google Dataflow.
+ */
+class NexmarkInProcessRunner extends NexmarkRunner<NexmarkInProcessDriver.NexmarkInProcessOptions> {
+  public NexmarkInProcessRunner(NexmarkInProcessDriver.NexmarkInProcessOptions options) {
+    super(options);
+  }
+
+  @Override
+  protected boolean isStreaming() {
+    return options.isStreaming();
+  }
+
+  @Override
+  protected int coresPerWorker() {
+    return 4;
+  }
+
+  @Override
+  protected int maxNumWorkers() {
+    return 1;
+  }
+
+  @Override
+  protected boolean canMonitor() {
+    return false;
+  }
+
+  @Override
+  protected void invokeBuilderForPublishOnlyPipeline(PipelineBuilder builder) {
+    throw new UnsupportedOperationException(
+        "Cannot use --pubSubMode=COMBINED with InProcessPipelineRunner");
+  }
+
+  /**
+   * Monitor the progress of the publisher job. Return when it has been generating events for
+   * at least {@code configuration.preloadSeconds}.
+   */
+  @Override
+  protected void waitForPublisherPreload() {
+    throw new UnsupportedOperationException(
+        "Cannot use --pubSubMode=COMBINED with InProcessPipelineRunner");
+  }
+
+  /**
+   * Monitor the performance and progress of a running job. Return final performance if
+   * it was measured.
+   */
+  @Override
+  @Nullable
+  protected NexmarkPerf monitor(NexmarkQuery query) {
+    throw new UnsupportedOperationException(
+        "Cannot use --monitorJobs=true with InProcessPipelineRunner");
+  }
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/1f08970a/integration/java/src/main/java/org/apache/beam/integration/nexmark/NexmarkPerf.java
----------------------------------------------------------------------
diff --git a/integration/java/src/main/java/org/apache/beam/integration/nexmark/NexmarkPerf.java b/integration/java/src/main/java/org/apache/beam/integration/nexmark/NexmarkPerf.java
new file mode 100644
index 0000000..6eb7267
--- /dev/null
+++ b/integration/java/src/main/java/org/apache/beam/integration/nexmark/NexmarkPerf.java
@@ -0,0 +1,212 @@
+/*
+ * 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.beam.integration.nexmark;
+
+import com.fasterxml.jackson.annotation.JsonProperty;
+import com.fasterxml.jackson.core.JsonProcessingException;
+
+import java.io.IOException;
+import java.util.List;
+
+import javax.annotation.Nullable;
+
+/**
+ * Summary of performance for a particular run of a configuration.
+ */
+class NexmarkPerf {
+  /**
+   * A sample of the number of events and number of results (if known) generated at
+   * a particular time.
+   */
+  public static class ProgressSnapshot {
+    /** Seconds since job was started (in wallclock time). */
+    @JsonProperty
+    double secSinceStart;
+
+    /** Job runtime in seconds (time from first event to last generated event or output result). */
+    @JsonProperty
+    double runtimeSec;
+
+    /** Cumulative number of events generated. -1 if not known. */
+    @JsonProperty
+    long numEvents;
+
+    /** Cumulative number of results emitted. -1 if not known. */
+    @JsonProperty
+    long numResults;
+
+    /**
+     * Return true if there looks to be activity between {@code this} and {@code that}
+     * snapshots.
+     */
+    public boolean anyActivity(ProgressSnapshot that) {
+      if (runtimeSec != that.runtimeSec) {
+        // An event or result end timestamp looks to have changed.
+        return true;
+      }
+      if (numEvents != that.numEvents) {
+        // Some more events were generated.
+        return true;
+      }
+      if (numResults != that.numResults) {
+        // Some more results were emitted.
+        return true;
+      }
+      return false;
+    }
+  }
+
+  /**
+   * Progess snapshots. Null if not yet calculated.
+   */
+  @JsonProperty
+  @Nullable
+  public List<ProgressSnapshot> snapshots = null;
+
+  /**
+   * Effective runtime, in seconds. Measured from timestamp of first generated event to latest of
+   * timestamp of last generated event and last emitted result. -1 if not known.
+   */
+  @JsonProperty
+  public double runtimeSec = -1.0;
+
+  /**
+   * Number of events generated. -1 if not known.
+   */
+  @JsonProperty
+  public long numEvents = -1;
+
+  /**
+   * Number of events generated per second of runtime. For batch this is number of events
+   * over the above runtime. For streaming this is the 'steady-state' event generation rate sampled
+   * over the lifetime of the job. -1 if not known.
+   */
+  @JsonProperty
+  public double eventsPerSec = -1.0;
+
+  /**
+   * Number of event bytes generated per second of runtime. -1 if not known.
+   */
+  @JsonProperty
+  public double eventBytesPerSec = -1.0;
+
+  /**
+   * Number of results emitted. -1 if not known.
+   */
+  @JsonProperty
+  public long numResults = -1;
+
+  /**
+   * Number of results generated per second of runtime. -1 if not known.
+   */
+  @JsonProperty
+  public double resultsPerSec = -1.0;
+
+  /**
+   * Number of result bytes generated per second of runtime. -1 if not known.
+   */
+  @JsonProperty
+  public double resultBytesPerSec = -1.0;
+
+  /**
+   * Delay between start of job and first event in second. -1 if not known.
+   */
+  @JsonProperty
+  public double startupDelaySec = -1.0;
+
+  /**
+   * Delay between first event and first result in seconds. -1 if not known.
+   */
+  @JsonProperty
+  public double processingDelaySec = -1.0;
+
+  /**
+   * Delay between last result and job completion in seconds. -1 if not known.
+   */
+  @JsonProperty
+  public double shutdownDelaySec = -1.0;
+
+  /**
+   * Time-dilation factor.  Calculate as event time advancement rate relative to real time.
+   * Greater than one implies we processed events faster than they would have been generated
+   * in real time. Less than one implies we could not keep up with events in real time.
+   * -1 if not known.
+   */
+  @JsonProperty
+  double timeDilation = -1.0;
+
+  /**
+   * List of errors encountered during job execution.
+   */
+  @JsonProperty
+  @Nullable
+  public List<String> errors = null;
+
+  /**
+   * The job id this perf was drawn from. Null if not known.
+   */
+  @JsonProperty
+  @Nullable
+  public String jobId = null;
+
+  /**
+   * Return a JSON representation of performance.
+   */
+  @Override
+  public String toString() {
+    try {
+      return NexmarkUtils.MAPPER.writeValueAsString(this);
+    } catch (JsonProcessingException e) {
+      throw new RuntimeException(e);
+    }
+  }
+
+  /**
+   * Parse a {@link NexmarkPerf} object from JSON {@code string}.
+   *
+   * @throws IOException
+   */
+  public static NexmarkPerf fromString(String string) {
+    try {
+      return NexmarkUtils.MAPPER.readValue(string, NexmarkPerf.class);
+    } catch (IOException e) {
+      throw new RuntimeException("Unable to parse nexmark perf: ", e);
+    }
+  }
+
+  /**
+   * Return true if there looks to be activity between {@code this} and {@code that}
+   * perf values.
+   */
+  public boolean anyActivity(NexmarkPerf that) {
+    if (runtimeSec != that.runtimeSec) {
+      // An event or result end timestamp looks to have changed.
+      return true;
+    }
+    if (numEvents != that.numEvents) {
+      // Some more events were generated.
+      return true;
+    }
+    if (numResults != that.numResults) {
+      // Some more results were emitted.
+      return true;
+    }
+    return false;
+  }
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/1f08970a/integration/java/src/main/java/org/apache/beam/integration/nexmark/NexmarkQuery.java
----------------------------------------------------------------------
diff --git a/integration/java/src/main/java/org/apache/beam/integration/nexmark/NexmarkQuery.java b/integration/java/src/main/java/org/apache/beam/integration/nexmark/NexmarkQuery.java
new file mode 100644
index 0000000..4626609
--- /dev/null
+++ b/integration/java/src/main/java/org/apache/beam/integration/nexmark/NexmarkQuery.java
@@ -0,0 +1,276 @@
+/*
+ * 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.beam.integration.nexmark;
+
+import org.apache.beam.sdk.transforms.Aggregator;
+import org.apache.beam.sdk.transforms.DoFn;
+import org.apache.beam.sdk.transforms.Filter;
+import org.apache.beam.sdk.transforms.PTransform;
+import org.apache.beam.sdk.transforms.ParDo;
+import org.apache.beam.sdk.transforms.SerializableFunction;
+import org.apache.beam.sdk.values.KV;
+import org.apache.beam.sdk.values.PCollection;
+import org.apache.beam.sdk.values.TimestampedValue;
+import org.apache.beam.sdk.values.TupleTag;
+
+import org.joda.time.Instant;
+
+import javax.annotation.Nullable;
+
+/**
+ * Base class for the eight 'NEXMark' queries. Supplies some fragments common to
+ * multiple queries.
+ */
+public abstract class NexmarkQuery
+    extends PTransform<PCollection<Event>, PCollection<TimestampedValue<KnownSize>>> {
+  protected static final TupleTag<Auction> AUCTION_TAG = new TupleTag<>("auctions");
+  protected static final TupleTag<Bid> BID_TAG = new TupleTag<>("bids");
+  protected static final TupleTag<Person> PERSON_TAG = new TupleTag<>("person");
+
+  /** Predicate to detect a new person event. */
+  protected static final SerializableFunction<Event, Boolean> IS_NEW_PERSON =
+      new SerializableFunction<Event, Boolean>() {
+        @Override
+        public Boolean apply(Event event) {
+          return event.newPerson != null;
+        }
+      };
+
+  /** DoFn to convert a new person event to a person. */
+  protected static final DoFn<Event, Person> AS_PERSON = new DoFn<Event, Person>() {
+    @Override
+    public void processElement(ProcessContext c) {
+      c.output(c.element().newPerson);
+    }
+  };
+
+  /** Predicate to detect a new auction event. */
+  protected static final SerializableFunction<Event, Boolean> IS_NEW_AUCTION =
+      new SerializableFunction<Event, Boolean>() {
+        @Override
+        public Boolean apply(Event event) {
+          return event.newAuction != null;
+        }
+      };
+
+  /** DoFn to convert a new auction event to an auction. */
+  protected static final DoFn<Event, Auction> AS_AUCTION = new DoFn<Event, Auction>() {
+    @Override
+    public void processElement(ProcessContext c) {
+      c.output(c.element().newAuction);
+    }
+  };
+
+  /** Predicate to detect a new bid event. */
+  protected static final SerializableFunction<Event, Boolean> IS_BID =
+      new SerializableFunction<Event, Boolean>() {
+        @Override
+        public Boolean apply(Event event) {
+          return event.bid != null;
+        }
+      };
+
+  /** DoFn to convert a bid event to a bid. */
+  protected static final DoFn<Event, Bid> AS_BID = new DoFn<Event, Bid>() {
+    @Override
+    public void processElement(ProcessContext c) {
+      c.output(c.element().bid);
+    }
+  };
+
+  /** Transform to key each person by their id. */
+  protected static final ParDo.Bound<Person, KV<Long, Person>> PERSON_BY_ID =
+      ParDo.named("PersonById")
+           .of(new DoFn<Person, KV<Long, Person>>() {
+             @Override
+             public void processElement(ProcessContext c) {
+               c.output(KV.of(c.element().id, c.element()));
+             }
+           });
+
+  /** Transform to key each auction by its id. */
+  protected static final ParDo.Bound<Auction, KV<Long, Auction>> AUCTION_BY_ID =
+      ParDo.named("AuctionById")
+           .of(new DoFn<Auction, KV<Long, Auction>>() {
+             @Override
+             public void processElement(ProcessContext c) {
+               c.output(KV.of(c.element().id, c.element()));
+             }
+           });
+
+  /** Transform to key each auction by its seller id. */
+  protected static final ParDo.Bound<Auction, KV<Long, Auction>> AUCTION_BY_SELLER =
+      ParDo.named("AuctionBySeller")
+           .of(new DoFn<Auction, KV<Long, Auction>>() {
+             @Override
+             public void processElement(ProcessContext c) {
+               c.output(KV.of(c.element().seller, c.element()));
+             }
+           });
+
+  /** Transform to key each bid by it's auction id. */
+  protected static final ParDo.Bound<Bid, KV<Long, Bid>> BID_BY_AUCTION =
+      ParDo.named("BidByAuction")
+           .of(new DoFn<Bid, KV<Long, Bid>>() {
+             @Override
+             public void processElement(ProcessContext c) {
+               c.output(KV.of(c.element().auction, c.element()));
+             }
+           });
+
+  /** Transform to project the auction id from each bid. */
+  protected static final ParDo.Bound<Bid, Long> BID_TO_AUCTION =
+      ParDo.named("BidToAuction")
+           .of(new DoFn<Bid, Long>() {
+             @Override
+             public void processElement(ProcessContext c) {
+               c.output(c.element().auction);
+             }
+           });
+
+  /** Transform to project the price from each bid. */
+  protected static final ParDo.Bound<Bid, Long> BID_TO_PRICE =
+      ParDo.named("BidToPrice")
+           .of(new DoFn<Bid, Long>() {
+             @Override
+             public void processElement(ProcessContext c) {
+               c.output(c.element().price);
+             }
+           });
+
+  /** Transform to emit each event with the timestamp embedded within it. */
+  public static final ParDo.Bound<Event, Event> EVENT_TIMESTAMP_FROM_DATA =
+      ParDo.named("OutputWithTimestamp")
+           .of(new DoFn<Event, Event>() {
+             @Override
+             public void processElement(ProcessContext c) {
+               Event e = c.element();
+               if (e.bid != null) {
+                 c.outputWithTimestamp(e, new Instant(e.bid.dateTime));
+               } else if (e.newPerson != null) {
+                 c.outputWithTimestamp(e, new Instant(e.newPerson.dateTime));
+               } else if (e.newAuction != null) {
+                 c.outputWithTimestamp(e, new Instant(e.newAuction.dateTime));
+               }
+             }
+           });
+
+  /**
+   * Transform to filter for just the new auction events.
+   */
+  protected static final PTransform<PCollection<Event>, PCollection<Auction>> JUST_NEW_AUCTIONS =
+      new PTransform<PCollection<Event>, PCollection<Auction>>("justNewAuctions") {
+        @Override
+        public PCollection<Auction> apply(PCollection<Event> input) {
+          return input.apply(Filter.byPredicate(IS_NEW_AUCTION).named("IsAuction"))
+                      .apply(ParDo.named("AsAuction").of(AS_AUCTION));
+        }
+      };
+
+  /**
+   * Transform to filter for just the new person events.
+   */
+  protected static final PTransform<PCollection<Event>, PCollection<Person>> JUST_NEW_PERSONS =
+      new PTransform<PCollection<Event>, PCollection<Person>>("justNewPersons") {
+        @Override
+        public PCollection<Person> apply(PCollection<Event> input) {
+          return input.apply(Filter.byPredicate(IS_NEW_PERSON).named("IsPerson"))
+                      .apply(ParDo.named("AsPerson").of(AS_PERSON));
+        }
+      };
+
+  /**
+   * Transform to filter for just the bid events.
+   */
+  protected static final PTransform<PCollection<Event>, PCollection<Bid>> JUST_BIDS =
+      new PTransform<PCollection<Event>, PCollection<Bid>>("justBids") {
+        @Override
+        public PCollection<Bid> apply(PCollection<Event> input) {
+          return input.apply(Filter.byPredicate(IS_BID).named("IsBid"))
+                      .apply(ParDo.named("AsBid").of(AS_BID));
+        }
+      };
+
+  protected final NexmarkConfiguration configuration;
+  public final Monitor<Event> eventMonitor;
+  public final Monitor<KnownSize> resultMonitor;
+  public final Monitor<Event> endOfStreamMonitor;
+
+  protected NexmarkQuery(NexmarkConfiguration configuration, String name) {
+    super(name);
+    this.configuration = configuration;
+    if (configuration.debug) {
+      eventMonitor = new Monitor<>(name + ".Events", "event");
+      resultMonitor = new Monitor<>(name + ".Results", "result");
+      endOfStreamMonitor = new Monitor<>(name + ".EndOfStream", "end");
+    } else {
+      eventMonitor = null;
+      resultMonitor = null;
+      endOfStreamMonitor = null;
+    }
+  }
+
+  /**
+   * Return the aggregator which counts fatal errors in this query. Return null if no such
+   * aggregator.
+   */
+  @Nullable
+  public Aggregator<Long, Long> getFatalCount() {
+    return null;
+  }
+
+  /**
+   * Implement the actual query. All we know about the result is it has a known encoded size.
+   */
+  protected abstract PCollection<KnownSize> applyPrim(PCollection<Event> events);
+
+  @Override
+  public PCollection<TimestampedValue<KnownSize>> apply(PCollection<Event> events) {
+
+    if (configuration.debug) {
+      events =
+          events
+              // Monitor events as they go by.
+              .apply(eventMonitor.getTransform())
+              // Count each type of event.
+              .apply(NexmarkUtils.snoop(name));
+    }
+
+    if (configuration.cpuDelayMs > 0) {
+      // Slow down by pegging one core at 100%.
+      events = events.apply(NexmarkUtils.<Event>cpuDelay(name, configuration.cpuDelayMs));
+    }
+
+    if (configuration.diskBusyBytes > 0) {
+      // Slow down by forcing bytes to durable store.
+      events = events.apply(NexmarkUtils.<Event>diskBusy(name, configuration.diskBusyBytes));
+    }
+
+    // Run the query.
+    PCollection<KnownSize> queryResults = applyPrim(events);
+
+    if (configuration.debug) {
+      // Monitor results as they go by.
+      queryResults = queryResults.apply(resultMonitor.getTransform());
+    }
+
+    // Timestamp the query results.
+    return queryResults.apply(NexmarkUtils.<KnownSize>stamp(name));
+  }
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/1f08970a/integration/java/src/main/java/org/apache/beam/integration/nexmark/NexmarkQueryModel.java
----------------------------------------------------------------------
diff --git a/integration/java/src/main/java/org/apache/beam/integration/nexmark/NexmarkQueryModel.java b/integration/java/src/main/java/org/apache/beam/integration/nexmark/NexmarkQueryModel.java
new file mode 100644
index 0000000..b42042f
--- /dev/null
+++ b/integration/java/src/main/java/org/apache/beam/integration/nexmark/NexmarkQueryModel.java
@@ -0,0 +1,123 @@
+/*
+ * 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.beam.integration.nexmark;
+
+import org.apache.beam.sdk.transforms.SerializableFunction;
+import org.apache.beam.sdk.values.TimestampedValue;
+
+import org.joda.time.Duration;
+import org.joda.time.Instant;
+import org.junit.Assert;
+
+import java.io.Serializable;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Set;
+
+/**
+ * Base class for models of the eight NEXMark queries. Provides an assertion
+ * function which can be applied against the actual query results to check their consistency
+ * with the model.
+ */
+public abstract class NexmarkQueryModel implements Serializable {
+  /**
+   * Return the start of the most recent window of {@code size} and {@code period} which ends
+   * strictly before {@code timestamp}.
+   */
+  public static Instant windowStart(Duration size, Duration period, Instant timestamp) {
+    long ts = timestamp.getMillis();
+    long p = period.getMillis();
+    long lim = ts - ts % p;
+    long s = size.getMillis();
+    return new Instant(lim - s);
+  }
+
+  protected final NexmarkConfiguration configuration;
+
+  public NexmarkQueryModel(NexmarkConfiguration configuration) {
+    this.configuration = configuration;
+  }
+
+  /**
+   * Convert {@code itr} to strings capturing values, timestamps and order.
+   */
+  protected static <T> List<String> toValueTimestampOrder(Iterator<TimestampedValue<T>> itr) {
+    List<String> strings = new ArrayList<>();
+    while (itr.hasNext()) {
+      strings.add(itr.next().toString());
+    }
+    return strings;
+  }
+
+  /**
+   * Convert {@code itr} to strings capturing values and order.
+   */
+  protected static <T> List<String> toValueOrder(Iterator<TimestampedValue<T>> itr) {
+    List<String> strings = new ArrayList<>();
+    while (itr.hasNext()) {
+      strings.add(itr.next().getValue().toString());
+    }
+    return strings;
+  }
+
+  /**
+   * Convert {@code itr} to strings capturing values only.
+   */
+  protected static <T> Set<String> toValue(Iterator<TimestampedValue<T>> itr) {
+    Set<String> strings = new HashSet<>();
+    while (itr.hasNext()) {
+      strings.add(itr.next().getValue().toString());
+    }
+    return strings;
+  }
+
+  /** Return simulator for query. */
+  protected abstract AbstractSimulator<?, ?> simulator();
+
+  /** Return sub-sequence of results which are significant for model. */
+  protected Iterable<TimestampedValue<KnownSize>> relevantResults(
+      Iterable<TimestampedValue<KnownSize>> results) {
+    return results;
+  }
+
+  /**
+   * Convert iterator of elements to collection of strings to use when testing coherence
+   * of model against actual query results.
+   */
+  protected abstract <T> Collection<String> toCollection(Iterator<TimestampedValue<T>> itr);
+
+  /**
+   * Return assertion to use on results of pipeline for this query.
+   */
+  public SerializableFunction<Iterable<TimestampedValue<KnownSize>>, Void> assertionFor() {
+    final Collection<String> expectedStrings = toCollection(simulator().results());
+
+    return new SerializableFunction<Iterable<TimestampedValue<KnownSize>>, Void>() {
+      @Override
+      public Void apply(Iterable<TimestampedValue<KnownSize>> actual) {
+        Collection<String> actualStrings = toCollection(relevantResults(actual).iterator());
+        Assert.assertEquals(expectedStrings, actualStrings);
+        return null;
+      }
+    };
+  }
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/1f08970a/integration/java/src/main/java/org/apache/beam/integration/nexmark/NexmarkRunner.java
----------------------------------------------------------------------
diff --git a/integration/java/src/main/java/org/apache/beam/integration/nexmark/NexmarkRunner.java b/integration/java/src/main/java/org/apache/beam/integration/nexmark/NexmarkRunner.java
new file mode 100644
index 0000000..b7151f8
--- /dev/null
+++ b/integration/java/src/main/java/org/apache/beam/integration/nexmark/NexmarkRunner.java
@@ -0,0 +1,746 @@
+/*
+ * 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.beam.integration.nexmark;
+
+import org.apache.beam.sdk.Pipeline;
+import org.apache.beam.sdk.PipelineResult;
+import org.apache.beam.sdk.io.AvroIO;
+import org.apache.beam.sdk.io.BigQueryIO;
+import org.apache.beam.sdk.io.PubsubIO;
+import org.apache.beam.sdk.io.TextIO;
+import org.apache.beam.sdk.testing.PAssert;
+import org.apache.beam.sdk.transforms.DoFn;
+import org.apache.beam.sdk.transforms.ParDo;
+import org.apache.beam.sdk.values.PCollection;
+import org.apache.beam.sdk.values.PCollectionTuple;
+import org.apache.beam.sdk.values.TimestampedValue;
+import org.apache.beam.sdk.values.TupleTag;
+import org.apache.beam.sdk.values.TupleTagList;
+
+import com.google.api.services.bigquery.model.TableFieldSchema;
+import com.google.api.services.bigquery.model.TableRow;
+import com.google.api.services.bigquery.model.TableSchema;
+import com.google.common.base.Preconditions;
+import com.google.common.base.Strings;
+import com.google.common.collect.ImmutableList;
+import com.google.common.collect.Lists;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.List;
+import java.util.concurrent.ThreadLocalRandom;
+import javax.annotation.Nullable;
+
+/**
+ * Run a single Nexmark query using a given configuration.
+ */
+public abstract class NexmarkRunner<OptionT extends Options> {
+  /**
+   * Options shared by all runs.
+   */
+  protected final OptionT options;
+
+  /**
+   * Which configuration we are running.
+   */
+  @Nullable
+  protected NexmarkConfiguration configuration;
+
+  /**
+   * Accumulate the pub/sub subscriptions etc which should be cleaned up on end of run.
+   */
+  @Nullable
+  protected PubsubHelper pubsub;
+
+  /**
+   * If in --pubsubMode=COMBINED, the event monitor for the publisher pipeline. Otherwise null.
+   */
+  @Nullable
+  protected Monitor<Event> publisherMonitor;
+
+  /**
+   * If in --pubsubMode=COMBINED, the pipeline result for the publisher pipeline. Otherwise null.
+   */
+  @Nullable
+  protected PipelineResult publisherResult;
+
+  /**
+   * Result for the main pipeline.
+   */
+  @Nullable
+  protected PipelineResult mainResult;
+
+  /**
+   * Query name we are running.
+   */
+  @Nullable
+  protected String queryName;
+
+  public NexmarkRunner(OptionT options) {
+    this.options = options;
+  }
+
+  /**
+   * Return a Pubsub helper.
+   */
+  private PubsubHelper getPubsub() {
+    if (pubsub == null) {
+      pubsub = PubsubHelper.create(options);
+    }
+    return pubsub;
+  }
+
+  // ================================================================================
+  // Overridden by each runner.
+  // ================================================================================
+
+  /**
+   * Is this query running in streaming mode?
+   */
+  protected abstract boolean isStreaming();
+
+  /**
+   * Return number of cores per worker.
+   */
+  protected abstract int coresPerWorker();
+
+  /**
+   * Return maximum number of workers.
+   */
+  protected abstract int maxNumWorkers();
+
+  /**
+   * Return true if runner can monitor running jobs.
+   */
+  protected abstract boolean canMonitor();
+
+  /**
+   * Build and run a pipeline using specified options.
+   */
+  protected interface PipelineBuilder<OptionT extends Options> {
+    void build(OptionT publishOnlyOptions);
+  }
+
+  /**
+   * Invoke the builder with options suitable for running a publish-only child pipeline.
+   */
+  protected abstract void invokeBuilderForPublishOnlyPipeline(PipelineBuilder builder);
+
+  /**
+   * If monitoring, wait until the publisher pipeline has run long enough to establish
+   * a backlog on the Pubsub topic. Otherwise, return immediately.
+   */
+  protected abstract void waitForPublisherPreload();
+
+  /**
+   * If monitoring, print stats on the main pipeline and return the final perf
+   * when it has run long enough. Otherwise, return {@literal null} immediately.
+   */
+  @Nullable
+  protected abstract NexmarkPerf monitor(NexmarkQuery query);
+
+  // ================================================================================
+  // Basic sources and sinks
+  // ================================================================================
+
+  /**
+   * Return a topic name.
+   */
+  private String shortTopic(long now) {
+    String baseTopic = options.getPubsubTopic();
+    if (Strings.isNullOrEmpty(baseTopic)) {
+      throw new RuntimeException("Missing --pubsubTopic");
+    }
+    switch (options.getResourceNameMode()) {
+      case VERBATIM:
+        return baseTopic;
+      case QUERY:
+        return String.format("%s_%s_source", baseTopic, queryName);
+      case QUERY_AND_SALT:
+        return String.format("%s_%s_%d_source", baseTopic, queryName, now);
+    }
+    throw new RuntimeException("Unrecognized enum " + options.getResourceNameMode());
+  }
+
+  /**
+   * Return a subscription name.
+   */
+  private String shortSubscription(long now) {
+    String baseSubscription = options.getPubsubSubscription();
+    if (Strings.isNullOrEmpty(baseSubscription)) {
+      throw new RuntimeException("Missing --pubsubSubscription");
+    }
+    switch (options.getResourceNameMode()) {
+      case VERBATIM:
+        return baseSubscription;
+      case QUERY:
+        return String.format("%s_%s_source", baseSubscription, queryName);
+      case QUERY_AND_SALT:
+        return String.format("%s_%s_%d_source", baseSubscription, queryName, now);
+    }
+    throw new RuntimeException("Unrecognized enum " + options.getResourceNameMode());
+  }
+
+  /**
+   * Return a file name for plain text.
+   */
+  private String textFilename(long now) {
+    String baseFilename = options.getOutputPath();
+    if (Strings.isNullOrEmpty(baseFilename)) {
+      throw new RuntimeException("Missing --outputPath");
+    }
+    switch (options.getResourceNameMode()) {
+      case VERBATIM:
+        return baseFilename;
+      case QUERY:
+        return String.format("%s/nexmark_%s.txt", baseFilename, queryName);
+      case QUERY_AND_SALT:
+        return String.format("%s/nexmark_%s_%d.txt", baseFilename, queryName, now);
+    }
+    throw new RuntimeException("Unrecognized enum " + options.getResourceNameMode());
+  }
+
+  /**
+   * Return a BigQuery table spec.
+   */
+  private String tableSpec(long now, String version) {
+    String baseTableName = options.getBigQueryTable();
+    if (Strings.isNullOrEmpty(baseTableName)) {
+      throw new RuntimeException("Missing --bigQueryTable");
+    }
+    switch (options.getResourceNameMode()) {
+      case VERBATIM:
+        return String.format("%s:nexmark.%s_%s",
+                             options.getProject(), baseTableName, version);
+      case QUERY:
+        return String.format("%s:nexmark.%s_%s_%s",
+                             options.getProject(), baseTableName, queryName, version);
+      case QUERY_AND_SALT:
+        return String.format("%s:nexmark.%s_%s_%s_%d",
+                             options.getProject(), baseTableName, queryName, version, now);
+    }
+    throw new RuntimeException("Unrecognized enum " + options.getResourceNameMode());
+  }
+
+  /**
+   * Return a directory for logs.
+   */
+  private String logsDir(long now) {
+    String baseFilename = options.getOutputPath();
+    if (Strings.isNullOrEmpty(baseFilename)) {
+      throw new RuntimeException("Missing --outputPath");
+    }
+    switch (options.getResourceNameMode()) {
+      case VERBATIM:
+        return baseFilename;
+      case QUERY:
+        return String.format("%s/logs_%s", baseFilename, queryName);
+      case QUERY_AND_SALT:
+        return String.format("%s/logs_%s_%d", baseFilename, queryName, now);
+    }
+    throw new RuntimeException("Unrecognized enum " + options.getResourceNameMode());
+  }
+
+  /**
+   * Return a source of synthetic events.
+   */
+  private PCollection<Event> sourceEventsFromSynthetic(Pipeline p) {
+    if (isStreaming()) {
+      NexmarkUtils.console("Generating %d events in streaming mode", configuration.numEvents);
+      return p.apply(NexmarkUtils.streamEventsSource(queryName, configuration));
+    } else {
+      NexmarkUtils.console("Generating %d events in batch mode", configuration.numEvents);
+      return p.apply(NexmarkUtils.batchEventsSource(queryName, configuration));
+    }
+  }
+
+  /**
+   * Return source of events from Pubsub.
+   */
+  private PCollection<Event> sourceEventsFromPubsub(Pipeline p, long now) {
+    String shortTopic = shortTopic(now);
+    String shortSubscription = shortSubscription(now);
+
+    // Create/confirm the subscription.
+    String subscription = null;
+    if (!options.getManageResources()) {
+      // The subscription should already have been created by the user.
+      subscription = getPubsub().reuseSubscription(shortTopic, shortSubscription).getPath();
+    } else {
+      subscription = getPubsub().createSubscription(shortTopic, shortSubscription).getPath();
+    }
+    NexmarkUtils.console("Reading events from Pubsub %s", subscription);
+    PubsubIO.Read.Bound<Event> io =
+        PubsubIO.Read.named(queryName + ".ReadPubsubEvents")
+                     .subscription(subscription)
+                     .idLabel(NexmarkUtils.PUBSUB_ID)
+                     .withCoder(Event.CODER);
+    if (!configuration.usePubsubPublishTime) {
+      io = io.timestampLabel(NexmarkUtils.PUBSUB_TIMESTAMP);
+    }
+    return p.apply(io);
+  }
+
+  /**
+   * Return Avro source of events from {@code options.getInputFilePrefix}.
+   */
+  private PCollection<Event> sourceEventsFromAvro(Pipeline p) {
+    String filename = options.getInputPath();
+    if (Strings.isNullOrEmpty(filename)) {
+      throw new RuntimeException("Missing --inputPath");
+    }
+    NexmarkUtils.console("Reading events from Avro files at %s", filename);
+    return p
+        .apply(AvroIO.Read.named(queryName + ".ReadAvroEvents")
+                          .from(filename + "*.avro")
+                          .withSchema(Event.class))
+        .apply(NexmarkQuery.EVENT_TIMESTAMP_FROM_DATA);
+  }
+
+  /**
+   * Send {@code events} to Pubsub.
+   */
+  private void sinkEventsToPubsub(PCollection<Event> events, long now) {
+    String shortTopic = shortTopic(now);
+
+    // Create/confirm the topic.
+    String topic;
+    if (!options.getManageResources()
+        || configuration.pubSubMode == NexmarkUtils.PubSubMode.SUBSCRIBE_ONLY) {
+      // The topic should already have been created by the user or
+      // a companion 'PUBLISH_ONLY' process.
+      topic = getPubsub().reuseTopic(shortTopic).getPath();
+    } else {
+      // Create a fresh topic to loopback via. It will be destroyed when the
+      // (necessarily blocking) job is done.
+      topic = getPubsub().createTopic(shortTopic).getPath();
+    }
+    NexmarkUtils.console("Writing events to Pubsub %s", topic);
+    PubsubIO.Write.Bound<Event> io =
+        PubsubIO.Write.named(queryName + ".WritePubsubEvents")
+                      .topic(topic)
+                      .idLabel(NexmarkUtils.PUBSUB_ID)
+                      .withCoder(Event.CODER);
+    if (!configuration.usePubsubPublishTime) {
+      io = io.timestampLabel(NexmarkUtils.PUBSUB_TIMESTAMP);
+    }
+    events.apply(io);
+  }
+
+  /**
+   * Send {@code formattedResults} to Pubsub.
+   */
+  private void sinkResultsToPubsub(PCollection<String> formattedResults, long now) {
+    String shortTopic = shortTopic(now);
+    String topic;
+    if (!options.getManageResources()) {
+      topic = getPubsub().reuseTopic(shortTopic).getPath();
+    } else {
+      topic = getPubsub().createTopic(shortTopic).getPath();
+    }
+    NexmarkUtils.console("Writing results to Pubsub %s", topic);
+    PubsubIO.Write.Bound<String> io =
+        PubsubIO.Write.named(queryName + ".WritePubsubResults")
+                      .topic(topic)
+                      .idLabel(NexmarkUtils.PUBSUB_ID);
+    if (!configuration.usePubsubPublishTime) {
+      io = io.timestampLabel(NexmarkUtils.PUBSUB_TIMESTAMP);
+    }
+    formattedResults.apply(io);
+  }
+
+  /**
+   * Sink all raw Events in {@code source} to {@code options.getOutputPath}.
+   * This will configure the job to write the following files:
+   * <ul>
+   * <li>{@code $outputPath/event*.avro} All Event entities.
+   * <li>{@code $outputPath/auction*.avro} Auction entities.
+   * <li>{@code $outputPath/bid*.avro} Bid entities.
+   * <li>{@code $outputPath/person*.avro} Person entities.
+   * </ul>
+   *
+   * @param source A PCollection of events.
+   */
+  private void sinkEventsToAvro(PCollection<Event> source) {
+    String filename = options.getOutputPath();
+    if (Strings.isNullOrEmpty(filename)) {
+      throw new RuntimeException("Missing --outputPath");
+    }
+    NexmarkUtils.console("Writing events to Avro files at %s", filename);
+    source.apply(AvroIO.Write.named(queryName + ".WriteAvroEvents")
+                             .to(filename + "/event")
+                             .withSuffix(".avro")
+                             .withSchema(Event.class));
+    source.apply(NexmarkQuery.JUST_BIDS)
+          .apply(AvroIO.Write.named(queryName + ".WriteAvroBids")
+                             .to(filename + "/bid")
+                             .withSuffix(".avro")
+                             .withSchema(Bid.class));
+    source.apply(NexmarkQuery.JUST_NEW_AUCTIONS)
+          .apply(AvroIO.Write.named(
+              queryName + ".WriteAvroAuctions")
+                             .to(filename + "/auction")
+                             .withSuffix(".avro")
+                             .withSchema(Auction.class));
+    source.apply(NexmarkQuery.JUST_NEW_PERSONS)
+          .apply(AvroIO.Write.named(queryName + ".WriteAvroPeople")
+                             .to(filename + "/person")
+                             .withSuffix(".avro")
+                             .withSchema(Person.class));
+  }
+
+  /**
+   * Send {@code formattedResults} to text files.
+   */
+  private void sinkResultsToText(PCollection<String> formattedResults, long now) {
+    String filename = textFilename(now);
+    NexmarkUtils.console("Writing results to text files at %s", filename);
+    formattedResults.apply(
+        TextIO.Write.named(queryName + ".WriteTextResults")
+                    .to(filename));
+  }
+
+  private static class StringToTableRow extends DoFn<String, TableRow> {
+    @Override
+    public void processElement(ProcessContext c) {
+      int n = ThreadLocalRandom.current().nextInt(10);
+      List<TableRow> records = new ArrayList<>(n);
+      for (int i = 0; i < n; i++) {
+        records.add(new TableRow().set("index", i).set("value", Integer.toString(i)));
+      }
+      c.output(new TableRow().set("result", c.element()).set("records", records));
+    }
+  }
+
+  /**
+   * Send {@code formattedResults} to BigQuery.
+   */
+  private void sinkResultsToBigQuery(
+      PCollection<String> formattedResults, long now,
+      String version) {
+    String tableSpec = tableSpec(now, version);
+    TableSchema tableSchema =
+        new TableSchema().setFields(ImmutableList.of(
+            new TableFieldSchema().setName("result").setType("STRING"),
+            new TableFieldSchema().setName("records").setMode("REPEATED").setType("RECORD")
+                                  .setFields(ImmutableList.of(
+                                      new TableFieldSchema().setName("index").setType("INTEGER"),
+                                      new TableFieldSchema().setName("value").setType("STRING")))));
+    NexmarkUtils.console("Writing results to BigQuery table %s", tableSpec);
+    BigQueryIO.Write.Bound io =
+        BigQueryIO.Write.named(queryName + ".WriteBigQueryResults")
+                        .to(tableSpec)
+                        .withSchema(tableSchema)
+                        .withCreateDisposition(BigQueryIO.Write.CreateDisposition.CREATE_IF_NEEDED)
+                        .withWriteDisposition(BigQueryIO.Write.WriteDisposition.WRITE_APPEND);
+    formattedResults
+        .apply(ParDo.named(queryName + ".StringToTableRow")
+                    .of(new StringToTableRow()))
+        .apply(io);
+  }
+
+  // ================================================================================
+  // Construct overall pipeline
+  // ================================================================================
+
+  /**
+   * Return source of events for this run, or null if we are simply publishing events
+   * to Pubsub.
+   */
+  private PCollection<Event> createSource(Pipeline p, final long now) {
+    PCollection<Event> source = null;
+    switch (configuration.sourceType) {
+      case DIRECT:
+        source = sourceEventsFromSynthetic(p);
+        break;
+      case AVRO:
+        source = sourceEventsFromAvro(p);
+        break;
+      case PUBSUB:
+        // Setup the sink for the publisher.
+        switch (configuration.pubSubMode) {
+          case SUBSCRIBE_ONLY:
+            // Nothing to publish.
+            break;
+          case PUBLISH_ONLY:
+            // Send synthesized events to Pubsub in this job.
+            sinkEventsToPubsub(sourceEventsFromSynthetic(p).apply(NexmarkUtils.snoop(queryName)),
+                               now);
+            break;
+          case COMBINED:
+            // Send synthesized events to Pubsub in separate publisher job.
+            // We won't start the main pipeline until the publisher has sent the pre-load events.
+            // We'll shutdown the publisher job when we notice the main job has finished.
+            invokeBuilderForPublishOnlyPipeline(new PipelineBuilder() {
+              @Override
+              public void build(Options publishOnlyOptions) {
+                Pipeline sp = Pipeline.create(options);
+                NexmarkUtils.setupPipeline(configuration.coderStrategy, sp);
+                publisherMonitor = new Monitor<Event>(queryName, "publisher");
+                sinkEventsToPubsub(
+                    sourceEventsFromSynthetic(sp).apply(publisherMonitor.getTransform()),
+                    now);
+                publisherResult = sp.run();
+              }
+            });
+            break;
+        }
+
+        // Setup the source for the consumer.
+        switch (configuration.pubSubMode) {
+          case PUBLISH_ONLY:
+            // Nothing to consume. Leave source null.
+            break;
+          case SUBSCRIBE_ONLY:
+          case COMBINED:
+            // Read events from pubsub.
+            source = sourceEventsFromPubsub(p, now);
+            break;
+        }
+        break;
+    }
+    return source;
+  }
+
+  private static final TupleTag<String> MAIN = new TupleTag<String>(){};
+  private static final TupleTag<String> SIDE = new TupleTag<String>(){};
+
+  private static class PartitionDoFn extends DoFn<String, String> {
+    @Override
+    public void processElement(ProcessContext c) {
+      if (c.element().hashCode() % 2 == 0) {
+        c.output(c.element());
+      } else {
+        c.sideOutput(SIDE, c.element());
+      }
+    }
+  }
+
+  /**
+   * Consume {@code results}.
+   */
+  private void sink(PCollection<TimestampedValue<KnownSize>> results, long now) {
+    if (configuration.sinkType == NexmarkUtils.SinkType.COUNT_ONLY) {
+      // Avoid the cost of formatting the results.
+      results.apply(NexmarkUtils.devNull(queryName));
+      return;
+    }
+
+    PCollection<String> formattedResults = results.apply(NexmarkUtils.format(queryName));
+    if (options.getLogResults()) {
+      formattedResults = formattedResults.apply(NexmarkUtils.<String>log(queryName + ".Results"));
+    }
+
+    switch (configuration.sinkType) {
+      case DEVNULL:
+        // Discard all results
+        formattedResults.apply(NexmarkUtils.devNull(queryName));
+        break;
+      case PUBSUB:
+        sinkResultsToPubsub(formattedResults, now);
+        break;
+      case TEXT:
+        sinkResultsToText(formattedResults, now);
+        break;
+      case AVRO:
+        NexmarkUtils.console(
+            "WARNING: with --sinkType=AVRO, actual query results will be discarded.");
+        break;
+      case BIGQUERY:
+        // Multiple BigQuery backends to mimic what most customers do.
+        PCollectionTuple res = formattedResults.apply(
+            ParDo.named(queryName + ".Partition")
+                 .withOutputTags(MAIN, TupleTagList.of(SIDE))
+                 .of(new PartitionDoFn()));
+        sinkResultsToBigQuery(res.get(MAIN), now, "main");
+        sinkResultsToBigQuery(res.get(SIDE), now, "side");
+        sinkResultsToBigQuery(formattedResults, now, "copy");
+        break;
+      case COUNT_ONLY:
+        // Short-circuited above.
+        throw new RuntimeException();
+    }
+  }
+
+  // ================================================================================
+  // Entry point
+  // ================================================================================
+
+  /**
+   * Calculate the distribution of the expected rate of results per minute (in event time, not
+   * wallclock time).
+   */
+  private void modelResultRates(NexmarkQueryModel model) {
+    List<Long> counts = Lists.newArrayList(model.simulator().resultsPerWindow());
+    Collections.sort(counts);
+    int n = counts.size();
+    if (n < 5) {
+      NexmarkUtils.console("Query%d: only %d samples", model.configuration.query, n);
+    } else {
+      NexmarkUtils.console("Query%d: N:%d; min:%d; 1st%%:%d; mean:%d; 3rd%%:%d; max:%d",
+                           model.configuration.query, n, counts.get(0), counts.get(n / 4),
+                           counts.get(n / 2),
+                           counts.get(n - 1 - n / 4), counts.get(n - 1));
+    }
+  }
+
+  /**
+   * Run {@code configuration} and return its performance if possible.
+   */
+  @Nullable
+  public NexmarkPerf run(NexmarkConfiguration runConfiguration) {
+    if (options.getMonitorJobs() && !canMonitor()) {
+      throw new RuntimeException("Cannot use --monitorJobs with this runner since it does not "
+                                 + "support monitoring.");
+    }
+    if (options.getManageResources() && !options.getMonitorJobs()) {
+      throw new RuntimeException("If using --manageResources then must also use --monitorJobs.");
+    }
+
+    //
+    // Setup per-run state.
+    //
+    Preconditions.checkState(configuration == null);
+    Preconditions.checkState(pubsub == null);
+    Preconditions.checkState(queryName == null);
+    configuration = runConfiguration;
+
+    // GCS URI patterns to delete on exit.
+    List<String> pathsToDelete = new ArrayList<>();
+
+    try {
+      NexmarkUtils.console("Running %s", configuration.toShortString());
+
+      if (configuration.numEvents < 0) {
+        NexmarkUtils.console("skipping since configuration is disabled");
+        return null;
+      }
+
+      List<NexmarkQuery> queries = Arrays.asList(new Query0(configuration),
+                                                 new Query1(configuration),
+                                                 new Query2(configuration),
+                                                 new Query3(configuration),
+                                                 new Query4(configuration),
+                                                 new Query5(configuration),
+                                                 new Query6(configuration),
+                                                 new Query7(configuration),
+                                                 new Query8(configuration),
+                                                 new Query9(configuration),
+                                                 new Query10(configuration),
+                                                 new Query11(configuration),
+                                                 new Query12(configuration));
+      NexmarkQuery query = queries.get(configuration.query);
+      queryName = query.getName();
+
+      List<NexmarkQueryModel> models = Arrays.asList(
+          new Query0Model(configuration),
+          new Query1Model(configuration),
+          new Query2Model(configuration),
+          new Query3Model(configuration),
+          new Query4Model(configuration),
+          new Query5Model(configuration),
+          new Query6Model(configuration),
+          new Query7Model(configuration),
+          new Query8Model(configuration),
+          new Query9Model(configuration),
+          null,
+          null,
+          null);
+      NexmarkQueryModel model = models.get(configuration.query);
+
+      if (options.getJustModelResultRate()) {
+        if (model == null) {
+          throw new RuntimeException(String.format("No model for %s", queryName));
+        }
+        modelResultRates(model);
+        return null;
+      }
+
+      long now = System.currentTimeMillis();
+      Pipeline p = Pipeline.create(options);
+      NexmarkUtils.setupPipeline(configuration.coderStrategy, p);
+
+      // Generate events.
+      PCollection<Event> source = createSource(p, now);
+
+      if (options.getLogEvents()) {
+        source = source.apply(NexmarkUtils.<Event>log(queryName + ".Events"));
+      }
+
+      // Source will be null if source type is PUBSUB and mode is PUBLISH_ONLY.
+      // In that case there's nothing more to add to pipeline.
+      if (source != null) {
+        // Optionally sink events in Avro format.
+        // (Query results are ignored).
+        if (configuration.sinkType == NexmarkUtils.SinkType.AVRO) {
+          sinkEventsToAvro(source);
+        }
+
+        // Special hacks for Query 10 (big logger).
+        if (configuration.query == 10) {
+          String path = null;
+          if (options.getOutputPath() != null && !options.getOutputPath().isEmpty()) {
+            path = logsDir(now);
+          }
+          ((Query10) query).setOutputPath(path);
+          ((Query10) query).setMaxNumWorkers(maxNumWorkers());
+          if (path != null && options.getManageResources()) {
+            pathsToDelete.add(path + "/**");
+          }
+        }
+
+        // Apply query.
+        PCollection<TimestampedValue<KnownSize>> results = source.apply(query);
+
+        if (options.getAssertCorrectness()) {
+          if (model == null) {
+            throw new RuntimeException(String.format("No model for %s", queryName));
+          }
+          // We know all our streams have a finite number of elements.
+          results.setIsBoundedInternal(PCollection.IsBounded.BOUNDED);
+          // If we have a finite number of events then assert our pipeline's
+          // results match those of a model using the same sequence of events.
+          PAssert.that(results).satisfies(model.assertionFor());
+        }
+
+        // Output results.
+        sink(results, now);
+      }
+
+      if (publisherResult != null) {
+        waitForPublisherPreload();
+      }
+      mainResult = p.run();
+      return monitor(query);
+    } finally {
+      //
+      // Cleanup per-run state.
+      //
+      if (pubsub != null) {
+        // Delete any subscriptions and topics we created.
+        pubsub.close();
+        pubsub = null;
+      }
+      configuration = null;
+      queryName = null;
+      // TODO: Cleanup pathsToDelete
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/1f08970a/integration/java/src/main/java/org/apache/beam/integration/nexmark/NexmarkSuite.java
----------------------------------------------------------------------
diff --git a/integration/java/src/main/java/org/apache/beam/integration/nexmark/NexmarkSuite.java b/integration/java/src/main/java/org/apache/beam/integration/nexmark/NexmarkSuite.java
new file mode 100644
index 0000000..cccaeb1
--- /dev/null
+++ b/integration/java/src/main/java/org/apache/beam/integration/nexmark/NexmarkSuite.java
@@ -0,0 +1,112 @@
+/*
+ * 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.beam.integration.nexmark;
+
+import java.util.ArrayList;
+import java.util.LinkedHashSet;
+import java.util.List;
+import java.util.Set;
+
+/**
+ * A set of {@link NexmarkConfiguration}s.
+ */
+public enum NexmarkSuite {
+  /**
+   * The default.
+   */
+  DEFAULT(defaultConf()),
+
+  /**
+   * Sweep through all 11 queries using the default configuration.
+   * 100k/10k events (depending on query).
+   */
+  SMOKE(smoke()),
+
+  /**
+   * As for SMOKE, but with 10m/1m events.
+   */
+  STRESS(stress()),
+
+  /**
+   * As for SMOKE, but with 1b/100m events.
+   */
+  FULL_THROTTLE(fullThrottle());
+
+  private static List<NexmarkConfiguration> defaultConf() {
+    List<NexmarkConfiguration> configurations = new ArrayList<>();
+    configurations.add(new NexmarkConfiguration());
+    return configurations;
+  }
+
+  private static List<NexmarkConfiguration> smoke() {
+    List<NexmarkConfiguration> configurations = new ArrayList<>();
+    for (int query = 0; query <= 12; query++) {
+      NexmarkConfiguration configuration = NexmarkConfiguration.DEFAULT.clone();
+      configuration.query = query;
+      configuration.numEvents = 100_000;
+      if (query == 4 || query == 6 || query == 9) {
+        // Scale back so overall runtimes are reasonably close across all queries.
+        configuration.numEvents /= 10;
+      }
+      configurations.add(configuration);
+    }
+    return configurations;
+  }
+
+  private static List<NexmarkConfiguration> stress() {
+    List<NexmarkConfiguration> configurations = smoke();
+    for (NexmarkConfiguration configuration : configurations) {
+      if (configuration.numEvents >= 0) {
+        configuration.numEvents *= 1000;
+      }
+    }
+    return configurations;
+  }
+
+  private static List<NexmarkConfiguration> fullThrottle() {
+    List<NexmarkConfiguration> configurations = smoke();
+    for (NexmarkConfiguration configuration : configurations) {
+      if (configuration.numEvents >= 0) {
+        configuration.numEvents *= 1000;
+      }
+    }
+    return configurations;
+  }
+
+  private final List<NexmarkConfiguration> configurations;
+
+  NexmarkSuite(List<NexmarkConfiguration> configurations) {
+    this.configurations = configurations;
+  }
+
+  /**
+   * Return the configurations corresponding to this suite. We'll override each configuration
+   * with any set command line flags, except for --isStreaming which is only respected for
+   * the {@link #DEFAULT} suite.
+   */
+  public Iterable<NexmarkConfiguration> getConfigurations(Options options) {
+    Set<NexmarkConfiguration> results = new LinkedHashSet<>();
+    for (NexmarkConfiguration configuration : configurations) {
+      NexmarkConfiguration result = configuration.clone();
+      result.overrideFromOptions(options);
+      results.add(result);
+    }
+    return results;
+  }
+}


[09/55] [abbrv] beam git commit: Refactor classes into packages

Posted by ie...@apache.org.
http://git-wip-us.apache.org/repos/asf/beam/blob/a7f9f7d0/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/sources/BoundedEventSource.java
----------------------------------------------------------------------
diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/sources/BoundedEventSource.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/sources/BoundedEventSource.java
new file mode 100644
index 0000000..be74151
--- /dev/null
+++ b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/sources/BoundedEventSource.java
@@ -0,0 +1,190 @@
+/*
+ * 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.beam.integration.nexmark.sources;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.NoSuchElementException;
+import javax.annotation.Nullable;
+import org.apache.beam.integration.nexmark.NexmarkUtils;
+import org.apache.beam.integration.nexmark.model.Event;
+import org.apache.beam.sdk.coders.Coder;
+import org.apache.beam.sdk.io.BoundedSource;
+import org.apache.beam.sdk.options.PipelineOptions;
+import org.apache.beam.sdk.values.TimestampedValue;
+import org.joda.time.Instant;
+
+/**
+ * A custom, bounded source of event records.
+ */
+public class BoundedEventSource extends BoundedSource<Event> {
+  /** Configuration we generate events against. */
+  private final GeneratorConfig config;
+
+  /** How many bounded sources to create. */
+  private final int numEventGenerators;
+
+  public BoundedEventSource(GeneratorConfig config, int numEventGenerators) {
+    this.config = config;
+    this.numEventGenerators = numEventGenerators;
+  }
+
+  /** A reader to pull events from the generator. */
+  private static class EventReader extends BoundedReader<Event> {
+    /**
+     * Event source we purporting to be reading from.
+     * (We can't use Java's capture-outer-class pointer since we must update
+     * this field on calls to splitAtFraction.)
+     */
+    private BoundedEventSource source;
+
+    /** Generator we are reading from. */
+    private final Generator generator;
+
+    private boolean reportedStop;
+
+    @Nullable
+    private TimestampedValue<Event> currentEvent;
+
+    public EventReader(BoundedEventSource source, GeneratorConfig config) {
+      this.source = source;
+      generator = new Generator(config);
+      reportedStop = false;
+    }
+
+    @Override
+    public synchronized boolean start() {
+      NexmarkUtils.info("starting bounded generator %s", generator);
+      return advance();
+    }
+
+    @Override
+    public synchronized boolean advance() {
+      if (!generator.hasNext()) {
+        // No more events.
+        if (!reportedStop) {
+          reportedStop = true;
+          NexmarkUtils.info("stopped bounded generator %s", generator);
+        }
+        return false;
+      }
+      currentEvent = generator.next();
+      return true;
+    }
+
+    @Override
+    public synchronized Event getCurrent() throws NoSuchElementException {
+      if (currentEvent == null) {
+        throw new NoSuchElementException();
+      }
+      return currentEvent.getValue();
+    }
+
+    @Override
+    public synchronized Instant getCurrentTimestamp() throws NoSuchElementException {
+      if (currentEvent == null) {
+        throw new NoSuchElementException();
+      }
+      return currentEvent.getTimestamp();
+    }
+
+    @Override
+    public void close() throws IOException {
+      // Nothing to close.
+    }
+
+    @Override
+    public synchronized Double getFractionConsumed() {
+      return generator.getFractionConsumed();
+    }
+
+    @Override
+    public synchronized BoundedSource<Event> getCurrentSource() {
+      return source;
+    }
+
+    @Override
+    @Nullable
+    public synchronized BoundedEventSource splitAtFraction(double fraction) {
+      long startId = generator.getCurrentConfig().getStartEventId();
+      long stopId = generator.getCurrentConfig().getStopEventId();
+      long size = stopId - startId;
+      long splitEventId = startId + Math.min((int) (size * fraction), size);
+      if (splitEventId <= generator.getNextEventId() || splitEventId == stopId) {
+        // Already passed this position or split results in left or right being empty.
+        NexmarkUtils.info("split failed for bounded generator %s at %f", generator, fraction);
+        return null;
+      }
+
+      NexmarkUtils.info("about to split bounded generator %s at %d", generator, splitEventId);
+
+      // Scale back the event space of the current generator, and return a generator config
+      // representing the event space we just 'stole' from the current generator.
+      GeneratorConfig remainingConfig = generator.splitAtEventId(splitEventId);
+
+      NexmarkUtils.info("split bounded generator into %s and %s", generator, remainingConfig);
+
+      // At this point
+      //   generator.events() ++ new Generator(remainingConfig).events()
+      //   == originalGenerator.events()
+
+      // We need a new source to represent the now smaller key space for this reader, so
+      // that we can maintain the invariant that
+      //   this.getCurrentSource().createReader(...)
+      // will yield the same output as this.
+      source = new BoundedEventSource(generator.getCurrentConfig(), source.numEventGenerators);
+
+      // Return a source from which we may read the 'stolen' event space.
+      return new BoundedEventSource(remainingConfig, source.numEventGenerators);
+    }
+  }
+
+  @Override
+  public List<BoundedEventSource> splitIntoBundles(
+      long desiredBundleSizeBytes, PipelineOptions options) {
+    NexmarkUtils.info("slitting bounded source %s into %d sub-sources", config, numEventGenerators);
+    List<BoundedEventSource> results = new ArrayList<>();
+    // Ignore desiredBundleSizeBytes and use numEventGenerators instead.
+    for (GeneratorConfig subConfig : config.split(numEventGenerators)) {
+      results.add(new BoundedEventSource(subConfig, 1));
+    }
+    return results;
+  }
+
+  @Override
+  public long getEstimatedSizeBytes(PipelineOptions options) {
+    return config.getEstimatedSizeBytes();
+  }
+
+  @Override
+  public EventReader createReader(PipelineOptions options) {
+    NexmarkUtils.info("creating initial bounded reader for %s", config);
+    return new EventReader(this, config);
+  }
+
+  @Override
+  public void validate() {
+    // Nothing to validate.
+  }
+
+  @Override
+  public Coder<Event> getDefaultOutputCoder() {
+    return Event.CODER;
+  }
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/a7f9f7d0/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/sources/Generator.java
----------------------------------------------------------------------
diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/sources/Generator.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/sources/Generator.java
new file mode 100644
index 0000000..cffc7a5
--- /dev/null
+++ b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/sources/Generator.java
@@ -0,0 +1,593 @@
+/*
+ * 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.beam.integration.nexmark.sources;
+
+import static com.google.common.base.Preconditions.checkNotNull;
+
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
+import java.io.Serializable;
+import java.util.Arrays;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Random;
+
+import org.apache.beam.integration.nexmark.model.Auction;
+import org.apache.beam.integration.nexmark.model.Bid;
+import org.apache.beam.integration.nexmark.model.Event;
+import org.apache.beam.integration.nexmark.model.Person;
+import org.apache.beam.sdk.coders.AtomicCoder;
+import org.apache.beam.sdk.coders.Coder;
+import org.apache.beam.sdk.coders.CoderException;
+import org.apache.beam.sdk.coders.VarLongCoder;
+import org.apache.beam.sdk.io.UnboundedSource;
+import org.apache.beam.sdk.values.TimestampedValue;
+import org.joda.time.Instant;
+
+/**
+ * A generator for synthetic events. We try to make the data vaguely reasonable. We also ensure
+ * most primary key/foreign key relations are correct. Eg: a {@link Bid} event will usually have
+ * valid auction and bidder ids which can be joined to already-generated Auction and Person events.
+ *
+ * <p>To help with testing, we generate timestamps relative to a given {@code baseTime}. Each new
+ * event is given a timestamp advanced from the previous timestamp by {@code interEventDelayUs}
+ * (in microseconds). The event stream is thus fully deterministic and does not depend on
+ * wallclock time.
+ *
+ * <p>This class implements {@link org.apache.beam.sdk.io.UnboundedSource.CheckpointMark}
+ * so that we can resume generating events from a saved snapshot.
+ */
+public class Generator implements Iterator<TimestampedValue<Event>>, Serializable {
+  /**
+   * Keep the number of categories small so the example queries will find results even with
+   * a small batch of events.
+   */
+  private static final int NUM_CATEGORIES = 5;
+
+  /** Smallest random string size. */
+  private static final int MIN_STRING_LENGTH = 3;
+
+  /**
+   * Keep the number of states small so that the example queries will find results even with
+   * a small batch of events.
+   */
+  private static final List<String> US_STATES = Arrays.asList(("AZ,CA,ID,OR,WA,WY").split(","));
+
+  private static final List<String> US_CITIES =
+      Arrays.asList(
+          ("Phoenix,Los Angeles,San Francisco,Boise,Portland,Bend,Redmond,Seattle,Kent,Cheyenne")
+              .split(","));
+
+  private static final List<String> FIRST_NAMES =
+      Arrays.asList(("Peter,Paul,Luke,John,Saul,Vicky,Kate,Julie,Sarah,Deiter,Walter").split(","));
+
+  private static final List<String> LAST_NAMES =
+      Arrays.asList(("Shultz,Abrams,Spencer,White,Bartels,Walton,Smith,Jones,Noris").split(","));
+
+  /**
+   * Number of yet-to-be-created people and auction ids allowed.
+   */
+  private static final int PERSON_ID_LEAD = 10;
+  private static final int AUCTION_ID_LEAD = 10;
+
+  /**
+   * Fraction of people/auctions which may be 'hot' sellers/bidders/auctions are 1
+   * over these values.
+   */
+  private static final int HOT_AUCTION_RATIO = 100;
+  private static final int HOT_SELLER_RATIO = 100;
+  private static final int HOT_BIDDER_RATIO = 100;
+
+  /**
+   * Just enough state to be able to restore a generator back to where it was checkpointed.
+   */
+  public static class Checkpoint implements UnboundedSource.CheckpointMark {
+    private static final Coder<Long> LONG_CODER = VarLongCoder.of();
+
+    /** Coder for this class. */
+    public static final Coder<Checkpoint> CODER_INSTANCE =
+        new AtomicCoder<Checkpoint>() {
+          @Override
+          public void encode(
+              Checkpoint value,
+              OutputStream outStream,
+              Coder.Context context)
+              throws CoderException, IOException {
+            LONG_CODER.encode(value.numEvents, outStream, Context.NESTED);
+            LONG_CODER.encode(value.wallclockBaseTime, outStream, Context.NESTED);
+          }
+
+          @Override
+          public Checkpoint decode(
+              InputStream inStream, Coder.Context context)
+              throws CoderException, IOException {
+            long numEvents = LONG_CODER.decode(inStream, Context.NESTED);
+            long wallclockBaseTime = LONG_CODER.decode(inStream, Context.NESTED);
+            return new Checkpoint(numEvents, wallclockBaseTime);
+          }
+        };
+
+    private long numEvents;
+    private long wallclockBaseTime;
+
+    private Checkpoint(long numEvents, long wallclockBaseTime) {
+      this.numEvents = numEvents;
+      this.wallclockBaseTime = wallclockBaseTime;
+    }
+
+    public Generator toGenerator(GeneratorConfig config) {
+      return new Generator(config, numEvents, wallclockBaseTime);
+    }
+
+    @Override
+    public void finalizeCheckpoint() throws IOException {
+      // Nothing to finalize.
+    }
+
+    @Override
+    public String toString() {
+      return String.format("Generator.Checkpoint{numEvents:%d;wallclockBaseTime:%d}",
+          numEvents, wallclockBaseTime);
+    }
+  }
+
+  /**
+   * The next event and its various timestamps. Ordered by increasing wallclock timestamp, then
+   * (arbitrary but stable) event hash order.
+   */
+  public static class NextEvent implements Comparable<NextEvent> {
+    /** When, in wallclock time, should this event be emitted? */
+    public final long wallclockTimestamp;
+
+    /** When, in event time, should this event be considered to have occured? */
+    public final long eventTimestamp;
+
+    /** The event itself. */
+    public final Event event;
+
+    /** The minimum of this and all future event timestamps. */
+    public final long watermark;
+
+    public NextEvent(long wallclockTimestamp, long eventTimestamp, Event event, long watermark) {
+      this.wallclockTimestamp = wallclockTimestamp;
+      this.eventTimestamp = eventTimestamp;
+      this.event = event;
+      this.watermark = watermark;
+    }
+
+    /**
+     * Return a deep clone of next event with delay added to wallclock timestamp and
+     * event annotate as 'LATE'.
+     */
+    public NextEvent withDelay(long delayMs) {
+      return new NextEvent(
+          wallclockTimestamp + delayMs, eventTimestamp, event.withAnnotation("LATE"), watermark);
+    }
+
+    @Override
+    public int compareTo(NextEvent other) {
+      int i = Long.compare(wallclockTimestamp, other.wallclockTimestamp);
+      if (i != 0) {
+        return i;
+      }
+      return Integer.compare(event.hashCode(), other.event.hashCode());
+    }
+  }
+
+  /**
+   * Configuration to generate events against. Note that it may be replaced by a call to
+   * {@link #splitAtEventId}.
+   */
+  private GeneratorConfig config;
+
+  /** Number of events generated by this generator. */
+  private long numEvents;
+
+  /**
+   * Wallclock time at which we emitted the first event (ms since epoch). Initially -1.
+   */
+  private long wallclockBaseTime;
+
+  private Generator(GeneratorConfig config, long numEvents, long wallclockBaseTime) {
+    checkNotNull(config);
+    this.config = config;
+    this.numEvents = numEvents;
+    this.wallclockBaseTime = wallclockBaseTime;
+  }
+
+  /**
+   * Create a fresh generator according to {@code config}.
+   */
+  public Generator(GeneratorConfig config) {
+    this(config, 0, -1);
+  }
+
+  /**
+   * Return a checkpoint for the current generator.
+   */
+  public Checkpoint toCheckpoint() {
+    return new Checkpoint(numEvents, wallclockBaseTime);
+  }
+
+  /**
+   * Return a deep clone of this generator.
+   */
+  @Override
+  public Generator clone() {
+    return new Generator(config.clone(), numEvents, wallclockBaseTime);
+  }
+
+  /**
+   * Return the current config for this generator. Note that configs may be replaced by {@link
+   * #splitAtEventId}.
+   */
+  public GeneratorConfig getCurrentConfig() {
+    return config;
+  }
+
+  /**
+   * Mutate this generator so that it will only generate events up to but not including
+   * {@code eventId}. Return a config to represent the events this generator will no longer yield.
+   * The generators will run in on a serial timeline.
+   */
+  public GeneratorConfig splitAtEventId(long eventId) {
+    long newMaxEvents = eventId - (config.firstEventId + config.firstEventNumber);
+    GeneratorConfig remainConfig = config.cloneWith(config.firstEventId,
+        config.maxEvents - newMaxEvents, config.firstEventNumber + newMaxEvents);
+    config = config.cloneWith(config.firstEventId, newMaxEvents, config.firstEventNumber);
+    return remainConfig;
+  }
+
+  /**
+   * Return the next 'event id'. Though events don't have ids we can simulate them to
+   * help with bookkeeping.
+   */
+  public long getNextEventId() {
+    return config.firstEventId + config.nextAdjustedEventNumber(numEvents);
+  }
+
+  /**
+   * Return the last valid person id (ignoring FIRST_PERSON_ID). Will be the current person id if
+   * due to generate a person.
+   */
+  private long lastBase0PersonId() {
+    long eventId = getNextEventId();
+    long epoch = eventId / GeneratorConfig.PROPORTION_DENOMINATOR;
+    long offset = eventId % GeneratorConfig.PROPORTION_DENOMINATOR;
+    if (offset >= GeneratorConfig.PERSON_PROPORTION) {
+      // About to generate an auction or bid.
+      // Go back to the last person generated in this epoch.
+      offset = GeneratorConfig.PERSON_PROPORTION - 1;
+    }
+    // About to generate a person.
+    return epoch * GeneratorConfig.PERSON_PROPORTION + offset;
+  }
+
+  /**
+   * Return the last valid auction id (ignoring FIRST_AUCTION_ID). Will be the current auction id if
+   * due to generate an auction.
+   */
+  private long lastBase0AuctionId() {
+    long eventId = getNextEventId();
+    long epoch = eventId / GeneratorConfig.PROPORTION_DENOMINATOR;
+    long offset = eventId % GeneratorConfig.PROPORTION_DENOMINATOR;
+    if (offset < GeneratorConfig.PERSON_PROPORTION) {
+      // About to generate a person.
+      // Go back to the last auction in the last epoch.
+      epoch--;
+      offset = GeneratorConfig.AUCTION_PROPORTION - 1;
+    } else if (offset >= GeneratorConfig.PERSON_PROPORTION + GeneratorConfig.AUCTION_PROPORTION) {
+      // About to generate a bid.
+      // Go back to the last auction generated in this epoch.
+      offset = GeneratorConfig.AUCTION_PROPORTION - 1;
+    } else {
+      // About to generate an auction.
+      offset -= GeneratorConfig.PERSON_PROPORTION;
+    }
+    return epoch * GeneratorConfig.AUCTION_PROPORTION + offset;
+  }
+
+  /** return a random US state. */
+  private static String nextUSState(Random random) {
+    return US_STATES.get(random.nextInt(US_STATES.size()));
+  }
+
+  /** Return a random US city. */
+  private static String nextUSCity(Random random) {
+    return US_CITIES.get(random.nextInt(US_CITIES.size()));
+  }
+
+  /** Return a random person name. */
+  private static String nextPersonName(Random random) {
+    return FIRST_NAMES.get(random.nextInt(FIRST_NAMES.size())) + " "
+        + LAST_NAMES.get(random.nextInt(LAST_NAMES.size()));
+  }
+
+  /** Return a random string of up to {@code maxLength}. */
+  private static String nextString(Random random, int maxLength) {
+    int len = MIN_STRING_LENGTH + random.nextInt(maxLength - MIN_STRING_LENGTH);
+    StringBuilder sb = new StringBuilder();
+    while (len-- > 0) {
+      if (random.nextInt(13) == 0) {
+        sb.append(' ');
+      } else {
+        sb.append((char) ('a' + random.nextInt(26)));
+      }
+    }
+    return sb.toString().trim();
+  }
+
+  /** Return a random string of exactly {@code length}. */
+  private static String nextExactString(Random random, int length) {
+    StringBuilder sb = new StringBuilder();
+    while (length-- > 0) {
+      sb.append((char) ('a' + random.nextInt(26)));
+    }
+    return sb.toString();
+  }
+
+  /** Return a random email address. */
+  private static String nextEmail(Random random) {
+    return nextString(random, 7) + "@" + nextString(random, 5) + ".com";
+  }
+
+  /** Return a random credit card number. */
+  private static String nextCreditCard(Random random) {
+    StringBuilder sb = new StringBuilder();
+    for (int i = 0; i < 4; i++) {
+      if (i > 0) {
+        sb.append(' ');
+      }
+      sb.append(String.format("%04d", random.nextInt(10000)));
+    }
+    return sb.toString();
+  }
+
+  /** Return a random price. */
+  private static long nextPrice(Random random) {
+    return Math.round(Math.pow(10.0, random.nextDouble() * 6.0) * 100.0);
+  }
+
+  /** Return a random time delay, in milliseconds, for length of auctions. */
+  private long nextAuctionLengthMs(Random random, long timestamp) {
+    // What's our current event number?
+    long currentEventNumber = config.nextAdjustedEventNumber(numEvents);
+    // How many events till we've generated numInFlightAuctions?
+    long numEventsForAuctions =
+        (config.configuration.numInFlightAuctions * GeneratorConfig.PROPORTION_DENOMINATOR)
+        / GeneratorConfig.AUCTION_PROPORTION;
+    // When will the auction numInFlightAuctions beyond now be generated?
+    long futureAuction =
+        config.timestampAndInterEventDelayUsForEvent(currentEventNumber + numEventsForAuctions)
+            .getKey();
+    // System.out.printf("*** auction will be for %dms (%d events ahead) ***\n",
+    //     futureAuction - timestamp, numEventsForAuctions);
+    // Choose a length with average horizonMs.
+    long horizonMs = futureAuction - timestamp;
+    return 1L + nextLong(random, Math.max(horizonMs * 2, 1L));
+  }
+
+  /**
+   * Return a random {@code string} such that {@code currentSize + string.length()} is on average
+   * {@code averageSize}.
+   */
+  private static String nextExtra(Random random, int currentSize, int desiredAverageSize) {
+    if (currentSize > desiredAverageSize) {
+      return "";
+    }
+    desiredAverageSize -= currentSize;
+    int delta = (int) Math.round(desiredAverageSize * 0.2);
+    int minSize = desiredAverageSize - delta;
+    int desiredSize = minSize + (delta == 0 ? 0 : random.nextInt(2 * delta));
+    return nextExactString(random, desiredSize);
+  }
+
+  /** Return a random long from {@code [0, n)}. */
+  private static long nextLong(Random random, long n) {
+    if (n < Integer.MAX_VALUE) {
+      return random.nextInt((int) n);
+    } else {
+      // TODO: Very skewed distribution! Bad!
+      return Math.abs(random.nextLong()) % n;
+    }
+  }
+
+  /**
+   * Generate and return a random person with next available id.
+   */
+  private Person nextPerson(Random random, long timestamp) {
+    long id = lastBase0PersonId() + GeneratorConfig.FIRST_PERSON_ID;
+    String name = nextPersonName(random);
+    String email = nextEmail(random);
+    String creditCard = nextCreditCard(random);
+    String city = nextUSCity(random);
+    String state = nextUSState(random);
+    int currentSize =
+        8 + name.length() + email.length() + creditCard.length() + city.length() + state.length();
+    String extra = nextExtra(random, currentSize, config.configuration.avgPersonByteSize);
+    return new Person(id, name, email, creditCard, city, state, timestamp, extra);
+  }
+
+  /**
+   * Return a random person id (base 0).
+   */
+  private long nextBase0PersonId(Random random) {
+    // Choose a random person from any of the 'active' people, plus a few 'leads'.
+    // By limiting to 'active' we ensure the density of bids or auctions per person
+    // does not decrease over time for long running jobs.
+    // By choosing a person id ahead of the last valid person id we will make
+    // newPerson and newAuction events appear to have been swapped in time.
+    long numPeople = lastBase0PersonId() + 1;
+    long activePeople = Math.min(numPeople, config.configuration.numActivePeople);
+    long n = nextLong(random, activePeople + PERSON_ID_LEAD);
+    return numPeople - activePeople + n;
+  }
+
+  /**
+   * Return a random auction id (base 0).
+   */
+  private long nextBase0AuctionId(Random random) {
+    // Choose a random auction for any of those which are likely to still be in flight,
+    // plus a few 'leads'.
+    // Note that ideally we'd track non-expired auctions exactly, but that state
+    // is difficult to split.
+    long minAuction = Math.max(lastBase0AuctionId() - config.configuration.numInFlightAuctions, 0);
+    long maxAuction = lastBase0AuctionId();
+    return minAuction + nextLong(random, maxAuction - minAuction + 1 + AUCTION_ID_LEAD);
+  }
+
+  /**
+   * Generate and return a random auction with next available id.
+   */
+  private Auction nextAuction(Random random, long timestamp) {
+    long id = lastBase0AuctionId() + GeneratorConfig.FIRST_AUCTION_ID;
+
+    long seller;
+    // Here P(auction will be for a hot seller) = 1 - 1/hotSellersRatio.
+    if (random.nextInt(config.configuration.hotSellersRatio) > 0) {
+      // Choose the first person in the batch of last HOT_SELLER_RATIO people.
+      seller = (lastBase0PersonId() / HOT_SELLER_RATIO) * HOT_SELLER_RATIO;
+    } else {
+      seller = nextBase0PersonId(random);
+    }
+    seller += GeneratorConfig.FIRST_PERSON_ID;
+
+    long category = GeneratorConfig.FIRST_CATEGORY_ID + random.nextInt(NUM_CATEGORIES);
+    long initialBid = nextPrice(random);
+    long dateTime = timestamp;
+    long expires = timestamp + nextAuctionLengthMs(random, timestamp);
+    String name = nextString(random, 20);
+    String desc = nextString(random, 100);
+    long reserve = initialBid + nextPrice(random);
+    int currentSize = 8 + name.length() + desc.length() + 8 + 8 + 8 + 8 + 8;
+    String extra = nextExtra(random, currentSize, config.configuration.avgAuctionByteSize);
+    return new Auction(id, name, desc, initialBid, reserve, dateTime, expires, seller, category,
+        extra);
+  }
+
+  /**
+   * Generate and return a random bid with next available id.
+   */
+  private Bid nextBid(Random random, long timestamp) {
+    long auction;
+    // Here P(bid will be for a hot auction) = 1 - 1/hotAuctionRatio.
+    if (random.nextInt(config.configuration.hotAuctionRatio) > 0) {
+      // Choose the first auction in the batch of last HOT_AUCTION_RATIO auctions.
+      auction = (lastBase0AuctionId() / HOT_AUCTION_RATIO) * HOT_AUCTION_RATIO;
+    } else {
+      auction = nextBase0AuctionId(random);
+    }
+    auction += GeneratorConfig.FIRST_AUCTION_ID;
+
+    long bidder;
+    // Here P(bid will be by a hot bidder) = 1 - 1/hotBiddersRatio
+    if (random.nextInt(config.configuration.hotBiddersRatio) > 0) {
+      // Choose the second person (so hot bidders and hot sellers don't collide) in the batch of
+      // last HOT_BIDDER_RATIO people.
+      bidder = (lastBase0PersonId() / HOT_BIDDER_RATIO) * HOT_BIDDER_RATIO + 1;
+    } else {
+      bidder = nextBase0PersonId(random);
+    }
+    bidder += GeneratorConfig.FIRST_PERSON_ID;
+
+    long price = nextPrice(random);
+    int currentSize = 8 + 8 + 8 + 8;
+    String extra = nextExtra(random, currentSize, config.configuration.avgBidByteSize);
+    return new Bid(auction, bidder, price, timestamp, extra);
+  }
+
+  @Override
+  public boolean hasNext() {
+    return numEvents < config.maxEvents;
+  }
+
+  /**
+   * Return the next event. The outer timestamp is in wallclock time and corresponds to
+   * when the event should fire. The inner timestamp is in event-time and represents the
+   * time the event is purported to have taken place in the simulation.
+   */
+  public NextEvent nextEvent() {
+    if (wallclockBaseTime < 0) {
+      wallclockBaseTime = System.currentTimeMillis();
+    }
+    // When, in event time, we should generate the event. Monotonic.
+    long eventTimestamp =
+        config.timestampAndInterEventDelayUsForEvent(config.nextEventNumber(numEvents)).getKey();
+    // When, in event time, the event should say it was generated. Depending on outOfOrderGroupSize
+    // may have local jitter.
+    long adjustedEventTimestamp =
+        config.timestampAndInterEventDelayUsForEvent(config.nextAdjustedEventNumber(numEvents))
+            .getKey();
+    // The minimum of this and all future adjusted event timestamps. Accounts for jitter in
+    // the event timestamp.
+    long watermark =
+        config.timestampAndInterEventDelayUsForEvent(config.nextEventNumberForWatermark(numEvents))
+            .getKey();
+    // When, in wallclock time, we should emit the event.
+    long wallclockTimestamp = wallclockBaseTime + (eventTimestamp - getCurrentConfig().baseTime);
+
+    // Seed the random number generator with the next 'event id'.
+    Random random = new Random(getNextEventId());
+    long rem = getNextEventId() % GeneratorConfig.PROPORTION_DENOMINATOR;
+
+    Event event;
+    if (rem < GeneratorConfig.PERSON_PROPORTION) {
+      event = new Event(nextPerson(random, adjustedEventTimestamp));
+    } else if (rem < GeneratorConfig.PERSON_PROPORTION + GeneratorConfig.AUCTION_PROPORTION) {
+      event = new Event(nextAuction(random, adjustedEventTimestamp));
+    } else {
+      event = new Event(nextBid(random, adjustedEventTimestamp));
+    }
+
+    numEvents++;
+    return new NextEvent(wallclockTimestamp, adjustedEventTimestamp, event, watermark);
+  }
+
+  @Override
+  public TimestampedValue<Event> next() {
+    NextEvent next = nextEvent();
+    return TimestampedValue.of(next.event, new Instant(next.eventTimestamp));
+  }
+
+  @Override
+  public void remove() {
+    throw new UnsupportedOperationException();
+  }
+
+  /**
+   * Return how many microseconds till we emit the next event.
+   */
+  public long currentInterEventDelayUs() {
+    return config.timestampAndInterEventDelayUsForEvent(config.nextEventNumber(numEvents))
+        .getValue();
+  }
+
+  /**
+   * Return an estimate of fraction of output consumed.
+   */
+  public double getFractionConsumed() {
+    return (double) numEvents / config.maxEvents;
+  }
+
+  @Override
+  public String toString() {
+    return String.format("Generator{config:%s; numEvents:%d; wallclockBaseTime:%d}", config,
+        numEvents, wallclockBaseTime);
+  }
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/a7f9f7d0/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/sources/GeneratorConfig.java
----------------------------------------------------------------------
diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/sources/GeneratorConfig.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/sources/GeneratorConfig.java
new file mode 100644
index 0000000..3caaf51
--- /dev/null
+++ b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/sources/GeneratorConfig.java
@@ -0,0 +1,296 @@
+/*
+ * 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.beam.integration.nexmark.sources;
+
+import java.io.Serializable;
+import java.util.ArrayList;
+import java.util.List;
+
+import org.apache.beam.integration.nexmark.NexmarkConfiguration;
+import org.apache.beam.integration.nexmark.model.Event;
+import org.apache.beam.sdk.values.KV;
+
+/**
+ * Parameters controlling how {@link Generator} synthesizes {@link Event} elements.
+ */
+public class GeneratorConfig implements Serializable {
+  /**
+   * We start the ids at specific values to help ensure the queries find a match even on
+   * small synthesized dataset sizes.
+   */
+  public static final long FIRST_AUCTION_ID = 1000L;
+  public static final long FIRST_PERSON_ID = 1000L;
+  public static final long FIRST_CATEGORY_ID = 10L;
+
+  /**
+   * Proportions of people/auctions/bids to synthesize.
+   */
+  public static final int PERSON_PROPORTION = 1;
+  public static final int AUCTION_PROPORTION = 3;
+  public static final int BID_PROPORTION = 46;
+  public static final int PROPORTION_DENOMINATOR =
+      PERSON_PROPORTION + AUCTION_PROPORTION + BID_PROPORTION;
+
+  /**
+   * Environment options.
+   */
+  public final NexmarkConfiguration configuration;
+
+  /**
+   * Delay between events, in microseconds. If the array has more than one entry then
+   * the rate is changed every {@link #stepLengthSec}, and wraps around.
+   */
+  public final long[] interEventDelayUs;
+
+  /**
+   * Delay before changing the current inter-event delay.
+   */
+  public final long stepLengthSec;
+
+  /**
+   * Time for first event (ms since epoch).
+   */
+  public final long baseTime;
+
+  /**
+   * Event id of first event to be generated. Event ids are unique over all generators, and
+   * are used as a seed to generate each event's data.
+   */
+  public final long firstEventId;
+
+  /**
+   * Maximum number of events to generate.
+   */
+  public final long maxEvents;
+
+  /**
+   * First event number. Generators running in parallel time may share the same event number,
+   * and the event number is used to determine the event timestamp.
+   */
+  public final long firstEventNumber;
+
+  /**
+   * True period of epoch in milliseconds. Derived from above.
+   * (Ie time to run through cycle for all interEventDelayUs entries).
+   */
+  public final long epochPeriodMs;
+
+  /**
+   * Number of events per epoch. Derived from above.
+   * (Ie number of events to run through cycle for all interEventDelayUs entries).
+   */
+  public final long eventsPerEpoch;
+
+  public GeneratorConfig(
+      NexmarkConfiguration configuration, long baseTime, long firstEventId,
+      long maxEventsOrZero, long firstEventNumber) {
+    this.configuration = configuration;
+    this.interEventDelayUs = configuration.rateShape.interEventDelayUs(
+        configuration.firstEventRate, configuration.nextEventRate,
+        configuration.rateUnit, configuration.numEventGenerators);
+    this.stepLengthSec = configuration.rateShape.stepLengthSec(configuration.ratePeriodSec);
+    this.baseTime = baseTime;
+    this.firstEventId = firstEventId;
+    if (maxEventsOrZero == 0) {
+      // Scale maximum down to avoid overflow in getEstimatedSizeBytes.
+      this.maxEvents =
+          Long.MAX_VALUE / (PROPORTION_DENOMINATOR
+                            * Math.max(
+              Math.max(configuration.avgPersonByteSize, configuration.avgAuctionByteSize),
+              configuration.avgBidByteSize));
+    } else {
+      this.maxEvents = maxEventsOrZero;
+    }
+    this.firstEventNumber = firstEventNumber;
+
+    long eventsPerEpoch = 0;
+    long epochPeriodMs = 0;
+    if (interEventDelayUs.length > 1) {
+      for (int i = 0; i < interEventDelayUs.length; i++) {
+        long numEventsForThisCycle = (stepLengthSec * 1_000_000L) / interEventDelayUs[i];
+        eventsPerEpoch += numEventsForThisCycle;
+        epochPeriodMs += (numEventsForThisCycle * interEventDelayUs[i]) / 1000L;
+      }
+    }
+    this.eventsPerEpoch = eventsPerEpoch;
+    this.epochPeriodMs = epochPeriodMs;
+  }
+
+  /**
+   * Return a clone of this config.
+   */
+  @Override
+  public GeneratorConfig clone() {
+    return new GeneratorConfig(configuration, baseTime, firstEventId, maxEvents, firstEventNumber);
+  }
+
+  /**
+   * Return clone of this config except with given parameters.
+   */
+  public GeneratorConfig cloneWith(long firstEventId, long maxEvents, long firstEventNumber) {
+    return new GeneratorConfig(configuration, baseTime, firstEventId, maxEvents, firstEventNumber);
+  }
+
+  /**
+   * Split this config into {@code n} sub-configs with roughly equal number of
+   * possible events, but distinct value spaces. The generators will run on parallel timelines.
+   * This config should no longer be used.
+   */
+  public List<GeneratorConfig> split(int n) {
+    List<GeneratorConfig> results = new ArrayList<>();
+    if (n == 1) {
+      // No split required.
+      results.add(this);
+    } else {
+      long subMaxEvents = maxEvents / n;
+      long subFirstEventId = firstEventId;
+      for (int i = 0; i < n; i++) {
+        if (i == n - 1) {
+          // Don't loose any events to round-down.
+          subMaxEvents = maxEvents - subMaxEvents * (n - 1);
+        }
+        results.add(cloneWith(subFirstEventId, subMaxEvents, firstEventNumber));
+        subFirstEventId += subMaxEvents;
+      }
+    }
+    return results;
+  }
+
+  /**
+   * Return an estimate of the bytes needed by {@code numEvents}.
+   */
+  public long estimatedBytesForEvents(long numEvents) {
+    long numPersons =
+        (numEvents * GeneratorConfig.PERSON_PROPORTION) / GeneratorConfig.PROPORTION_DENOMINATOR;
+    long numAuctions = (numEvents * AUCTION_PROPORTION) / PROPORTION_DENOMINATOR;
+    long numBids = (numEvents * BID_PROPORTION) / PROPORTION_DENOMINATOR;
+    return numPersons * configuration.avgPersonByteSize
+           + numAuctions * configuration.avgAuctionByteSize
+           + numBids * configuration.avgBidByteSize;
+  }
+
+  /**
+   * Return an estimate of the byte-size of all events a generator for this config would yield.
+   */
+  public long getEstimatedSizeBytes() {
+    return estimatedBytesForEvents(maxEvents);
+  }
+
+  /**
+   * Return the first 'event id' which could be generated from this config. Though events don't
+   * have ids we can simulate them to help bookkeeping.
+   */
+  public long getStartEventId() {
+    return firstEventId + firstEventNumber;
+  }
+
+  /**
+   * Return one past the last 'event id' which could be generated from this config.
+   */
+  public long getStopEventId() {
+    return firstEventId + firstEventNumber + maxEvents;
+  }
+
+  /**
+   * Return the next event number for a generator which has so far emitted {@code numEvents}.
+   */
+  public long nextEventNumber(long numEvents) {
+    return firstEventNumber + numEvents;
+  }
+
+  /**
+   * Return the next event number for a generator which has so far emitted {@code numEvents},
+   * but adjusted to account for {@code outOfOrderGroupSize}.
+   */
+  public long nextAdjustedEventNumber(long numEvents) {
+    long n = configuration.outOfOrderGroupSize;
+    long eventNumber = nextEventNumber(numEvents);
+    long base = (eventNumber / n) * n;
+    long offset = (eventNumber * 953) % n;
+    return base + offset;
+  }
+
+  /**
+   * Return the event number who's event time will be a suitable watermark for
+   * a generator which has so far emitted {@code numEvents}.
+   */
+  public long nextEventNumberForWatermark(long numEvents) {
+    long n = configuration.outOfOrderGroupSize;
+    long eventNumber = nextEventNumber(numEvents);
+    return (eventNumber / n) * n;
+  }
+
+  /**
+   * What timestamp should the event with {@code eventNumber} have for this generator? And
+   * what inter-event delay (in microseconds) is current?
+   */
+  public KV<Long, Long> timestampAndInterEventDelayUsForEvent(long eventNumber) {
+    if (interEventDelayUs.length == 1) {
+      long timestamp = baseTime + (eventNumber * interEventDelayUs[0]) / 1000L;
+      return KV.of(timestamp, interEventDelayUs[0]);
+    }
+
+    long epoch = eventNumber / eventsPerEpoch;
+    long n = eventNumber % eventsPerEpoch;
+    long offsetInEpochMs = 0;
+    for (int i = 0; i < interEventDelayUs.length; i++) {
+      long numEventsForThisCycle = (stepLengthSec * 1_000_000L) / interEventDelayUs[i];
+      if (n < numEventsForThisCycle) {
+        long offsetInCycleUs = n * interEventDelayUs[i];
+        long timestamp =
+            baseTime + epoch * epochPeriodMs + offsetInEpochMs + (offsetInCycleUs / 1000L);
+        return KV.of(timestamp, interEventDelayUs[i]);
+      }
+      n -= numEventsForThisCycle;
+      offsetInEpochMs += (numEventsForThisCycle * interEventDelayUs[i]) / 1000L;
+    }
+    throw new RuntimeException("internal eventsPerEpoch incorrect"); // can't reach
+  }
+
+  @Override
+  public String toString() {
+    StringBuilder sb = new StringBuilder();
+    sb.append("GeneratorConfig");
+    sb.append("{configuration:");
+    sb.append(configuration.toString());
+    sb.append(";interEventDelayUs=[");
+    for (int i = 0; i < interEventDelayUs.length; i++) {
+      if (i > 0) {
+        sb.append(",");
+      }
+      sb.append(interEventDelayUs[i]);
+    }
+    sb.append("]");
+    sb.append(";stepLengthSec:");
+    sb.append(stepLengthSec);
+    sb.append(";baseTime:");
+    sb.append(baseTime);
+    sb.append(";firstEventId:");
+    sb.append(firstEventId);
+    sb.append(";maxEvents:");
+    sb.append(maxEvents);
+    sb.append(";firstEventNumber:");
+    sb.append(firstEventNumber);
+    sb.append(";epochPeriodMs:");
+    sb.append(epochPeriodMs);
+    sb.append(";eventsPerEpoch:");
+    sb.append(eventsPerEpoch);
+    sb.append("}");
+    return sb.toString();
+  }
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/a7f9f7d0/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/sources/UnboundedEventSource.java
----------------------------------------------------------------------
diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/sources/UnboundedEventSource.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/sources/UnboundedEventSource.java
new file mode 100644
index 0000000..286c576
--- /dev/null
+++ b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/sources/UnboundedEventSource.java
@@ -0,0 +1,330 @@
+/*
+ * 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.beam.integration.nexmark.sources;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.NoSuchElementException;
+import java.util.PriorityQueue;
+import java.util.Queue;
+import java.util.concurrent.ThreadLocalRandom;
+import javax.annotation.Nullable;
+
+import org.apache.beam.integration.nexmark.NexmarkUtils;
+import org.apache.beam.integration.nexmark.model.Event;
+import org.apache.beam.sdk.coders.Coder;
+import org.apache.beam.sdk.io.UnboundedSource;
+import org.apache.beam.sdk.options.PipelineOptions;
+import org.apache.beam.sdk.transforms.windowing.BoundedWindow;
+import org.apache.beam.sdk.values.TimestampedValue;
+import org.joda.time.Duration;
+import org.joda.time.Instant;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * A custom, unbounded source of event records.
+ *
+ * <p>If {@code isRateLimited} is true, events become available for return from the reader such
+ * that the overall rate respect the {@code interEventDelayUs} period if possible. Otherwise,
+ * events are returned every time the system asks for one.
+ */
+public class UnboundedEventSource extends UnboundedSource<Event, Generator.Checkpoint> {
+  private static final Duration BACKLOG_PERIOD = Duration.standardSeconds(30);
+  private static final Logger LOG = LoggerFactory.getLogger(UnboundedEventSource.class);
+
+  /** Configuration for generator to use when reading synthetic events. May be split. */
+  private final GeneratorConfig config;
+
+  /** How many unbounded sources to create. */
+  private final int numEventGenerators;
+
+  /** How many seconds to hold back the watermark. */
+  private final long watermarkHoldbackSec;
+
+  /** Are we rate limiting the events? */
+  private final boolean isRateLimited;
+
+  public UnboundedEventSource(GeneratorConfig config, int numEventGenerators,
+      long watermarkHoldbackSec, boolean isRateLimited) {
+    this.config = config;
+    this.numEventGenerators = numEventGenerators;
+    this.watermarkHoldbackSec = watermarkHoldbackSec;
+    this.isRateLimited = isRateLimited;
+  }
+
+  /** A reader to pull events from the generator. */
+  private class EventReader extends UnboundedReader<Event> {
+    /** Generator we are reading from. */
+    private final Generator generator;
+
+    /**
+     * Current watermark (ms since epoch). Initially set to beginning of time.
+     * Then updated to be the time of the next generated event.
+     * Then, once all events have been generated, set to the end of time.
+     */
+    private long watermark;
+
+    /**
+     * Current backlog (ms), as delay between timestamp of last returned event and the timestamp
+     * we should be up to according to wall-clock time. Used only for logging.
+     */
+    private long backlogDurationMs;
+
+    /**
+     * Current backlog, as estimated number of event bytes we are behind, or null if
+     * unknown. Reported to callers.
+     */
+    @Nullable
+    private Long backlogBytes;
+
+    /**
+     * Wallclock time (ms since epoch) we last reported the backlog, or -1 if never reported.
+     */
+    private long lastReportedBacklogWallclock;
+
+    /**
+     * Event time (ms since epoch) of pending event at last reported backlog, or -1 if never
+     * calculated.
+     */
+    private long timestampAtLastReportedBacklogMs;
+
+    /** Next event to make 'current' when wallclock time has advanced sufficiently. */
+    @Nullable
+    private TimestampedValue<Event> pendingEvent;
+
+    /** Wallclock time when {@link #pendingEvent} is due, or -1 if no pending event. */
+    private long pendingEventWallclockTime;
+
+    /** Current event to return from getCurrent. */
+    @Nullable
+    private TimestampedValue<Event> currentEvent;
+
+    /** Events which have been held back so as to force them to be late. */
+    private Queue<Generator.NextEvent> heldBackEvents = new PriorityQueue<>();
+
+    public EventReader(Generator generator) {
+      this.generator = generator;
+      watermark = NexmarkUtils.BEGINNING_OF_TIME.getMillis();
+      lastReportedBacklogWallclock = -1;
+      pendingEventWallclockTime = -1;
+      timestampAtLastReportedBacklogMs = -1;
+    }
+
+    public EventReader(GeneratorConfig config) {
+      this(new Generator(config));
+    }
+
+    @Override
+    public boolean start() {
+      LOG.trace("starting unbounded generator {}", generator);
+      return advance();
+    }
+
+
+    @Override
+    public boolean advance() {
+      long now = System.currentTimeMillis();
+
+      while (pendingEvent == null) {
+        if (!generator.hasNext() && heldBackEvents.isEmpty()) {
+          // No more events, EVER.
+          if (isRateLimited) {
+            updateBacklog(System.currentTimeMillis(), 0);
+          }
+          if (watermark < BoundedWindow.TIMESTAMP_MAX_VALUE.getMillis()) {
+            watermark = BoundedWindow.TIMESTAMP_MAX_VALUE.getMillis();
+            LOG.trace("stopped unbounded generator {}", generator);
+          }
+          return false;
+        }
+
+        Generator.NextEvent next = heldBackEvents.peek();
+        if (next != null && next.wallclockTimestamp <= now) {
+          // Time to use the held-back event.
+          heldBackEvents.poll();
+          LOG.debug("replaying held-back event {}ms behind watermark",
+                             watermark - next.eventTimestamp);
+        } else if (generator.hasNext()) {
+          next = generator.nextEvent();
+          if (isRateLimited && config.configuration.probDelayedEvent > 0.0
+              && config.configuration.occasionalDelaySec > 0
+              && ThreadLocalRandom.current().nextDouble() < config.configuration.probDelayedEvent) {
+            // We'll hold back this event and go around again.
+            long delayMs =
+                ThreadLocalRandom.current().nextLong(config.configuration.occasionalDelaySec * 1000)
+                + 1L;
+            LOG.debug("delaying event by {}ms", delayMs);
+            heldBackEvents.add(next.withDelay(delayMs));
+            continue;
+          }
+        } else {
+          // Waiting for held-back event to fire.
+          if (isRateLimited) {
+            updateBacklog(now, 0);
+          }
+          return false;
+        }
+
+        pendingEventWallclockTime = next.wallclockTimestamp;
+        pendingEvent = TimestampedValue.of(next.event, new Instant(next.eventTimestamp));
+        long newWatermark =
+            next.watermark - Duration.standardSeconds(watermarkHoldbackSec).getMillis();
+        if (newWatermark > watermark) {
+          watermark = newWatermark;
+        }
+      }
+
+      if (isRateLimited) {
+        if (pendingEventWallclockTime > now) {
+          // We want this event to fire in the future. Try again later.
+          updateBacklog(now, 0);
+          return false;
+        }
+        updateBacklog(now, now - pendingEventWallclockTime);
+      }
+
+      // This event is ready to fire.
+      currentEvent = pendingEvent;
+      pendingEvent = null;
+      return true;
+    }
+
+    private void updateBacklog(long now, long newBacklogDurationMs) {
+      backlogDurationMs = newBacklogDurationMs;
+      long interEventDelayUs = generator.currentInterEventDelayUs();
+      if (interEventDelayUs != 0) {
+        long backlogEvents = (backlogDurationMs * 1000 + interEventDelayUs - 1) / interEventDelayUs;
+        backlogBytes = generator.getCurrentConfig().estimatedBytesForEvents(backlogEvents);
+      }
+      if (lastReportedBacklogWallclock < 0
+          || now - lastReportedBacklogWallclock > BACKLOG_PERIOD.getMillis()) {
+        double timeDialation = Double.NaN;
+        if (pendingEvent != null
+            && lastReportedBacklogWallclock >= 0
+            && timestampAtLastReportedBacklogMs >= 0) {
+          long wallclockProgressionMs = now - lastReportedBacklogWallclock;
+          long eventTimeProgressionMs =
+              pendingEvent.getTimestamp().getMillis() - timestampAtLastReportedBacklogMs;
+          timeDialation = (double) eventTimeProgressionMs / (double) wallclockProgressionMs;
+        }
+        LOG.debug(
+            "unbounded generator backlog now {}ms ({} bytes) at {}us interEventDelay "
+            + "with {} time dilation",
+            backlogDurationMs, backlogBytes, interEventDelayUs, timeDialation);
+        lastReportedBacklogWallclock = now;
+        if (pendingEvent != null) {
+          timestampAtLastReportedBacklogMs = pendingEvent.getTimestamp().getMillis();
+        }
+      }
+    }
+
+    @Override
+    public Event getCurrent() {
+      if (currentEvent == null) {
+        throw new NoSuchElementException();
+      }
+      return currentEvent.getValue();
+    }
+
+    @Override
+    public Instant getCurrentTimestamp() {
+      if (currentEvent == null) {
+        throw new NoSuchElementException();
+      }
+      return currentEvent.getTimestamp();
+    }
+
+    @Override
+    public void close() {
+      // Nothing to close.
+    }
+
+    @Override
+    public UnboundedEventSource getCurrentSource() {
+      return UnboundedEventSource.this;
+    }
+
+    @Override
+    public Instant getWatermark() {
+      return new Instant(watermark);
+    }
+
+    @Override
+    public Generator.Checkpoint getCheckpointMark() {
+      return generator.toCheckpoint();
+    }
+
+    @Override
+    public long getSplitBacklogBytes() {
+      return backlogBytes == null ? BACKLOG_UNKNOWN : backlogBytes;
+    }
+
+    @Override
+    public String toString() {
+      return String.format("EventReader(%d, %d, %d)",
+          generator.getCurrentConfig().getStartEventId(), generator.getNextEventId(),
+          generator.getCurrentConfig().getStopEventId());
+    }
+  }
+
+  @Override
+  public Coder<Generator.Checkpoint> getCheckpointMarkCoder() {
+    return Generator.Checkpoint.CODER_INSTANCE;
+  }
+
+  @Override
+  public List<UnboundedEventSource> generateInitialSplits(
+      int desiredNumSplits, PipelineOptions options) {
+    LOG.trace("splitting unbounded source into {} sub-sources", numEventGenerators);
+    List<UnboundedEventSource> results = new ArrayList<>();
+    // Ignore desiredNumSplits and use numEventGenerators instead.
+    for (GeneratorConfig subConfig : config.split(numEventGenerators)) {
+      results.add(new UnboundedEventSource(subConfig, 1, watermarkHoldbackSec, isRateLimited));
+    }
+    return results;
+  }
+
+  @Override
+  public EventReader createReader(
+      PipelineOptions options, @Nullable Generator.Checkpoint checkpoint) {
+    if (checkpoint == null) {
+      LOG.trace("creating initial unbounded reader for {}", config);
+      return new EventReader(config);
+    } else {
+      LOG.trace("resuming unbounded reader from {}", checkpoint);
+      return new EventReader(checkpoint.toGenerator(config));
+    }
+  }
+
+  @Override
+  public void validate() {
+    // Nothing to validate.
+  }
+
+  @Override
+  public Coder<Event> getDefaultOutputCoder() {
+    return Event.CODER;
+  }
+
+  @Override
+  public String toString() {
+    return String.format(
+        "UnboundedEventSource(%d, %d)", config.getStartEventId(), config.getStopEventId());
+  }
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/a7f9f7d0/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/sources/package-info.java
----------------------------------------------------------------------
diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/sources/package-info.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/sources/package-info.java
new file mode 100644
index 0000000..ceaec9d
--- /dev/null
+++ b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/sources/package-info.java
@@ -0,0 +1,22 @@
+/*
+ * 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.
+ */
+
+/**
+ * Nexmark Synthetic Sources.
+ */
+package org.apache.beam.integration.nexmark.sources;

http://git-wip-us.apache.org/repos/asf/beam/blob/a7f9f7d0/integration/java/nexmark/src/main/resources/log4j.properties
----------------------------------------------------------------------
diff --git a/integration/java/nexmark/src/main/resources/log4j.properties b/integration/java/nexmark/src/main/resources/log4j.properties
index bc09794..30d0a9d 100644
--- a/integration/java/nexmark/src/main/resources/log4j.properties
+++ b/integration/java/nexmark/src/main/resources/log4j.properties
@@ -22,9 +22,13 @@ log4j.appender.console.target=System.err
 log4j.appender.console.layout=org.apache.log4j.PatternLayout
 log4j.appender.console.layout.ConversionPattern=%d{yy/MM/dd HH:mm:ss} %p %c: %m%n
 
+# General Beam loggers
 log4j.logger.org.apache.beam.runners.direct=WARN
 log4j.logger.org.apache.beam.sdk=WARN
 
+# Nexmark specific
+log4j.logger.org.apache.beam.integration.nexmark=ALL
+
 # Settings to quiet third party logs that are too verbose
 log4j.logger.org.spark_project.jetty=WARN
 log4j.logger.org.spark_project.jetty.util.component.AbstractLifeCycle=ERROR

http://git-wip-us.apache.org/repos/asf/beam/blob/a7f9f7d0/integration/java/nexmark/src/test/java/org/apache/beam/integration/nexmark/BoundedEventSourceTest.java
----------------------------------------------------------------------
diff --git a/integration/java/nexmark/src/test/java/org/apache/beam/integration/nexmark/BoundedEventSourceTest.java b/integration/java/nexmark/src/test/java/org/apache/beam/integration/nexmark/BoundedEventSourceTest.java
deleted file mode 100644
index 77957e5..0000000
--- a/integration/java/nexmark/src/test/java/org/apache/beam/integration/nexmark/BoundedEventSourceTest.java
+++ /dev/null
@@ -1,70 +0,0 @@
-/*
- * 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.beam.integration.nexmark;
-
-import org.apache.beam.runners.dataflow.options.DataflowPipelineOptions;
-import org.apache.beam.sdk.options.PipelineOptionsFactory;
-import org.apache.beam.sdk.testing.SourceTestUtils;
-
-import org.junit.Test;
-import org.junit.runner.RunWith;
-import org.junit.runners.JUnit4;
-
-/**
- * Test {@link BoundedEventSource}.
- */
-@RunWith(JUnit4.class)
-public class BoundedEventSourceTest {
-  private GeneratorConfig makeConfig(long n) {
-    return new GeneratorConfig(
-        NexmarkConfiguration.DEFAULT, System.currentTimeMillis(), 0, n, 0);
-  }
-
-  @Test
-  public void sourceAndReadersWork() throws Exception {
-    DataflowPipelineOptions options = PipelineOptionsFactory.as(DataflowPipelineOptions.class);
-    long n = 200L;
-    BoundedEventSource source = new BoundedEventSource(makeConfig(n), 1);
-
-    SourceTestUtils.assertUnstartedReaderReadsSameAsItsSource(
-        source.createReader(options), options);
-  }
-
-  @Test
-  public void splitAtFractionRespectsContract() throws Exception {
-    DataflowPipelineOptions options = PipelineOptionsFactory.as(DataflowPipelineOptions.class);
-    long n = 20L;
-    BoundedEventSource source = new BoundedEventSource(makeConfig(n), 1);
-
-    // Can't split if already consumed.
-    SourceTestUtils.assertSplitAtFractionFails(source, 10, 0.3, options);
-
-    SourceTestUtils.assertSplitAtFractionSucceedsAndConsistent(source, 5, 0.3, options);
-
-    SourceTestUtils.assertSplitAtFractionExhaustive(source, options);
-  }
-
-  @Test
-  public void splitIntoBundlesRespectsContract() throws Exception {
-    DataflowPipelineOptions options = PipelineOptionsFactory.as(DataflowPipelineOptions.class);
-    long n = 200L;
-    BoundedEventSource source = new BoundedEventSource(makeConfig(n), 1);
-    SourceTestUtils.assertSourcesEqualReferenceSource(
-        source, source.splitIntoBundles(10, options), options);
-  }
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/a7f9f7d0/integration/java/nexmark/src/test/java/org/apache/beam/integration/nexmark/GeneratorTest.java
----------------------------------------------------------------------
diff --git a/integration/java/nexmark/src/test/java/org/apache/beam/integration/nexmark/GeneratorTest.java b/integration/java/nexmark/src/test/java/org/apache/beam/integration/nexmark/GeneratorTest.java
deleted file mode 100644
index 4b821ea..0000000
--- a/integration/java/nexmark/src/test/java/org/apache/beam/integration/nexmark/GeneratorTest.java
+++ /dev/null
@@ -1,110 +0,0 @@
-/*
- * 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.beam.integration.nexmark;
-
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertTrue;
-
-import java.util.ArrayList;
-import java.util.Iterator;
-import java.util.List;
-
-import org.junit.Test;
-import org.junit.runner.RunWith;
-import org.junit.runners.JUnit4;
-
-/**
- * Test {@link Generator}.
- */
-@RunWith(JUnit4.class)
-public class GeneratorTest {
-  private GeneratorConfig makeConfig(long n) {
-    return new GeneratorConfig(
-        NexmarkConfiguration.DEFAULT, System.currentTimeMillis(), 0, n, 0);
-  }
-
-  private <T> long consume(long n, Iterator<T> itr) {
-    for (long i = 0; i < n; i++) {
-      assertTrue(itr.hasNext());
-      itr.next();
-    }
-    return n;
-  }
-
-  private <T> long consume(Iterator<T> itr) {
-    long n = 0;
-    while (itr.hasNext()) {
-      itr.next();
-      n++;
-    }
-    return n;
-  }
-
-  @Test
-  public void splitAtFractionPreservesOverallEventCount() {
-    long n = 55729L;
-    GeneratorConfig initialConfig = makeConfig(n);
-    long expected = initialConfig.getStopEventId() - initialConfig.getStartEventId();
-
-    long actual = 0;
-
-    Generator initialGenerator = new Generator(initialConfig);
-
-    // Consume some events.
-    actual += consume(5000, initialGenerator);
-
-
-    // Split once.
-    GeneratorConfig remainConfig1 = initialGenerator.splitAtEventId(9000L);
-    Generator remainGenerator1 = new Generator(remainConfig1);
-
-    // Consume some more events.
-    actual += consume(2000, initialGenerator);
-    actual += consume(3000, remainGenerator1);
-
-    // Split again.
-    GeneratorConfig remainConfig2 = remainGenerator1.splitAtEventId(30000L);
-    Generator remainGenerator2 = new Generator(remainConfig2);
-
-    // Run to completion.
-    actual += consume(initialGenerator);
-    actual += consume(remainGenerator1);
-    actual += consume(remainGenerator2);
-
-    assertEquals(expected, actual);
-  }
-
-  @Test
-  public void splitPreservesOverallEventCount() {
-    long n = 51237L;
-    GeneratorConfig initialConfig = makeConfig(n);
-    long expected = initialConfig.getStopEventId() - initialConfig.getStartEventId();
-
-    List<Generator> generators = new ArrayList<>();
-    for (GeneratorConfig subConfig : initialConfig.split(20)) {
-      generators.add(new Generator(subConfig));
-    }
-
-    long actual = 0;
-    for (Generator generator : generators) {
-      actual += consume(generator);
-    }
-
-    assertEquals(expected, actual);
-  }
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/a7f9f7d0/integration/java/nexmark/src/test/java/org/apache/beam/integration/nexmark/QueryTest.java
----------------------------------------------------------------------
diff --git a/integration/java/nexmark/src/test/java/org/apache/beam/integration/nexmark/QueryTest.java b/integration/java/nexmark/src/test/java/org/apache/beam/integration/nexmark/QueryTest.java
deleted file mode 100644
index e481eac..0000000
--- a/integration/java/nexmark/src/test/java/org/apache/beam/integration/nexmark/QueryTest.java
+++ /dev/null
@@ -1,107 +0,0 @@
-/*
- * 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.beam.integration.nexmark;
-
-import org.apache.beam.sdk.Pipeline;
-import org.apache.beam.sdk.testing.PAssert;
-import org.apache.beam.sdk.testing.TestPipeline;
-import org.apache.beam.sdk.values.PCollection;
-//import org.apache.beam.sdk.values.PCollection.IsBounded;
-import org.apache.beam.sdk.values.TimestampedValue;
-
-import org.junit.Ignore;
-import org.junit.Rule;
-import org.junit.Test;
-import org.junit.runner.RunWith;
-import org.junit.runners.JUnit4;
-
-/**
- * Test the various NEXMark queries yield results coherent with their models.
- */
-@RunWith(JUnit4.class)
-public class QueryTest {
-  private static final NexmarkConfiguration CONFIG = NexmarkConfiguration.DEFAULT.clone();
-  @Rule
-  public TestPipeline p = TestPipeline.create();
-
-  static {
-    //careful, results of tests are linked to numEvents value
-    CONFIG.numEvents = 100;
-  }
-
-  /** Test {@code query} matches {@code model}. */
-  private void queryMatchesModel(String name, NexmarkQuery query, NexmarkQueryModel model) {
-    NexmarkUtils.setupPipeline(NexmarkUtils.CoderStrategy.HAND, p);
-    PCollection<TimestampedValue<KnownSize>> results =
-        p.apply(name + ".ReadBounded", NexmarkUtils.batchEventsSource(CONFIG)).apply(query);
-    //TODO Ismael this should not be called explicitly
-    results.setIsBoundedInternal(PCollection.IsBounded.BOUNDED);
-    PAssert.that(results).satisfies(model.assertionFor());
-    p.run().waitUntilFinish();
-  }
-
-  @Test
-  public void query0MatchesModel() {
-    queryMatchesModel("Query0Test", new Query0(CONFIG), new Query0Model(CONFIG));
-  }
-
-  @Test
-  public void query1MatchesModel() {
-    queryMatchesModel("Query1Test", new Query1(CONFIG), new Query1Model(CONFIG));
-  }
-
-  @Test
-  public void query2MatchesModel() {
-    queryMatchesModel("Query2Test", new Query2(CONFIG), new Query2Model(CONFIG));
-  }
-
-  @Test
-  public void query3MatchesModel() {
-    queryMatchesModel("Query3Test", new Query3(CONFIG), new Query3Model(CONFIG));
-  }
-
-  @Test
-  public void query4MatchesModel() {
-    queryMatchesModel("Query4Test", new Query4(CONFIG), new Query4Model(CONFIG));
-  }
-
-  @Test
-  public void query5MatchesModel() {
-    queryMatchesModel("Query5Test", new Query5(CONFIG), new Query5Model(CONFIG));
-  }
-
-  @Test
-  public void query6MatchesModel() {
-    queryMatchesModel("Query6Test", new Query6(CONFIG), new Query6Model(CONFIG));
-  }
-
-  @Test
-  public void query7MatchesModel() {
-    queryMatchesModel("Query7Test", new Query7(CONFIG), new Query7Model(CONFIG));
-  }
-
-  @Test
-  public void query8MatchesModel() {
-    queryMatchesModel("Query8Test", new Query8(CONFIG), new Query8Model(CONFIG));
-  }
-
-  @Test
-  public void query9MatchesModel() {
-    queryMatchesModel("Query9Test", new Query9(CONFIG), new Query9Model(CONFIG));
-  }
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/a7f9f7d0/integration/java/nexmark/src/test/java/org/apache/beam/integration/nexmark/UnboundedEventSourceTest.java
----------------------------------------------------------------------
diff --git a/integration/java/nexmark/src/test/java/org/apache/beam/integration/nexmark/UnboundedEventSourceTest.java b/integration/java/nexmark/src/test/java/org/apache/beam/integration/nexmark/UnboundedEventSourceTest.java
deleted file mode 100644
index 35b3aed..0000000
--- a/integration/java/nexmark/src/test/java/org/apache/beam/integration/nexmark/UnboundedEventSourceTest.java
+++ /dev/null
@@ -1,108 +0,0 @@
-/*
- * 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.beam.integration.nexmark;
-
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertFalse;
-import static org.junit.Assert.assertTrue;
-
-import java.io.IOException;
-import java.util.HashSet;
-import java.util.Random;
-import java.util.Set;
-
-import org.apache.beam.sdk.Pipeline;
-import org.apache.beam.sdk.io.UnboundedSource.CheckpointMark;
-import org.apache.beam.sdk.io.UnboundedSource.UnboundedReader;
-import org.apache.beam.sdk.options.PipelineOptions;
-import org.apache.beam.sdk.testing.TestPipeline;
-
-import org.junit.Test;
-import org.junit.runner.RunWith;
-import org.junit.runners.JUnit4;
-
-/**
- * Test UnboundedEventSource.
- */
-@RunWith(JUnit4.class)
-public class UnboundedEventSourceTest {
-  private GeneratorConfig makeConfig(long n) {
-    return new GeneratorConfig(
-        NexmarkConfiguration.DEFAULT, System.currentTimeMillis(), 0, n, 0);
-  }
-
-  /**
-   * Helper for tracking which ids we've seen (so we can detect dups) and
-   * confirming reading events match the model events.
-   */
-  private static class EventIdChecker {
-    private Set<Long> seenPersonIds = new HashSet<>();
-    private Set<Long> seenAuctionIds = new HashSet<>();
-
-    public void add(Event event) {
-      if (event.newAuction != null) {
-        assertTrue(seenAuctionIds.add(event.newAuction.id));
-      } else if (event.newPerson != null) {
-        assertTrue(seenPersonIds.add(event.newPerson.id));
-      }
-    }
-
-    public void add(int n, UnboundedReader<Event> reader, Generator modelGenerator)
-        throws IOException {
-      for (int i = 0; i < n; i++) {
-        assertTrue(modelGenerator.hasNext());
-        Event modelEvent = modelGenerator.next().getValue();
-        assertTrue(reader.advance());
-        Event actualEvent = reader.getCurrent();
-        assertEquals(modelEvent.toString(), actualEvent.toString());
-        add(actualEvent);
-      }
-    }
-  }
-
-  /**
-   * Check aggressively checkpointing and resuming a reader gives us exactly the
-   * same event stream as reading directly.
-   */
-  @Test
-  public void resumeFromCheckpoint() throws IOException {
-    Random random = new Random(297);
-    int n = 47293;
-    GeneratorConfig config = makeConfig(n);
-    Generator modelGenerator = new Generator(config);
-
-    EventIdChecker checker = new EventIdChecker();
-    PipelineOptions options = TestPipeline.testingPipelineOptions();
-    Pipeline p = TestPipeline.create(options);
-    UnboundedEventSource source = new UnboundedEventSource(config, 1, 0, false);
-    UnboundedReader<Event> reader = source.createReader(options, null);
-
-    while (n > 0) {
-      int m = Math.min(459 + random.nextInt(455), n);
-      System.out.printf("reading %d...\n", m);
-      checker.add(m, reader, modelGenerator);
-      n -= m;
-      System.out.printf("splitting with %d remaining...\n", n);
-      CheckpointMark checkpointMark = reader.getCheckpointMark();
-      assertTrue(checkpointMark instanceof Generator.Checkpoint);
-      reader = source.createReader(options, (Generator.Checkpoint) checkpointMark);
-    }
-
-    assertFalse(reader.advance());
-  }
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/a7f9f7d0/integration/java/nexmark/src/test/java/org/apache/beam/integration/nexmark/queries/QueryTest.java
----------------------------------------------------------------------
diff --git a/integration/java/nexmark/src/test/java/org/apache/beam/integration/nexmark/queries/QueryTest.java b/integration/java/nexmark/src/test/java/org/apache/beam/integration/nexmark/queries/QueryTest.java
new file mode 100644
index 0000000..5cf4287
--- /dev/null
+++ b/integration/java/nexmark/src/test/java/org/apache/beam/integration/nexmark/queries/QueryTest.java
@@ -0,0 +1,111 @@
+/*
+ * 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.beam.integration.nexmark.queries;
+
+import org.apache.beam.integration.nexmark.NexmarkConfiguration;
+import org.apache.beam.integration.nexmark.NexmarkQuery;
+import org.apache.beam.integration.nexmark.NexmarkQueryModel;
+import org.apache.beam.integration.nexmark.NexmarkUtils;
+import org.apache.beam.integration.nexmark.model.KnownSize;
+import org.apache.beam.sdk.PipelineResult;
+import org.apache.beam.sdk.testing.PAssert;
+import org.apache.beam.sdk.testing.TestPipeline;
+import org.apache.beam.sdk.values.PCollection;
+import org.apache.beam.sdk.values.TimestampedValue;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.JUnit4;
+
+/**
+ * Test the various NEXMark queries yield results coherent with their models.
+ */
+@RunWith(JUnit4.class)
+public class QueryTest {
+  private static final NexmarkConfiguration CONFIG = NexmarkConfiguration.DEFAULT.clone();
+  @Rule
+  public TestPipeline p = TestPipeline.create();
+
+  static {
+    //careful, results of tests are linked to numEvents value
+    CONFIG.numEventGenerators = 1;
+    CONFIG.numEvents = 100;
+  }
+
+  /** Test {@code query} matches {@code model}. */
+  private void queryMatchesModel(String name, NexmarkQuery query, NexmarkQueryModel model) {
+    NexmarkUtils.setupPipeline(NexmarkUtils.CoderStrategy.HAND, p);
+    PCollection<TimestampedValue<KnownSize>> results =
+        p.apply(name + ".ReadBounded", NexmarkUtils.batchEventsSource(CONFIG)).apply(query);
+    //TODO Ismael this should not be called explicitly
+    results.setIsBoundedInternal(PCollection.IsBounded.BOUNDED);
+    PAssert.that(results).satisfies(model.assertionFor());
+    PipelineResult result = p.run();
+    result.waitUntilFinish();
+  }
+
+  @Test
+  public void query0MatchesModel() {
+    queryMatchesModel("Query0Test", new Query0(CONFIG), new Query0Model(CONFIG));
+  }
+
+  @Test
+  public void query1MatchesModel() {
+    queryMatchesModel("Query1Test", new Query1(CONFIG), new Query1Model(CONFIG));
+  }
+
+  @Test
+  public void query2MatchesModel() {
+    queryMatchesModel("Query2Test", new Query2(CONFIG), new Query2Model(CONFIG));
+  }
+
+  @Test
+  public void query3MatchesModel() {
+    queryMatchesModel("Query3Test", new Query3(CONFIG), new Query3Model(CONFIG));
+  }
+
+  @Test
+  public void query4MatchesModel() {
+    queryMatchesModel("Query4Test", new Query4(CONFIG), new Query4Model(CONFIG));
+  }
+
+  @Test
+  public void query5MatchesModel() {
+    queryMatchesModel("Query5Test", new Query5(CONFIG), new Query5Model(CONFIG));
+  }
+
+  @Test
+  public void query6MatchesModel() {
+    queryMatchesModel("Query6Test", new Query6(CONFIG), new Query6Model(CONFIG));
+  }
+
+  @Test
+  public void query7MatchesModel() {
+    queryMatchesModel("Query7Test", new Query7(CONFIG), new Query7Model(CONFIG));
+  }
+
+  @Test
+  public void query8MatchesModel() {
+    queryMatchesModel("Query8Test", new Query8(CONFIG), new Query8Model(CONFIG));
+  }
+
+  @Test
+  public void query9MatchesModel() {
+    queryMatchesModel("Query9Test", new Query9(CONFIG), new Query9Model(CONFIG));
+  }
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/a7f9f7d0/integration/java/nexmark/src/test/java/org/apache/beam/integration/nexmark/sources/BoundedEventSourceTest.java
----------------------------------------------------------------------
diff --git a/integration/java/nexmark/src/test/java/org/apache/beam/integration/nexmark/sources/BoundedEventSourceTest.java b/integration/java/nexmark/src/test/java/org/apache/beam/integration/nexmark/sources/BoundedEventSourceTest.java
new file mode 100644
index 0000000..3f85bab
--- /dev/null
+++ b/integration/java/nexmark/src/test/java/org/apache/beam/integration/nexmark/sources/BoundedEventSourceTest.java
@@ -0,0 +1,71 @@
+/*
+ * 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.beam.integration.nexmark.sources;
+
+import org.apache.beam.integration.nexmark.NexmarkConfiguration;
+import org.apache.beam.runners.dataflow.options.DataflowPipelineOptions;
+import org.apache.beam.sdk.options.PipelineOptionsFactory;
+import org.apache.beam.sdk.testing.SourceTestUtils;
+
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.JUnit4;
+
+/**
+ * Test {@link BoundedEventSource}.
+ */
+@RunWith(JUnit4.class)
+public class BoundedEventSourceTest {
+  private GeneratorConfig makeConfig(long n) {
+    return new GeneratorConfig(
+        NexmarkConfiguration.DEFAULT, System.currentTimeMillis(), 0, n, 0);
+  }
+
+  @Test
+  public void sourceAndReadersWork() throws Exception {
+    DataflowPipelineOptions options = PipelineOptionsFactory.as(DataflowPipelineOptions.class);
+    long n = 200L;
+    BoundedEventSource source = new BoundedEventSource(makeConfig(n), 1);
+
+    SourceTestUtils.assertUnstartedReaderReadsSameAsItsSource(
+        source.createReader(options), options);
+  }
+
+  @Test
+  public void splitAtFractionRespectsContract() throws Exception {
+    DataflowPipelineOptions options = PipelineOptionsFactory.as(DataflowPipelineOptions.class);
+    long n = 20L;
+    BoundedEventSource source = new BoundedEventSource(makeConfig(n), 1);
+
+    // Can't split if already consumed.
+    SourceTestUtils.assertSplitAtFractionFails(source, 10, 0.3, options);
+
+    SourceTestUtils.assertSplitAtFractionSucceedsAndConsistent(source, 5, 0.3, options);
+
+    SourceTestUtils.assertSplitAtFractionExhaustive(source, options);
+  }
+
+  @Test
+  public void splitIntoBundlesRespectsContract() throws Exception {
+    DataflowPipelineOptions options = PipelineOptionsFactory.as(DataflowPipelineOptions.class);
+    long n = 200L;
+    BoundedEventSource source = new BoundedEventSource(makeConfig(n), 1);
+    SourceTestUtils.assertSourcesEqualReferenceSource(
+        source, source.splitIntoBundles(10, options), options);
+  }
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/a7f9f7d0/integration/java/nexmark/src/test/java/org/apache/beam/integration/nexmark/sources/GeneratorTest.java
----------------------------------------------------------------------
diff --git a/integration/java/nexmark/src/test/java/org/apache/beam/integration/nexmark/sources/GeneratorTest.java b/integration/java/nexmark/src/test/java/org/apache/beam/integration/nexmark/sources/GeneratorTest.java
new file mode 100644
index 0000000..b0dff2f
--- /dev/null
+++ b/integration/java/nexmark/src/test/java/org/apache/beam/integration/nexmark/sources/GeneratorTest.java
@@ -0,0 +1,111 @@
+/*
+ * 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.beam.integration.nexmark.sources;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+
+import java.util.ArrayList;
+import java.util.Iterator;
+import java.util.List;
+
+import org.apache.beam.integration.nexmark.NexmarkConfiguration;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.JUnit4;
+
+/**
+ * Test {@link Generator}.
+ */
+@RunWith(JUnit4.class)
+public class GeneratorTest {
+  private GeneratorConfig makeConfig(long n) {
+    return new GeneratorConfig(
+        NexmarkConfiguration.DEFAULT, System.currentTimeMillis(), 0, n, 0);
+  }
+
+  private <T> long consume(long n, Iterator<T> itr) {
+    for (long i = 0; i < n; i++) {
+      assertTrue(itr.hasNext());
+      itr.next();
+    }
+    return n;
+  }
+
+  private <T> long consume(Iterator<T> itr) {
+    long n = 0;
+    while (itr.hasNext()) {
+      itr.next();
+      n++;
+    }
+    return n;
+  }
+
+  @Test
+  public void splitAtFractionPreservesOverallEventCount() {
+    long n = 55729L;
+    GeneratorConfig initialConfig = makeConfig(n);
+    long expected = initialConfig.getStopEventId() - initialConfig.getStartEventId();
+
+    long actual = 0;
+
+    Generator initialGenerator = new Generator(initialConfig);
+
+    // Consume some events.
+    actual += consume(5000, initialGenerator);
+
+
+    // Split once.
+    GeneratorConfig remainConfig1 = initialGenerator.splitAtEventId(9000L);
+    Generator remainGenerator1 = new Generator(remainConfig1);
+
+    // Consume some more events.
+    actual += consume(2000, initialGenerator);
+    actual += consume(3000, remainGenerator1);
+
+    // Split again.
+    GeneratorConfig remainConfig2 = remainGenerator1.splitAtEventId(30000L);
+    Generator remainGenerator2 = new Generator(remainConfig2);
+
+    // Run to completion.
+    actual += consume(initialGenerator);
+    actual += consume(remainGenerator1);
+    actual += consume(remainGenerator2);
+
+    assertEquals(expected, actual);
+  }
+
+  @Test
+  public void splitPreservesOverallEventCount() {
+    long n = 51237L;
+    GeneratorConfig initialConfig = makeConfig(n);
+    long expected = initialConfig.getStopEventId() - initialConfig.getStartEventId();
+
+    List<Generator> generators = new ArrayList<>();
+    for (GeneratorConfig subConfig : initialConfig.split(20)) {
+      generators.add(new Generator(subConfig));
+    }
+
+    long actual = 0;
+    for (Generator generator : generators) {
+      actual += consume(generator);
+    }
+
+    assertEquals(expected, actual);
+  }
+}


[47/55] [abbrv] beam git commit: Move module beam-integration-java-nexmark to beam-sdks-java-nexmark

Posted by ie...@apache.org.
http://git-wip-us.apache.org/repos/asf/beam/blob/f4333df7/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/WinningBidsSimulator.java
----------------------------------------------------------------------
diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/WinningBidsSimulator.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/WinningBidsSimulator.java
deleted file mode 100644
index 9624a9d..0000000
--- a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/WinningBidsSimulator.java
+++ /dev/null
@@ -1,206 +0,0 @@
-/*
- * 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.beam.integration.nexmark.queries;
-
-import java.util.ArrayList;
-import java.util.Iterator;
-import java.util.List;
-import java.util.Map;
-import java.util.Set;
-import java.util.TreeMap;
-import java.util.TreeSet;
-import javax.annotation.Nullable;
-
-import org.apache.beam.integration.nexmark.NexmarkConfiguration;
-import org.apache.beam.integration.nexmark.NexmarkUtils;
-import org.apache.beam.integration.nexmark.model.Auction;
-import org.apache.beam.integration.nexmark.model.AuctionBid;
-import org.apache.beam.integration.nexmark.model.Bid;
-import org.apache.beam.integration.nexmark.model.Event;
-import org.apache.beam.sdk.transforms.windowing.BoundedWindow;
-import org.apache.beam.sdk.values.TimestampedValue;
-import org.joda.time.Instant;
-
-/**
- * A simulator of the {@code WinningBids} query.
- */
-public class WinningBidsSimulator extends AbstractSimulator<Event, AuctionBid> {
-  /** Auctions currently still open, indexed by auction id. */
-  private final Map<Long, Auction> openAuctions;
-
-  /** The ids of auctions known to be closed. */
-  private final Set<Long> closedAuctions;
-
-  /** Current best valid bids for open auctions, indexed by auction id. */
-  private final Map<Long, Bid> bestBids;
-
-  /** Bids for auctions we havn't seen yet. */
-  private final List<Bid> bidsWithoutAuctions;
-
-  /**
-   * Timestamp of last new auction or bid event (ms since epoch).
-   */
-  private long lastTimestamp;
-
-  public WinningBidsSimulator(NexmarkConfiguration configuration) {
-    super(NexmarkUtils.standardEventIterator(configuration));
-    openAuctions = new TreeMap<>();
-    closedAuctions = new TreeSet<>();
-    bestBids = new TreeMap<>();
-    bidsWithoutAuctions = new ArrayList<>();
-    lastTimestamp = BoundedWindow.TIMESTAMP_MIN_VALUE.getMillis();
-  }
-
-  /**
-   * Try to account for {@code bid} in state. Return true if bid has now been
-   * accounted for by {@code bestBids}.
-   */
-  private boolean captureBestBid(Bid bid, boolean shouldLog) {
-    if (closedAuctions.contains(bid.auction)) {
-      // Ignore bids for known, closed auctions.
-      if (shouldLog) {
-        NexmarkUtils.info("closed auction: %s", bid);
-      }
-      return true;
-    }
-    Auction auction = openAuctions.get(bid.auction);
-    if (auction == null) {
-      // We don't have an auction for this bid yet, so can't determine if it is
-      // winning or not.
-      if (shouldLog) {
-        NexmarkUtils.info("pending auction: %s", bid);
-      }
-      return false;
-    }
-    if (bid.price < auction.reserve) {
-      // Bid price is too low.
-      if (shouldLog) {
-        NexmarkUtils.info("below reserve: %s", bid);
-      }
-      return true;
-    }
-    Bid existingBid = bestBids.get(bid.auction);
-    if (existingBid == null || Bid.PRICE_THEN_DESCENDING_TIME.compare(existingBid, bid) < 0) {
-      // We've found a (new) best bid for a known auction.
-      bestBids.put(bid.auction, bid);
-      if (shouldLog) {
-        NexmarkUtils.info("new winning bid: %s", bid);
-      }
-    } else {
-      if (shouldLog) {
-        NexmarkUtils.info("ignoring low bid: %s", bid);
-      }
-    }
-    return true;
-  }
-
-  /**
-   * Try to match bids without auctions to auctions.
-   */
-  private void flushBidsWithoutAuctions() {
-    Iterator<Bid> itr = bidsWithoutAuctions.iterator();
-    while (itr.hasNext()) {
-      Bid bid = itr.next();
-      if (captureBestBid(bid, false)) {
-        NexmarkUtils.info("bid now accounted for: %s", bid);
-        itr.remove();
-      }
-    }
-  }
-
-  /**
-   * Return the next winning bid for an expired auction relative to {@code timestamp}.
-   * Return null if no more winning bids, in which case all expired auctions will
-   * have been removed from our state. Retire auctions in order of expire time.
-   */
-  @Nullable
-  private TimestampedValue<AuctionBid> nextWinningBid(long timestamp) {
-    Map<Long, List<Long>> toBeRetired = new TreeMap<>();
-    for (Map.Entry<Long, Auction> entry : openAuctions.entrySet()) {
-      if (entry.getValue().expires <= timestamp) {
-        List<Long> idsAtTime = toBeRetired.get(entry.getValue().expires);
-        if (idsAtTime == null) {
-          idsAtTime = new ArrayList<>();
-          toBeRetired.put(entry.getValue().expires, idsAtTime);
-        }
-        idsAtTime.add(entry.getKey());
-      }
-    }
-    for (Map.Entry<Long, List<Long>> entry : toBeRetired.entrySet()) {
-      for (long id : entry.getValue()) {
-        Auction auction = openAuctions.get(id);
-        NexmarkUtils.info("retiring auction: %s", auction);
-        openAuctions.remove(id);
-        Bid bestBid = bestBids.get(id);
-        if (bestBid != null) {
-          TimestampedValue<AuctionBid> result =
-              TimestampedValue.of(new AuctionBid(auction, bestBid), new Instant(auction.expires));
-          NexmarkUtils.info("winning: %s", result);
-          return result;
-        }
-      }
-    }
-    return null;
-  }
-
-  @Override
-  protected void run() {
-    if (lastTimestamp > BoundedWindow.TIMESTAMP_MIN_VALUE.getMillis()) {
-      // We may have finally seen the auction a bid was intended for.
-      flushBidsWithoutAuctions();
-      TimestampedValue<AuctionBid> result = nextWinningBid(lastTimestamp);
-      if (result != null) {
-        addResult(result);
-        return;
-      }
-    }
-
-    TimestampedValue<Event> timestampedEvent = nextInput();
-    if (timestampedEvent == null) {
-      // No more events. Flush any still open auctions.
-      TimestampedValue<AuctionBid> result =
-          nextWinningBid(BoundedWindow.TIMESTAMP_MAX_VALUE.getMillis());
-      if (result == null) {
-        // We are done.
-        allDone();
-        return;
-      }
-      addResult(result);
-      return;
-    }
-
-    Event event = timestampedEvent.getValue();
-    if (event.newPerson != null) {
-      // Ignore new person events.
-      return;
-    }
-
-    lastTimestamp = timestampedEvent.getTimestamp().getMillis();
-    if (event.newAuction != null) {
-      // Add this new open auction to our state.
-      openAuctions.put(event.newAuction.id, event.newAuction);
-    } else {
-      if (!captureBestBid(event.bid, true)) {
-        // We don't know what to do with this bid yet.
-        NexmarkUtils.info("bid not yet accounted for: %s", event.bid);
-        bidsWithoutAuctions.add(event.bid);
-      }
-    }
-    // Keep looking for winning bids.
-  }
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/f4333df7/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/package-info.java
----------------------------------------------------------------------
diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/package-info.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/package-info.java
deleted file mode 100644
index 7a56733..0000000
--- a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/package-info.java
+++ /dev/null
@@ -1,22 +0,0 @@
-/*
- * 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.
- */
-
-/**
- * Nexmark Queries.
- */
-package org.apache.beam.integration.nexmark.queries;

http://git-wip-us.apache.org/repos/asf/beam/blob/f4333df7/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/sources/BoundedEventSource.java
----------------------------------------------------------------------
diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/sources/BoundedEventSource.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/sources/BoundedEventSource.java
deleted file mode 100644
index 43d6690..0000000
--- a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/sources/BoundedEventSource.java
+++ /dev/null
@@ -1,190 +0,0 @@
-/*
- * 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.beam.integration.nexmark.sources;
-
-import java.io.IOException;
-import java.util.ArrayList;
-import java.util.List;
-import java.util.NoSuchElementException;
-import javax.annotation.Nullable;
-import org.apache.beam.integration.nexmark.NexmarkUtils;
-import org.apache.beam.integration.nexmark.model.Event;
-import org.apache.beam.sdk.coders.Coder;
-import org.apache.beam.sdk.io.BoundedSource;
-import org.apache.beam.sdk.options.PipelineOptions;
-import org.apache.beam.sdk.values.TimestampedValue;
-import org.joda.time.Instant;
-
-/**
- * A custom, bounded source of event records.
- */
-public class BoundedEventSource extends BoundedSource<Event> {
-  /** Configuration we generate events against. */
-  private final GeneratorConfig config;
-
-  /** How many bounded sources to create. */
-  private final int numEventGenerators;
-
-  public BoundedEventSource(GeneratorConfig config, int numEventGenerators) {
-    this.config = config;
-    this.numEventGenerators = numEventGenerators;
-  }
-
-  /** A reader to pull events from the generator. */
-  private static class EventReader extends BoundedReader<Event> {
-    /**
-     * Event source we purporting to be reading from.
-     * (We can't use Java's capture-outer-class pointer since we must update
-     * this field on calls to splitAtFraction.)
-     */
-    private BoundedEventSource source;
-
-    /** Generator we are reading from. */
-    private final Generator generator;
-
-    private boolean reportedStop;
-
-    @Nullable
-    private TimestampedValue<Event> currentEvent;
-
-    public EventReader(BoundedEventSource source, GeneratorConfig config) {
-      this.source = source;
-      generator = new Generator(config);
-      reportedStop = false;
-    }
-
-    @Override
-    public synchronized boolean start() {
-      NexmarkUtils.info("starting bounded generator %s", generator);
-      return advance();
-    }
-
-    @Override
-    public synchronized boolean advance() {
-      if (!generator.hasNext()) {
-        // No more events.
-        if (!reportedStop) {
-          reportedStop = true;
-          NexmarkUtils.info("stopped bounded generator %s", generator);
-        }
-        return false;
-      }
-      currentEvent = generator.next();
-      return true;
-    }
-
-    @Override
-    public synchronized Event getCurrent() throws NoSuchElementException {
-      if (currentEvent == null) {
-        throw new NoSuchElementException();
-      }
-      return currentEvent.getValue();
-    }
-
-    @Override
-    public synchronized Instant getCurrentTimestamp() throws NoSuchElementException {
-      if (currentEvent == null) {
-        throw new NoSuchElementException();
-      }
-      return currentEvent.getTimestamp();
-    }
-
-    @Override
-    public void close() throws IOException {
-      // Nothing to close.
-    }
-
-    @Override
-    public synchronized Double getFractionConsumed() {
-      return generator.getFractionConsumed();
-    }
-
-    @Override
-    public synchronized BoundedSource<Event> getCurrentSource() {
-      return source;
-    }
-
-    @Override
-    @Nullable
-    public synchronized BoundedEventSource splitAtFraction(double fraction) {
-      long startId = generator.getCurrentConfig().getStartEventId();
-      long stopId = generator.getCurrentConfig().getStopEventId();
-      long size = stopId - startId;
-      long splitEventId = startId + Math.min((int) (size * fraction), size);
-      if (splitEventId <= generator.getNextEventId() || splitEventId == stopId) {
-        // Already passed this position or split results in left or right being empty.
-        NexmarkUtils.info("split failed for bounded generator %s at %f", generator, fraction);
-        return null;
-      }
-
-      NexmarkUtils.info("about to split bounded generator %s at %d", generator, splitEventId);
-
-      // Scale back the event space of the current generator, and return a generator config
-      // representing the event space we just 'stole' from the current generator.
-      GeneratorConfig remainingConfig = generator.splitAtEventId(splitEventId);
-
-      NexmarkUtils.info("split bounded generator into %s and %s", generator, remainingConfig);
-
-      // At this point
-      //   generator.events() ++ new Generator(remainingConfig).events()
-      //   == originalGenerator.events()
-
-      // We need a new source to represent the now smaller key space for this reader, so
-      // that we can maintain the invariant that
-      //   this.getCurrentSource().createReader(...)
-      // will yield the same output as this.
-      source = new BoundedEventSource(generator.getCurrentConfig(), source.numEventGenerators);
-
-      // Return a source from which we may read the 'stolen' event space.
-      return new BoundedEventSource(remainingConfig, source.numEventGenerators);
-    }
-  }
-
-  @Override
-  public List<BoundedEventSource> split(
-      long desiredBundleSizeBytes, PipelineOptions options) {
-    NexmarkUtils.info("slitting bounded source %s into %d sub-sources", config, numEventGenerators);
-    List<BoundedEventSource> results = new ArrayList<>();
-    // Ignore desiredBundleSizeBytes and use numEventGenerators instead.
-    for (GeneratorConfig subConfig : config.split(numEventGenerators)) {
-      results.add(new BoundedEventSource(subConfig, 1));
-    }
-    return results;
-  }
-
-  @Override
-  public long getEstimatedSizeBytes(PipelineOptions options) {
-    return config.getEstimatedSizeBytes();
-  }
-
-  @Override
-  public EventReader createReader(PipelineOptions options) {
-    NexmarkUtils.info("creating initial bounded reader for %s", config);
-    return new EventReader(this, config);
-  }
-
-  @Override
-  public void validate() {
-    // Nothing to validate.
-  }
-
-  @Override
-  public Coder<Event> getDefaultOutputCoder() {
-    return Event.CODER;
-  }
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/f4333df7/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/sources/Generator.java
----------------------------------------------------------------------
diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/sources/Generator.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/sources/Generator.java
deleted file mode 100644
index f6deceb..0000000
--- a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/sources/Generator.java
+++ /dev/null
@@ -1,609 +0,0 @@
-/*
- * 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.beam.integration.nexmark.sources;
-
-import static com.google.common.base.Preconditions.checkNotNull;
-
-import java.io.IOException;
-import java.io.InputStream;
-import java.io.OutputStream;
-import java.io.Serializable;
-import java.util.Arrays;
-import java.util.Iterator;
-import java.util.List;
-import java.util.Objects;
-import java.util.Random;
-import org.apache.beam.integration.nexmark.model.Auction;
-import org.apache.beam.integration.nexmark.model.Bid;
-import org.apache.beam.integration.nexmark.model.Event;
-import org.apache.beam.integration.nexmark.model.Person;
-import org.apache.beam.sdk.coders.Coder;
-import org.apache.beam.sdk.coders.CoderException;
-import org.apache.beam.sdk.coders.CustomCoder;
-import org.apache.beam.sdk.coders.VarLongCoder;
-import org.apache.beam.sdk.io.UnboundedSource;
-import org.apache.beam.sdk.values.TimestampedValue;
-import org.joda.time.Instant;
-
-/**
- * A generator for synthetic events. We try to make the data vaguely reasonable. We also ensure
- * most primary key/foreign key relations are correct. Eg: a {@link Bid} event will usually have
- * valid auction and bidder ids which can be joined to already-generated Auction and Person events.
- *
- * <p>To help with testing, we generate timestamps relative to a given {@code baseTime}. Each new
- * event is given a timestamp advanced from the previous timestamp by {@code interEventDelayUs}
- * (in microseconds). The event stream is thus fully deterministic and does not depend on
- * wallclock time.
- *
- * <p>This class implements {@link org.apache.beam.sdk.io.UnboundedSource.CheckpointMark}
- * so that we can resume generating events from a saved snapshot.
- */
-public class Generator implements Iterator<TimestampedValue<Event>>, Serializable {
-  /**
-   * Keep the number of categories small so the example queries will find results even with
-   * a small batch of events.
-   */
-  private static final int NUM_CATEGORIES = 5;
-
-  /** Smallest random string size. */
-  private static final int MIN_STRING_LENGTH = 3;
-
-  /**
-   * Keep the number of states small so that the example queries will find results even with
-   * a small batch of events.
-   */
-  private static final List<String> US_STATES = Arrays.asList(("AZ,CA,ID,OR,WA,WY").split(","));
-
-  private static final List<String> US_CITIES =
-      Arrays.asList(
-          ("Phoenix,Los Angeles,San Francisco,Boise,Portland,Bend,Redmond,Seattle,Kent,Cheyenne")
-              .split(","));
-
-  private static final List<String> FIRST_NAMES =
-      Arrays.asList(("Peter,Paul,Luke,John,Saul,Vicky,Kate,Julie,Sarah,Deiter,Walter").split(","));
-
-  private static final List<String> LAST_NAMES =
-      Arrays.asList(("Shultz,Abrams,Spencer,White,Bartels,Walton,Smith,Jones,Noris").split(","));
-
-  /**
-   * Number of yet-to-be-created people and auction ids allowed.
-   */
-  private static final int PERSON_ID_LEAD = 10;
-  private static final int AUCTION_ID_LEAD = 10;
-
-  /**
-   * Fraction of people/auctions which may be 'hot' sellers/bidders/auctions are 1
-   * over these values.
-   */
-  private static final int HOT_AUCTION_RATIO = 100;
-  private static final int HOT_SELLER_RATIO = 100;
-  private static final int HOT_BIDDER_RATIO = 100;
-
-  /**
-   * Just enough state to be able to restore a generator back to where it was checkpointed.
-   */
-  public static class Checkpoint implements UnboundedSource.CheckpointMark {
-    private static final Coder<Long> LONG_CODER = VarLongCoder.of();
-
-    /** Coder for this class. */
-    public static final Coder<Checkpoint> CODER_INSTANCE =
-        new CustomCoder<Checkpoint>() {
-          @Override public void encode(Checkpoint value, OutputStream outStream)
-          throws CoderException, IOException {
-            LONG_CODER.encode(value.numEvents, outStream);
-            LONG_CODER.encode(value.wallclockBaseTime, outStream);
-          }
-
-          @Override
-          public Checkpoint decode(InputStream inStream)
-              throws CoderException, IOException {
-            long numEvents = LONG_CODER.decode(inStream);
-            long wallclockBaseTime = LONG_CODER.decode(inStream);
-            return new Checkpoint(numEvents, wallclockBaseTime);
-          }
-          @Override public void verifyDeterministic() throws NonDeterministicException {}
-        };
-
-    private final long numEvents;
-    private final long wallclockBaseTime;
-
-    private Checkpoint(long numEvents, long wallclockBaseTime) {
-      this.numEvents = numEvents;
-      this.wallclockBaseTime = wallclockBaseTime;
-    }
-
-    public Generator toGenerator(GeneratorConfig config) {
-      return new Generator(config, numEvents, wallclockBaseTime);
-    }
-
-    @Override
-    public void finalizeCheckpoint() throws IOException {
-      // Nothing to finalize.
-    }
-
-    @Override
-    public String toString() {
-      return String.format("Generator.Checkpoint{numEvents:%d;wallclockBaseTime:%d}",
-          numEvents, wallclockBaseTime);
-    }
-  }
-
-  /**
-   * The next event and its various timestamps. Ordered by increasing wallclock timestamp, then
-   * (arbitrary but stable) event hash order.
-   */
-  public static class NextEvent implements Comparable<NextEvent> {
-    /** When, in wallclock time, should this event be emitted? */
-    public final long wallclockTimestamp;
-
-    /** When, in event time, should this event be considered to have occured? */
-    public final long eventTimestamp;
-
-    /** The event itself. */
-    public final Event event;
-
-    /** The minimum of this and all future event timestamps. */
-    public final long watermark;
-
-    public NextEvent(long wallclockTimestamp, long eventTimestamp, Event event, long watermark) {
-      this.wallclockTimestamp = wallclockTimestamp;
-      this.eventTimestamp = eventTimestamp;
-      this.event = event;
-      this.watermark = watermark;
-    }
-
-    /**
-     * Return a deep copy of next event with delay added to wallclock timestamp and
-     * event annotate as 'LATE'.
-     */
-    public NextEvent withDelay(long delayMs) {
-      return new NextEvent(
-          wallclockTimestamp + delayMs, eventTimestamp, event.withAnnotation("LATE"), watermark);
-    }
-
-    @Override public boolean equals(Object o) {
-      if (this == o) {
-        return true;
-      }
-      if (o == null || getClass() != o.getClass()) {
-        return false;
-      }
-
-      NextEvent nextEvent = (NextEvent) o;
-
-      return (wallclockTimestamp == nextEvent.wallclockTimestamp
-          && eventTimestamp == nextEvent.eventTimestamp
-          && watermark == nextEvent.watermark
-          && event.equals(nextEvent.event));
-    }
-
-    @Override public int hashCode() {
-      return Objects.hash(wallclockTimestamp, eventTimestamp, watermark, event);
-    }
-
-    @Override
-    public int compareTo(NextEvent other) {
-      int i = Long.compare(wallclockTimestamp, other.wallclockTimestamp);
-      if (i != 0) {
-        return i;
-      }
-      return Integer.compare(event.hashCode(), other.event.hashCode());
-    }
-  }
-
-  /**
-   * Configuration to generate events against. Note that it may be replaced by a call to
-   * {@link #splitAtEventId}.
-   */
-  private GeneratorConfig config;
-
-  /** Number of events generated by this generator. */
-  private long numEvents;
-
-  /**
-   * Wallclock time at which we emitted the first event (ms since epoch). Initially -1.
-   */
-  private long wallclockBaseTime;
-
-  private Generator(GeneratorConfig config, long numEvents, long wallclockBaseTime) {
-    checkNotNull(config);
-    this.config = config;
-    this.numEvents = numEvents;
-    this.wallclockBaseTime = wallclockBaseTime;
-  }
-
-  /**
-   * Create a fresh generator according to {@code config}.
-   */
-  public Generator(GeneratorConfig config) {
-    this(config, 0, -1);
-  }
-
-  /**
-   * Return a checkpoint for the current generator.
-   */
-  public Checkpoint toCheckpoint() {
-    return new Checkpoint(numEvents, wallclockBaseTime);
-  }
-
-  /**
-   * Return a deep copy of this generator.
-   */
-  public Generator copy() {
-    checkNotNull(config);
-    Generator result = new Generator(config, numEvents, wallclockBaseTime);
-    return result;
-  }
-
-  /**
-   * Return the current config for this generator. Note that configs may be replaced by {@link
-   * #splitAtEventId}.
-   */
-  public GeneratorConfig getCurrentConfig() {
-    return config;
-  }
-
-  /**
-   * Mutate this generator so that it will only generate events up to but not including
-   * {@code eventId}. Return a config to represent the events this generator will no longer yield.
-   * The generators will run in on a serial timeline.
-   */
-  public GeneratorConfig splitAtEventId(long eventId) {
-    long newMaxEvents = eventId - (config.firstEventId + config.firstEventNumber);
-    GeneratorConfig remainConfig = config.copyWith(config.firstEventId,
-        config.maxEvents - newMaxEvents, config.firstEventNumber + newMaxEvents);
-    config = config.copyWith(config.firstEventId, newMaxEvents, config.firstEventNumber);
-    return remainConfig;
-  }
-
-  /**
-   * Return the next 'event id'. Though events don't have ids we can simulate them to
-   * help with bookkeeping.
-   */
-  public long getNextEventId() {
-    return config.firstEventId + config.nextAdjustedEventNumber(numEvents);
-  }
-
-  /**
-   * Return the last valid person id (ignoring FIRST_PERSON_ID). Will be the current person id if
-   * due to generate a person.
-   */
-  private long lastBase0PersonId() {
-    long eventId = getNextEventId();
-    long epoch = eventId / GeneratorConfig.PROPORTION_DENOMINATOR;
-    long offset = eventId % GeneratorConfig.PROPORTION_DENOMINATOR;
-    if (offset >= GeneratorConfig.PERSON_PROPORTION) {
-      // About to generate an auction or bid.
-      // Go back to the last person generated in this epoch.
-      offset = GeneratorConfig.PERSON_PROPORTION - 1;
-    }
-    // About to generate a person.
-    return epoch * GeneratorConfig.PERSON_PROPORTION + offset;
-  }
-
-  /**
-   * Return the last valid auction id (ignoring FIRST_AUCTION_ID). Will be the current auction id if
-   * due to generate an auction.
-   */
-  private long lastBase0AuctionId() {
-    long eventId = getNextEventId();
-    long epoch = eventId / GeneratorConfig.PROPORTION_DENOMINATOR;
-    long offset = eventId % GeneratorConfig.PROPORTION_DENOMINATOR;
-    if (offset < GeneratorConfig.PERSON_PROPORTION) {
-      // About to generate a person.
-      // Go back to the last auction in the last epoch.
-      epoch--;
-      offset = GeneratorConfig.AUCTION_PROPORTION - 1;
-    } else if (offset >= GeneratorConfig.PERSON_PROPORTION + GeneratorConfig.AUCTION_PROPORTION) {
-      // About to generate a bid.
-      // Go back to the last auction generated in this epoch.
-      offset = GeneratorConfig.AUCTION_PROPORTION - 1;
-    } else {
-      // About to generate an auction.
-      offset -= GeneratorConfig.PERSON_PROPORTION;
-    }
-    return epoch * GeneratorConfig.AUCTION_PROPORTION + offset;
-  }
-
-  /** return a random US state. */
-  private static String nextUSState(Random random) {
-    return US_STATES.get(random.nextInt(US_STATES.size()));
-  }
-
-  /** Return a random US city. */
-  private static String nextUSCity(Random random) {
-    return US_CITIES.get(random.nextInt(US_CITIES.size()));
-  }
-
-  /** Return a random person name. */
-  private static String nextPersonName(Random random) {
-    return FIRST_NAMES.get(random.nextInt(FIRST_NAMES.size())) + " "
-        + LAST_NAMES.get(random.nextInt(LAST_NAMES.size()));
-  }
-
-  /** Return a random string of up to {@code maxLength}. */
-  private static String nextString(Random random, int maxLength) {
-    int len = MIN_STRING_LENGTH + random.nextInt(maxLength - MIN_STRING_LENGTH);
-    StringBuilder sb = new StringBuilder();
-    while (len-- > 0) {
-      if (random.nextInt(13) == 0) {
-        sb.append(' ');
-      } else {
-        sb.append((char) ('a' + random.nextInt(26)));
-      }
-    }
-    return sb.toString().trim();
-  }
-
-  /** Return a random string of exactly {@code length}. */
-  private static String nextExactString(Random random, int length) {
-    StringBuilder sb = new StringBuilder();
-    while (length-- > 0) {
-      sb.append((char) ('a' + random.nextInt(26)));
-    }
-    return sb.toString();
-  }
-
-  /** Return a random email address. */
-  private static String nextEmail(Random random) {
-    return nextString(random, 7) + "@" + nextString(random, 5) + ".com";
-  }
-
-  /** Return a random credit card number. */
-  private static String nextCreditCard(Random random) {
-    StringBuilder sb = new StringBuilder();
-    for (int i = 0; i < 4; i++) {
-      if (i > 0) {
-        sb.append(' ');
-      }
-      sb.append(String.format("%04d", random.nextInt(10000)));
-    }
-    return sb.toString();
-  }
-
-  /** Return a random price. */
-  private static long nextPrice(Random random) {
-    return Math.round(Math.pow(10.0, random.nextDouble() * 6.0) * 100.0);
-  }
-
-  /** Return a random time delay, in milliseconds, for length of auctions. */
-  private long nextAuctionLengthMs(Random random, long timestamp) {
-    // What's our current event number?
-    long currentEventNumber = config.nextAdjustedEventNumber(numEvents);
-    // How many events till we've generated numInFlightAuctions?
-    long numEventsForAuctions =
-        (config.configuration.numInFlightAuctions * GeneratorConfig.PROPORTION_DENOMINATOR)
-        / GeneratorConfig.AUCTION_PROPORTION;
-    // When will the auction numInFlightAuctions beyond now be generated?
-    long futureAuction =
-        config.timestampAndInterEventDelayUsForEvent(currentEventNumber + numEventsForAuctions)
-            .getKey();
-    // System.out.printf("*** auction will be for %dms (%d events ahead) ***\n",
-    //     futureAuction - timestamp, numEventsForAuctions);
-    // Choose a length with average horizonMs.
-    long horizonMs = futureAuction - timestamp;
-    return 1L + nextLong(random, Math.max(horizonMs * 2, 1L));
-  }
-
-  /**
-   * Return a random {@code string} such that {@code currentSize + string.length()} is on average
-   * {@code averageSize}.
-   */
-  private static String nextExtra(Random random, int currentSize, int desiredAverageSize) {
-    if (currentSize > desiredAverageSize) {
-      return "";
-    }
-    desiredAverageSize -= currentSize;
-    int delta = (int) Math.round(desiredAverageSize * 0.2);
-    int minSize = desiredAverageSize - delta;
-    int desiredSize = minSize + (delta == 0 ? 0 : random.nextInt(2 * delta));
-    return nextExactString(random, desiredSize);
-  }
-
-  /** Return a random long from {@code [0, n)}. */
-  private static long nextLong(Random random, long n) {
-    if (n < Integer.MAX_VALUE) {
-      return random.nextInt((int) n);
-    } else {
-      // WARNING: Very skewed distribution! Bad!
-      return Math.abs(random.nextLong() % n);
-    }
-  }
-
-  /**
-   * Generate and return a random person with next available id.
-   */
-  private Person nextPerson(Random random, long timestamp) {
-    long id = lastBase0PersonId() + GeneratorConfig.FIRST_PERSON_ID;
-    String name = nextPersonName(random);
-    String email = nextEmail(random);
-    String creditCard = nextCreditCard(random);
-    String city = nextUSCity(random);
-    String state = nextUSState(random);
-    int currentSize =
-        8 + name.length() + email.length() + creditCard.length() + city.length() + state.length();
-    String extra = nextExtra(random, currentSize, config.configuration.avgPersonByteSize);
-    return new Person(id, name, email, creditCard, city, state, timestamp, extra);
-  }
-
-  /**
-   * Return a random person id (base 0).
-   */
-  private long nextBase0PersonId(Random random) {
-    // Choose a random person from any of the 'active' people, plus a few 'leads'.
-    // By limiting to 'active' we ensure the density of bids or auctions per person
-    // does not decrease over time for long running jobs.
-    // By choosing a person id ahead of the last valid person id we will make
-    // newPerson and newAuction events appear to have been swapped in time.
-    long numPeople = lastBase0PersonId() + 1;
-    long activePeople = Math.min(numPeople, config.configuration.numActivePeople);
-    long n = nextLong(random, activePeople + PERSON_ID_LEAD);
-    return numPeople - activePeople + n;
-  }
-
-  /**
-   * Return a random auction id (base 0).
-   */
-  private long nextBase0AuctionId(Random random) {
-    // Choose a random auction for any of those which are likely to still be in flight,
-    // plus a few 'leads'.
-    // Note that ideally we'd track non-expired auctions exactly, but that state
-    // is difficult to split.
-    long minAuction = Math.max(lastBase0AuctionId() - config.configuration.numInFlightAuctions, 0);
-    long maxAuction = lastBase0AuctionId();
-    return minAuction + nextLong(random, maxAuction - minAuction + 1 + AUCTION_ID_LEAD);
-  }
-
-  /**
-   * Generate and return a random auction with next available id.
-   */
-  private Auction nextAuction(Random random, long timestamp) {
-    long id = lastBase0AuctionId() + GeneratorConfig.FIRST_AUCTION_ID;
-
-    long seller;
-    // Here P(auction will be for a hot seller) = 1 - 1/hotSellersRatio.
-    if (random.nextInt(config.configuration.hotSellersRatio) > 0) {
-      // Choose the first person in the batch of last HOT_SELLER_RATIO people.
-      seller = (lastBase0PersonId() / HOT_SELLER_RATIO) * HOT_SELLER_RATIO;
-    } else {
-      seller = nextBase0PersonId(random);
-    }
-    seller += GeneratorConfig.FIRST_PERSON_ID;
-
-    long category = GeneratorConfig.FIRST_CATEGORY_ID + random.nextInt(NUM_CATEGORIES);
-    long initialBid = nextPrice(random);
-    long expires = timestamp + nextAuctionLengthMs(random, timestamp);
-    String name = nextString(random, 20);
-    String desc = nextString(random, 100);
-    long reserve = initialBid + nextPrice(random);
-    int currentSize = 8 + name.length() + desc.length() + 8 + 8 + 8 + 8 + 8;
-    String extra = nextExtra(random, currentSize, config.configuration.avgAuctionByteSize);
-    return new Auction(id, name, desc, initialBid, reserve, timestamp, expires, seller, category,
-        extra);
-  }
-
-  /**
-   * Generate and return a random bid with next available id.
-   */
-  private Bid nextBid(Random random, long timestamp) {
-    long auction;
-    // Here P(bid will be for a hot auction) = 1 - 1/hotAuctionRatio.
-    if (random.nextInt(config.configuration.hotAuctionRatio) > 0) {
-      // Choose the first auction in the batch of last HOT_AUCTION_RATIO auctions.
-      auction = (lastBase0AuctionId() / HOT_AUCTION_RATIO) * HOT_AUCTION_RATIO;
-    } else {
-      auction = nextBase0AuctionId(random);
-    }
-    auction += GeneratorConfig.FIRST_AUCTION_ID;
-
-    long bidder;
-    // Here P(bid will be by a hot bidder) = 1 - 1/hotBiddersRatio
-    if (random.nextInt(config.configuration.hotBiddersRatio) > 0) {
-      // Choose the second person (so hot bidders and hot sellers don't collide) in the batch of
-      // last HOT_BIDDER_RATIO people.
-      bidder = (lastBase0PersonId() / HOT_BIDDER_RATIO) * HOT_BIDDER_RATIO + 1;
-    } else {
-      bidder = nextBase0PersonId(random);
-    }
-    bidder += GeneratorConfig.FIRST_PERSON_ID;
-
-    long price = nextPrice(random);
-    int currentSize = 8 + 8 + 8 + 8;
-    String extra = nextExtra(random, currentSize, config.configuration.avgBidByteSize);
-    return new Bid(auction, bidder, price, timestamp, extra);
-  }
-
-  @Override
-  public boolean hasNext() {
-    return numEvents < config.maxEvents;
-  }
-
-  /**
-   * Return the next event. The outer timestamp is in wallclock time and corresponds to
-   * when the event should fire. The inner timestamp is in event-time and represents the
-   * time the event is purported to have taken place in the simulation.
-   */
-  public NextEvent nextEvent() {
-    if (wallclockBaseTime < 0) {
-      wallclockBaseTime = System.currentTimeMillis();
-    }
-    // When, in event time, we should generate the event. Monotonic.
-    long eventTimestamp =
-        config.timestampAndInterEventDelayUsForEvent(config.nextEventNumber(numEvents)).getKey();
-    // When, in event time, the event should say it was generated. Depending on outOfOrderGroupSize
-    // may have local jitter.
-    long adjustedEventTimestamp =
-        config.timestampAndInterEventDelayUsForEvent(config.nextAdjustedEventNumber(numEvents))
-            .getKey();
-    // The minimum of this and all future adjusted event timestamps. Accounts for jitter in
-    // the event timestamp.
-    long watermark =
-        config.timestampAndInterEventDelayUsForEvent(config.nextEventNumberForWatermark(numEvents))
-            .getKey();
-    // When, in wallclock time, we should emit the event.
-    long wallclockTimestamp = wallclockBaseTime + (eventTimestamp - getCurrentConfig().baseTime);
-
-    // Seed the random number generator with the next 'event id'.
-    Random random = new Random(getNextEventId());
-    long rem = getNextEventId() % GeneratorConfig.PROPORTION_DENOMINATOR;
-
-    Event event;
-    if (rem < GeneratorConfig.PERSON_PROPORTION) {
-      event = new Event(nextPerson(random, adjustedEventTimestamp));
-    } else if (rem < GeneratorConfig.PERSON_PROPORTION + GeneratorConfig.AUCTION_PROPORTION) {
-      event = new Event(nextAuction(random, adjustedEventTimestamp));
-    } else {
-      event = new Event(nextBid(random, adjustedEventTimestamp));
-    }
-
-    numEvents++;
-    return new NextEvent(wallclockTimestamp, adjustedEventTimestamp, event, watermark);
-  }
-
-  @Override
-  public TimestampedValue<Event> next() {
-    NextEvent next = nextEvent();
-    return TimestampedValue.of(next.event, new Instant(next.eventTimestamp));
-  }
-
-  @Override
-  public void remove() {
-    throw new UnsupportedOperationException();
-  }
-
-  /**
-   * Return how many microseconds till we emit the next event.
-   */
-  public long currentInterEventDelayUs() {
-    return config.timestampAndInterEventDelayUsForEvent(config.nextEventNumber(numEvents))
-        .getValue();
-  }
-
-  /**
-   * Return an estimate of fraction of output consumed.
-   */
-  public double getFractionConsumed() {
-    return (double) numEvents / config.maxEvents;
-  }
-
-  @Override
-  public String toString() {
-    return String.format("Generator{config:%s; numEvents:%d; wallclockBaseTime:%d}", config,
-        numEvents, wallclockBaseTime);
-  }
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/f4333df7/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/sources/GeneratorConfig.java
----------------------------------------------------------------------
diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/sources/GeneratorConfig.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/sources/GeneratorConfig.java
deleted file mode 100644
index 95c276b..0000000
--- a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/sources/GeneratorConfig.java
+++ /dev/null
@@ -1,301 +0,0 @@
-/*
- * 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.beam.integration.nexmark.sources;
-
-import static com.google.common.base.Preconditions.checkNotNull;
-
-import java.io.Serializable;
-import java.util.ArrayList;
-import java.util.List;
-
-import org.apache.beam.integration.nexmark.NexmarkConfiguration;
-import org.apache.beam.integration.nexmark.model.Event;
-import org.apache.beam.sdk.values.KV;
-
-/**
- * Parameters controlling how {@link Generator} synthesizes {@link Event} elements.
- */
-public class GeneratorConfig implements Serializable {
-
-  /**
-   * We start the ids at specific values to help ensure the queries find a match even on
-   * small synthesized dataset sizes.
-   */
-  public static final long FIRST_AUCTION_ID = 1000L;
-  public static final long FIRST_PERSON_ID = 1000L;
-  public static final long FIRST_CATEGORY_ID = 10L;
-
-  /**
-   * Proportions of people/auctions/bids to synthesize.
-   */
-  public static final int PERSON_PROPORTION = 1;
-  public static final int AUCTION_PROPORTION = 3;
-  private static final int BID_PROPORTION = 46;
-  public static final int PROPORTION_DENOMINATOR =
-      PERSON_PROPORTION + AUCTION_PROPORTION + BID_PROPORTION;
-
-  /**
-   * Environment options.
-   */
-  public final NexmarkConfiguration configuration;
-
-  /**
-   * Delay between events, in microseconds. If the array has more than one entry then
-   * the rate is changed every {@link #stepLengthSec}, and wraps around.
-   */
-  private final long[] interEventDelayUs;
-
-  /**
-   * Delay before changing the current inter-event delay.
-   */
-  private final long stepLengthSec;
-
-  /**
-   * Time for first event (ms since epoch).
-   */
-  public final long baseTime;
-
-  /**
-   * Event id of first event to be generated. Event ids are unique over all generators, and
-   * are used as a seed to generate each event's data.
-   */
-  public final long firstEventId;
-
-  /**
-   * Maximum number of events to generate.
-   */
-  public final long maxEvents;
-
-  /**
-   * First event number. Generators running in parallel time may share the same event number,
-   * and the event number is used to determine the event timestamp.
-   */
-  public final long firstEventNumber;
-
-  /**
-   * True period of epoch in milliseconds. Derived from above.
-   * (Ie time to run through cycle for all interEventDelayUs entries).
-   */
-  private final long epochPeriodMs;
-
-  /**
-   * Number of events per epoch. Derived from above.
-   * (Ie number of events to run through cycle for all interEventDelayUs entries).
-   */
-  private final long eventsPerEpoch;
-
-  public GeneratorConfig(
-      NexmarkConfiguration configuration, long baseTime, long firstEventId,
-      long maxEventsOrZero, long firstEventNumber) {
-    this.configuration = configuration;
-    this.interEventDelayUs = configuration.rateShape.interEventDelayUs(
-        configuration.firstEventRate, configuration.nextEventRate,
-        configuration.rateUnit, configuration.numEventGenerators);
-    this.stepLengthSec = configuration.rateShape.stepLengthSec(configuration.ratePeriodSec);
-    this.baseTime = baseTime;
-    this.firstEventId = firstEventId;
-    if (maxEventsOrZero == 0) {
-      // Scale maximum down to avoid overflow in getEstimatedSizeBytes.
-      this.maxEvents =
-          Long.MAX_VALUE / (PROPORTION_DENOMINATOR
-                            * Math.max(
-              Math.max(configuration.avgPersonByteSize, configuration.avgAuctionByteSize),
-              configuration.avgBidByteSize));
-    } else {
-      this.maxEvents = maxEventsOrZero;
-    }
-    this.firstEventNumber = firstEventNumber;
-
-    long eventsPerEpoch = 0;
-    long epochPeriodMs = 0;
-    if (interEventDelayUs.length > 1) {
-      for (long interEventDelayU : interEventDelayUs) {
-        long numEventsForThisCycle = (stepLengthSec * 1_000_000L) / interEventDelayU;
-        eventsPerEpoch += numEventsForThisCycle;
-        epochPeriodMs += (numEventsForThisCycle * interEventDelayU) / 1000L;
-      }
-    }
-    this.eventsPerEpoch = eventsPerEpoch;
-    this.epochPeriodMs = epochPeriodMs;
-  }
-
-  /**
-   * Return a copy of this config.
-   */
-  public GeneratorConfig copy() {
-    GeneratorConfig result;
-      result = new GeneratorConfig(configuration, baseTime, firstEventId,
-          maxEvents, firstEventNumber);
-    return result;
-  }
-
-  /**
-   * Split this config into {@code n} sub-configs with roughly equal number of
-   * possible events, but distinct value spaces. The generators will run on parallel timelines.
-   * This config should no longer be used.
-   */
-  public List<GeneratorConfig> split(int n) {
-    List<GeneratorConfig> results = new ArrayList<>();
-    if (n == 1) {
-      // No split required.
-      results.add(this);
-    } else {
-      long subMaxEvents = maxEvents / n;
-      long subFirstEventId = firstEventId;
-      for (int i = 0; i < n; i++) {
-        if (i == n - 1) {
-          // Don't loose any events to round-down.
-          subMaxEvents = maxEvents - subMaxEvents * (n - 1);
-        }
-        results.add(copyWith(subFirstEventId, subMaxEvents, firstEventNumber));
-        subFirstEventId += subMaxEvents;
-      }
-    }
-    return results;
-  }
-
-  /**
-   * Return copy of this config except with given parameters.
-   */
-  public GeneratorConfig copyWith(long firstEventId, long maxEvents, long firstEventNumber) {
-    return new GeneratorConfig(configuration, baseTime, firstEventId, maxEvents, firstEventNumber);
-  }
-
-  /**
-   * Return an estimate of the bytes needed by {@code numEvents}.
-   */
-  public long estimatedBytesForEvents(long numEvents) {
-    long numPersons =
-        (numEvents * GeneratorConfig.PERSON_PROPORTION) / GeneratorConfig.PROPORTION_DENOMINATOR;
-    long numAuctions = (numEvents * AUCTION_PROPORTION) / PROPORTION_DENOMINATOR;
-    long numBids = (numEvents * BID_PROPORTION) / PROPORTION_DENOMINATOR;
-    return numPersons * configuration.avgPersonByteSize
-           + numAuctions * configuration.avgAuctionByteSize
-           + numBids * configuration.avgBidByteSize;
-  }
-
-  /**
-   * Return an estimate of the byte-size of all events a generator for this config would yield.
-   */
-  public long getEstimatedSizeBytes() {
-    return estimatedBytesForEvents(maxEvents);
-  }
-
-  /**
-   * Return the first 'event id' which could be generated from this config. Though events don't
-   * have ids we can simulate them to help bookkeeping.
-   */
-  public long getStartEventId() {
-    return firstEventId + firstEventNumber;
-  }
-
-  /**
-   * Return one past the last 'event id' which could be generated from this config.
-   */
-  public long getStopEventId() {
-    return firstEventId + firstEventNumber + maxEvents;
-  }
-
-  /**
-   * Return the next event number for a generator which has so far emitted {@code numEvents}.
-   */
-  public long nextEventNumber(long numEvents) {
-    return firstEventNumber + numEvents;
-  }
-
-  /**
-   * Return the next event number for a generator which has so far emitted {@code numEvents},
-   * but adjusted to account for {@code outOfOrderGroupSize}.
-   */
-  public long nextAdjustedEventNumber(long numEvents) {
-    long n = configuration.outOfOrderGroupSize;
-    long eventNumber = nextEventNumber(numEvents);
-    long base = (eventNumber / n) * n;
-    long offset = (eventNumber * 953) % n;
-    return base + offset;
-  }
-
-  /**
-   * Return the event number who's event time will be a suitable watermark for
-   * a generator which has so far emitted {@code numEvents}.
-   */
-  public long nextEventNumberForWatermark(long numEvents) {
-    long n = configuration.outOfOrderGroupSize;
-    long eventNumber = nextEventNumber(numEvents);
-    return (eventNumber / n) * n;
-  }
-
-  /**
-   * What timestamp should the event with {@code eventNumber} have for this generator? And
-   * what inter-event delay (in microseconds) is current?
-   */
-  public KV<Long, Long> timestampAndInterEventDelayUsForEvent(long eventNumber) {
-    if (interEventDelayUs.length == 1) {
-      long timestamp = baseTime + (eventNumber * interEventDelayUs[0]) / 1000L;
-      return KV.of(timestamp, interEventDelayUs[0]);
-    }
-
-    long epoch = eventNumber / eventsPerEpoch;
-    long n = eventNumber % eventsPerEpoch;
-    long offsetInEpochMs = 0;
-    for (long interEventDelayU : interEventDelayUs) {
-      long numEventsForThisCycle = (stepLengthSec * 1_000_000L) / interEventDelayU;
-      if (n < numEventsForThisCycle) {
-        long offsetInCycleUs = n * interEventDelayU;
-        long timestamp =
-            baseTime + epoch * epochPeriodMs + offsetInEpochMs + (offsetInCycleUs / 1000L);
-        return KV.of(timestamp, interEventDelayU);
-      }
-      n -= numEventsForThisCycle;
-      offsetInEpochMs += (numEventsForThisCycle * interEventDelayU) / 1000L;
-    }
-    throw new RuntimeException("internal eventsPerEpoch incorrect"); // can't reach
-  }
-
-  @Override
-  public String toString() {
-    StringBuilder sb = new StringBuilder();
-    sb.append("GeneratorConfig");
-    sb.append("{configuration:");
-    sb.append(configuration.toString());
-    sb.append(";interEventDelayUs=[");
-    for (int i = 0; i < interEventDelayUs.length; i++) {
-      if (i > 0) {
-        sb.append(",");
-      }
-      sb.append(interEventDelayUs[i]);
-    }
-    sb.append("]");
-    sb.append(";stepLengthSec:");
-    sb.append(stepLengthSec);
-    sb.append(";baseTime:");
-    sb.append(baseTime);
-    sb.append(";firstEventId:");
-    sb.append(firstEventId);
-    sb.append(";maxEvents:");
-    sb.append(maxEvents);
-    sb.append(";firstEventNumber:");
-    sb.append(firstEventNumber);
-    sb.append(";epochPeriodMs:");
-    sb.append(epochPeriodMs);
-    sb.append(";eventsPerEpoch:");
-    sb.append(eventsPerEpoch);
-    sb.append("}");
-    return sb.toString();
-  }
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/f4333df7/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/sources/UnboundedEventSource.java
----------------------------------------------------------------------
diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/sources/UnboundedEventSource.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/sources/UnboundedEventSource.java
deleted file mode 100644
index 09d945d..0000000
--- a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/sources/UnboundedEventSource.java
+++ /dev/null
@@ -1,330 +0,0 @@
-/*
- * 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.beam.integration.nexmark.sources;
-
-import java.util.ArrayList;
-import java.util.List;
-import java.util.NoSuchElementException;
-import java.util.PriorityQueue;
-import java.util.Queue;
-import java.util.concurrent.ThreadLocalRandom;
-import javax.annotation.Nullable;
-
-import org.apache.beam.integration.nexmark.NexmarkUtils;
-import org.apache.beam.integration.nexmark.model.Event;
-import org.apache.beam.sdk.coders.Coder;
-import org.apache.beam.sdk.io.UnboundedSource;
-import org.apache.beam.sdk.options.PipelineOptions;
-import org.apache.beam.sdk.transforms.windowing.BoundedWindow;
-import org.apache.beam.sdk.values.TimestampedValue;
-import org.joda.time.Duration;
-import org.joda.time.Instant;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-/**
- * A custom, unbounded source of event records.
- *
- * <p>If {@code isRateLimited} is true, events become available for return from the reader such
- * that the overall rate respect the {@code interEventDelayUs} period if possible. Otherwise,
- * events are returned every time the system asks for one.
- */
-public class UnboundedEventSource extends UnboundedSource<Event, Generator.Checkpoint> {
-  private static final Duration BACKLOG_PERIOD = Duration.standardSeconds(30);
-  private static final Logger LOG = LoggerFactory.getLogger(UnboundedEventSource.class);
-
-  /** Configuration for generator to use when reading synthetic events. May be split. */
-  private final GeneratorConfig config;
-
-  /** How many unbounded sources to create. */
-  private final int numEventGenerators;
-
-  /** How many seconds to hold back the watermark. */
-  private final long watermarkHoldbackSec;
-
-  /** Are we rate limiting the events? */
-  private final boolean isRateLimited;
-
-  public UnboundedEventSource(GeneratorConfig config, int numEventGenerators,
-      long watermarkHoldbackSec, boolean isRateLimited) {
-    this.config = config;
-    this.numEventGenerators = numEventGenerators;
-    this.watermarkHoldbackSec = watermarkHoldbackSec;
-    this.isRateLimited = isRateLimited;
-  }
-
-  /** A reader to pull events from the generator. */
-  private class EventReader extends UnboundedReader<Event> {
-    /** Generator we are reading from. */
-    private final Generator generator;
-
-    /**
-     * Current watermark (ms since epoch). Initially set to beginning of time.
-     * Then updated to be the time of the next generated event.
-     * Then, once all events have been generated, set to the end of time.
-     */
-    private long watermark;
-
-    /**
-     * Current backlog (ms), as delay between timestamp of last returned event and the timestamp
-     * we should be up to according to wall-clock time. Used only for logging.
-     */
-    private long backlogDurationMs;
-
-    /**
-     * Current backlog, as estimated number of event bytes we are behind, or null if
-     * unknown. Reported to callers.
-     */
-    @Nullable
-    private Long backlogBytes;
-
-    /**
-     * Wallclock time (ms since epoch) we last reported the backlog, or -1 if never reported.
-     */
-    private long lastReportedBacklogWallclock;
-
-    /**
-     * Event time (ms since epoch) of pending event at last reported backlog, or -1 if never
-     * calculated.
-     */
-    private long timestampAtLastReportedBacklogMs;
-
-    /** Next event to make 'current' when wallclock time has advanced sufficiently. */
-    @Nullable
-    private TimestampedValue<Event> pendingEvent;
-
-    /** Wallclock time when {@link #pendingEvent} is due, or -1 if no pending event. */
-    private long pendingEventWallclockTime;
-
-    /** Current event to return from getCurrent. */
-    @Nullable
-    private TimestampedValue<Event> currentEvent;
-
-    /** Events which have been held back so as to force them to be late. */
-    private final Queue<Generator.NextEvent> heldBackEvents = new PriorityQueue<>();
-
-    public EventReader(Generator generator) {
-      this.generator = generator;
-      watermark = NexmarkUtils.BEGINNING_OF_TIME.getMillis();
-      lastReportedBacklogWallclock = -1;
-      pendingEventWallclockTime = -1;
-      timestampAtLastReportedBacklogMs = -1;
-    }
-
-    public EventReader(GeneratorConfig config) {
-      this(new Generator(config));
-    }
-
-    @Override
-    public boolean start() {
-      LOG.trace("starting unbounded generator {}", generator);
-      return advance();
-    }
-
-
-    @Override
-    public boolean advance() {
-      long now = System.currentTimeMillis();
-
-      while (pendingEvent == null) {
-        if (!generator.hasNext() && heldBackEvents.isEmpty()) {
-          // No more events, EVER.
-          if (isRateLimited) {
-            updateBacklog(System.currentTimeMillis(), 0);
-          }
-          if (watermark < BoundedWindow.TIMESTAMP_MAX_VALUE.getMillis()) {
-            watermark = BoundedWindow.TIMESTAMP_MAX_VALUE.getMillis();
-            LOG.trace("stopped unbounded generator {}", generator);
-          }
-          return false;
-        }
-
-        Generator.NextEvent next = heldBackEvents.peek();
-        if (next != null && next.wallclockTimestamp <= now) {
-          // Time to use the held-back event.
-          heldBackEvents.poll();
-          LOG.debug("replaying held-back event {}ms behind watermark",
-                             watermark - next.eventTimestamp);
-        } else if (generator.hasNext()) {
-          next = generator.nextEvent();
-          if (isRateLimited && config.configuration.probDelayedEvent > 0.0
-              && config.configuration.occasionalDelaySec > 0
-              && ThreadLocalRandom.current().nextDouble() < config.configuration.probDelayedEvent) {
-            // We'll hold back this event and go around again.
-            long delayMs =
-                ThreadLocalRandom.current().nextLong(config.configuration.occasionalDelaySec * 1000)
-                + 1L;
-            LOG.debug("delaying event by {}ms", delayMs);
-            heldBackEvents.add(next.withDelay(delayMs));
-            continue;
-          }
-        } else {
-          // Waiting for held-back event to fire.
-          if (isRateLimited) {
-            updateBacklog(now, 0);
-          }
-          return false;
-        }
-
-        pendingEventWallclockTime = next.wallclockTimestamp;
-        pendingEvent = TimestampedValue.of(next.event, new Instant(next.eventTimestamp));
-        long newWatermark =
-            next.watermark - Duration.standardSeconds(watermarkHoldbackSec).getMillis();
-        if (newWatermark > watermark) {
-          watermark = newWatermark;
-        }
-      }
-
-      if (isRateLimited) {
-        if (pendingEventWallclockTime > now) {
-          // We want this event to fire in the future. Try again later.
-          updateBacklog(now, 0);
-          return false;
-        }
-        updateBacklog(now, now - pendingEventWallclockTime);
-      }
-
-      // This event is ready to fire.
-      currentEvent = pendingEvent;
-      pendingEvent = null;
-      return true;
-    }
-
-    private void updateBacklog(long now, long newBacklogDurationMs) {
-      backlogDurationMs = newBacklogDurationMs;
-      long interEventDelayUs = generator.currentInterEventDelayUs();
-      if (interEventDelayUs != 0) {
-        long backlogEvents = (backlogDurationMs * 1000 + interEventDelayUs - 1) / interEventDelayUs;
-        backlogBytes = generator.getCurrentConfig().estimatedBytesForEvents(backlogEvents);
-      }
-      if (lastReportedBacklogWallclock < 0
-          || now - lastReportedBacklogWallclock > BACKLOG_PERIOD.getMillis()) {
-        double timeDialation = Double.NaN;
-        if (pendingEvent != null
-            && lastReportedBacklogWallclock >= 0
-            && timestampAtLastReportedBacklogMs >= 0) {
-          long wallclockProgressionMs = now - lastReportedBacklogWallclock;
-          long eventTimeProgressionMs =
-              pendingEvent.getTimestamp().getMillis() - timestampAtLastReportedBacklogMs;
-          timeDialation = (double) eventTimeProgressionMs / (double) wallclockProgressionMs;
-        }
-        LOG.debug(
-            "unbounded generator backlog now {}ms ({} bytes) at {}us interEventDelay "
-            + "with {} time dilation",
-            backlogDurationMs, backlogBytes, interEventDelayUs, timeDialation);
-        lastReportedBacklogWallclock = now;
-        if (pendingEvent != null) {
-          timestampAtLastReportedBacklogMs = pendingEvent.getTimestamp().getMillis();
-        }
-      }
-    }
-
-    @Override
-    public Event getCurrent() {
-      if (currentEvent == null) {
-        throw new NoSuchElementException();
-      }
-      return currentEvent.getValue();
-    }
-
-    @Override
-    public Instant getCurrentTimestamp() {
-      if (currentEvent == null) {
-        throw new NoSuchElementException();
-      }
-      return currentEvent.getTimestamp();
-    }
-
-    @Override
-    public void close() {
-      // Nothing to close.
-    }
-
-    @Override
-    public UnboundedEventSource getCurrentSource() {
-      return UnboundedEventSource.this;
-    }
-
-    @Override
-    public Instant getWatermark() {
-      return new Instant(watermark);
-    }
-
-    @Override
-    public Generator.Checkpoint getCheckpointMark() {
-      return generator.toCheckpoint();
-    }
-
-    @Override
-    public long getSplitBacklogBytes() {
-      return backlogBytes == null ? BACKLOG_UNKNOWN : backlogBytes;
-    }
-
-    @Override
-    public String toString() {
-      return String.format("EventReader(%d, %d, %d)",
-          generator.getCurrentConfig().getStartEventId(), generator.getNextEventId(),
-          generator.getCurrentConfig().getStopEventId());
-    }
-  }
-
-  @Override
-  public Coder<Generator.Checkpoint> getCheckpointMarkCoder() {
-    return Generator.Checkpoint.CODER_INSTANCE;
-  }
-
-  @Override
-  public List<UnboundedEventSource> split(
-      int desiredNumSplits, PipelineOptions options) {
-    LOG.trace("splitting unbounded source into {} sub-sources", numEventGenerators);
-    List<UnboundedEventSource> results = new ArrayList<>();
-    // Ignore desiredNumSplits and use numEventGenerators instead.
-    for (GeneratorConfig subConfig : config.split(numEventGenerators)) {
-      results.add(new UnboundedEventSource(subConfig, 1, watermarkHoldbackSec, isRateLimited));
-    }
-    return results;
-  }
-
-  @Override
-  public EventReader createReader(
-      PipelineOptions options, @Nullable Generator.Checkpoint checkpoint) {
-    if (checkpoint == null) {
-      LOG.trace("creating initial unbounded reader for {}", config);
-      return new EventReader(config);
-    } else {
-      LOG.trace("resuming unbounded reader from {}", checkpoint);
-      return new EventReader(checkpoint.toGenerator(config));
-    }
-  }
-
-  @Override
-  public void validate() {
-    // Nothing to validate.
-  }
-
-  @Override
-  public Coder<Event> getDefaultOutputCoder() {
-    return Event.CODER;
-  }
-
-  @Override
-  public String toString() {
-    return String.format(
-        "UnboundedEventSource(%d, %d)", config.getStartEventId(), config.getStopEventId());
-  }
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/f4333df7/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/sources/package-info.java
----------------------------------------------------------------------
diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/sources/package-info.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/sources/package-info.java
deleted file mode 100644
index ceaec9d..0000000
--- a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/sources/package-info.java
+++ /dev/null
@@ -1,22 +0,0 @@
-/*
- * 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.
- */
-
-/**
- * Nexmark Synthetic Sources.
- */
-package org.apache.beam.integration.nexmark.sources;

http://git-wip-us.apache.org/repos/asf/beam/blob/f4333df7/integration/java/nexmark/src/main/resources/log4j.properties
----------------------------------------------------------------------
diff --git a/integration/java/nexmark/src/main/resources/log4j.properties b/integration/java/nexmark/src/main/resources/log4j.properties
deleted file mode 100644
index 7dd57b5..0000000
--- a/integration/java/nexmark/src/main/resources/log4j.properties
+++ /dev/null
@@ -1,55 +0,0 @@
-#
-# 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.
-#
-
-# Set everything to be logged to the console
-log4j.rootCategory=DEBUG, console
-log4j.appender.console=org.apache.log4j.ConsoleAppender
-log4j.appender.console.target=System.err
-log4j.appender.console.layout=org.apache.log4j.PatternLayout
-log4j.appender.console.layout.ConversionPattern=%d{yy/MM/dd HH:mm:ss} %p %c: %m%n
-
-# General Beam loggers
-log4j.logger.org.apache.beam.runners.direct=WARN
-log4j.logger.org.apache.beam.sdk=WARN
-
-# Nexmark specific
-log4j.logger.org.apache.beam.integration.nexmark=WARN
-
-# Settings to quiet third party logs that are too verbose
-log4j.logger.org.spark_project.jetty=WARN
-log4j.logger.org.spark_project.jetty.util.component.AbstractLifeCycle=ERROR
-
-# Setting to quiet spark logs, Beam logs should standout
-log4j.logger.org.apache.beam.runners.spark=INFO
-log4j.logger.org.apache.spark=WARN
-log4j.logger.org.spark-project=WARN
-log4j.logger.io.netty=INFO
-
-# Settings to quiet flink logs
-log4j.logger.org.apache.flink=WARN
-
-# Settings to quiet apex logs
-log4j.logger.org.apache.beam.runners.apex=INFO
-log4j.logger.com.datatorrent=ERROR
-log4j.logger.org.apache.hadoop.metrics2=WARN
-log4j.logger.org.apache.commons=WARN
-log4j.logger.org.apache.hadoop.security=WARN
-log4j.logger.org.apache.hadoop.util=WARN
-
-# SPARK-9183: Settings to avoid annoying messages when looking up nonexistent UDFs in SparkSQL with Hive support
-log4j.logger.org.apache.hadoop.hive.metastore.RetryingHMSHandler=FATAL
-log4j.logger.org.apache.hadoop.hive.ql.exec.FunctionRegistry=ERROR

http://git-wip-us.apache.org/repos/asf/beam/blob/f4333df7/integration/java/nexmark/src/test/java/org/apache/beam/integration/nexmark/queries/QueryTest.java
----------------------------------------------------------------------
diff --git a/integration/java/nexmark/src/test/java/org/apache/beam/integration/nexmark/queries/QueryTest.java b/integration/java/nexmark/src/test/java/org/apache/beam/integration/nexmark/queries/QueryTest.java
deleted file mode 100644
index 64a8e4f..0000000
--- a/integration/java/nexmark/src/test/java/org/apache/beam/integration/nexmark/queries/QueryTest.java
+++ /dev/null
@@ -1,185 +0,0 @@
-/*
- * 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.beam.integration.nexmark.queries;
-
-import org.apache.beam.integration.nexmark.NexmarkConfiguration;
-import org.apache.beam.integration.nexmark.NexmarkUtils;
-import org.apache.beam.integration.nexmark.model.KnownSize;
-import org.apache.beam.sdk.PipelineResult;
-import org.apache.beam.sdk.testing.NeedsRunner;
-import org.apache.beam.sdk.testing.PAssert;
-import org.apache.beam.sdk.testing.TestPipeline;
-import org.apache.beam.sdk.testing.UsesStatefulParDo;
-import org.apache.beam.sdk.testing.UsesTimersInParDo;
-import org.apache.beam.sdk.values.PCollection;
-import org.apache.beam.sdk.values.TimestampedValue;
-import org.junit.Rule;
-import org.junit.Test;
-import org.junit.experimental.categories.Category;
-import org.junit.runner.RunWith;
-import org.junit.runners.JUnit4;
-
-/** Test the various NEXMark queries yield results coherent with their models. */
-@RunWith(JUnit4.class)
-public class QueryTest {
-  private static final NexmarkConfiguration CONFIG = NexmarkConfiguration.DEFAULT.copy();
-
-  static {
-    // careful, results of tests are linked to numEventGenerators because of timestamp generation
-    CONFIG.numEventGenerators = 1;
-    CONFIG.numEvents = 1000;
-  }
-
-  @Rule public TestPipeline p = TestPipeline.create();
-
-  /** Test {@code query} matches {@code model}. */
-  private void queryMatchesModel(
-      String name, NexmarkQuery query, NexmarkQueryModel model, boolean streamingMode) {
-    NexmarkUtils.setupPipeline(NexmarkUtils.CoderStrategy.HAND, p);
-    PCollection<TimestampedValue<KnownSize>> results;
-    if (streamingMode) {
-      results =
-          p.apply(name + ".ReadUnBounded", NexmarkUtils.streamEventsSource(CONFIG)).apply(query);
-    } else {
-      results = p.apply(name + ".ReadBounded", NexmarkUtils.batchEventsSource(CONFIG)).apply(query);
-    }
-    PAssert.that(results).satisfies(model.assertionFor());
-    PipelineResult result = p.run();
-    result.waitUntilFinish();
-  }
-
-  @Test
-  @Category(NeedsRunner.class)
-  public void query0MatchesModelBatch() {
-    queryMatchesModel("Query0TestBatch", new Query0(CONFIG), new Query0Model(CONFIG), false);
-  }
-
-  @Test
-  @Category(NeedsRunner.class)
-  public void query0MatchesModelStreaming() {
-    queryMatchesModel("Query0TestStreaming", new Query0(CONFIG), new Query0Model(CONFIG), true);
-  }
-
-  @Test
-  @Category(NeedsRunner.class)
-  public void query1MatchesModelBatch() {
-    queryMatchesModel("Query1TestBatch", new Query1(CONFIG), new Query1Model(CONFIG), false);
-  }
-
-  @Test
-  @Category(NeedsRunner.class)
-  public void query1MatchesModelStreaming() {
-    queryMatchesModel("Query1TestStreaming", new Query1(CONFIG), new Query1Model(CONFIG), true);
-  }
-
-  @Test
-  @Category(NeedsRunner.class)
-  public void query2MatchesModelBatch() {
-    queryMatchesModel("Query2TestBatch", new Query2(CONFIG), new Query2Model(CONFIG), false);
-  }
-
-  @Test
-  @Category(NeedsRunner.class)
-  public void query2MatchesModelStreaming() {
-    queryMatchesModel("Query2TestStreaming", new Query2(CONFIG), new Query2Model(CONFIG), true);
-  }
-
-  @Test
-  @Category({NeedsRunner.class, UsesStatefulParDo.class, UsesTimersInParDo.class})
-  public void query3MatchesModelBatch() {
-    queryMatchesModel("Query3TestBatch", new Query3(CONFIG), new Query3Model(CONFIG), false);
-  }
-
-  @Test
-  @Category({NeedsRunner.class, UsesStatefulParDo.class, UsesTimersInParDo.class})
-  public void query3MatchesModelStreaming() {
-    queryMatchesModel("Query3TestStreaming", new Query3(CONFIG), new Query3Model(CONFIG), true);
-  }
-
-  @Test
-  @Category(NeedsRunner.class)
-  public void query4MatchesModelBatch() {
-    queryMatchesModel("Query4TestBatch", new Query4(CONFIG), new Query4Model(CONFIG), false);
-  }
-
-  @Test
-  @Category(NeedsRunner.class)
-  public void query4MatchesModelStreaming() {
-    queryMatchesModel("Query4TestStreaming", new Query4(CONFIG), new Query4Model(CONFIG), true);
-  }
-
-  @Test
-  @Category(NeedsRunner.class)
-  public void query5MatchesModelBatch() {
-    queryMatchesModel("Query5TestBatch", new Query5(CONFIG), new Query5Model(CONFIG), false);
-  }
-
-  @Test
-  @Category(NeedsRunner.class)
-  public void query5MatchesModelStreaming() {
-    queryMatchesModel("Query5TestStreaming", new Query5(CONFIG), new Query5Model(CONFIG), true);
-  }
-
-  @Test
-  @Category(NeedsRunner.class)
-  public void query6MatchesModelBatch() {
-    queryMatchesModel("Query6TestBatch", new Query6(CONFIG), new Query6Model(CONFIG), false);
-  }
-
-  @Test
-  @Category(NeedsRunner.class)
-  public void query6MatchesModelStreaming() {
-    queryMatchesModel("Query6TestStreaming", new Query6(CONFIG), new Query6Model(CONFIG), true);
-  }
-
-  @Test
-  @Category(NeedsRunner.class)
-  public void query7MatchesModelBatch() {
-    queryMatchesModel("Query7TestBatch", new Query7(CONFIG), new Query7Model(CONFIG), false);
-  }
-
-  @Test
-  @Category(NeedsRunner.class)
-  public void query7MatchesModelStreaming() {
-    queryMatchesModel("Query7TestStreaming", new Query7(CONFIG), new Query7Model(CONFIG), true);
-  }
-
-  @Test
-  @Category(NeedsRunner.class)
-  public void query8MatchesModelBatch() {
-    queryMatchesModel("Query8TestBatch", new Query8(CONFIG), new Query8Model(CONFIG), false);
-  }
-
-  @Test
-  @Category(NeedsRunner.class)
-  public void query8MatchesModelStreaming() {
-    queryMatchesModel("Query8TestStreaming", new Query8(CONFIG), new Query8Model(CONFIG), true);
-  }
-
-  @Test
-  @Category(NeedsRunner.class)
-  public void query9MatchesModelBatch() {
-    queryMatchesModel("Query9TestBatch", new Query9(CONFIG), new Query9Model(CONFIG), false);
-  }
-
-  @Test
-  @Category(NeedsRunner.class)
-  public void query9MatchesModelStreaming() {
-    queryMatchesModel("Query9TestStreaming", new Query9(CONFIG), new Query9Model(CONFIG), true);
-  }
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/f4333df7/integration/java/nexmark/src/test/java/org/apache/beam/integration/nexmark/sources/BoundedEventSourceTest.java
----------------------------------------------------------------------
diff --git a/integration/java/nexmark/src/test/java/org/apache/beam/integration/nexmark/sources/BoundedEventSourceTest.java b/integration/java/nexmark/src/test/java/org/apache/beam/integration/nexmark/sources/BoundedEventSourceTest.java
deleted file mode 100644
index d95461a..0000000
--- a/integration/java/nexmark/src/test/java/org/apache/beam/integration/nexmark/sources/BoundedEventSourceTest.java
+++ /dev/null
@@ -1,71 +0,0 @@
-/*
- * 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.beam.integration.nexmark.sources;
-
-import org.apache.beam.integration.nexmark.NexmarkConfiguration;
-import org.apache.beam.integration.nexmark.NexmarkOptions;
-import org.apache.beam.sdk.options.PipelineOptionsFactory;
-import org.apache.beam.sdk.testing.SourceTestUtils;
-
-import org.junit.Test;
-import org.junit.runner.RunWith;
-import org.junit.runners.JUnit4;
-
-/**
- * Test {@link BoundedEventSource}.
- */
-@RunWith(JUnit4.class)
-public class BoundedEventSourceTest {
-  private GeneratorConfig makeConfig(long n) {
-    return new GeneratorConfig(
-        NexmarkConfiguration.DEFAULT, System.currentTimeMillis(), 0, n, 0);
-  }
-
-  @Test
-  public void sourceAndReadersWork() throws Exception {
-    NexmarkOptions options = PipelineOptionsFactory.as(NexmarkOptions.class);
-    long n = 200L;
-    BoundedEventSource source = new BoundedEventSource(makeConfig(n), 1);
-
-    SourceTestUtils.assertUnstartedReaderReadsSameAsItsSource(
-        source.createReader(options), options);
-  }
-
-  @Test
-  public void splitAtFractionRespectsContract() throws Exception {
-    NexmarkOptions options = PipelineOptionsFactory.as(NexmarkOptions.class);
-    long n = 20L;
-    BoundedEventSource source = new BoundedEventSource(makeConfig(n), 1);
-
-    // Can't split if already consumed.
-    SourceTestUtils.assertSplitAtFractionFails(source, 10, 0.3, options);
-
-    SourceTestUtils.assertSplitAtFractionSucceedsAndConsistent(source, 5, 0.3, options);
-
-    SourceTestUtils.assertSplitAtFractionExhaustive(source, options);
-  }
-
-  @Test
-  public void splitIntoBundlesRespectsContract() throws Exception {
-    NexmarkOptions options = PipelineOptionsFactory.as(NexmarkOptions.class);
-    long n = 200L;
-    BoundedEventSource source = new BoundedEventSource(makeConfig(n), 1);
-    SourceTestUtils.assertSourcesEqualReferenceSource(
-        source, source.split(10, options), options);
-  }
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/f4333df7/integration/java/nexmark/src/test/java/org/apache/beam/integration/nexmark/sources/GeneratorTest.java
----------------------------------------------------------------------
diff --git a/integration/java/nexmark/src/test/java/org/apache/beam/integration/nexmark/sources/GeneratorTest.java b/integration/java/nexmark/src/test/java/org/apache/beam/integration/nexmark/sources/GeneratorTest.java
deleted file mode 100644
index b0dff2f..0000000
--- a/integration/java/nexmark/src/test/java/org/apache/beam/integration/nexmark/sources/GeneratorTest.java
+++ /dev/null
@@ -1,111 +0,0 @@
-/*
- * 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.beam.integration.nexmark.sources;
-
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertTrue;
-
-import java.util.ArrayList;
-import java.util.Iterator;
-import java.util.List;
-
-import org.apache.beam.integration.nexmark.NexmarkConfiguration;
-import org.junit.Test;
-import org.junit.runner.RunWith;
-import org.junit.runners.JUnit4;
-
-/**
- * Test {@link Generator}.
- */
-@RunWith(JUnit4.class)
-public class GeneratorTest {
-  private GeneratorConfig makeConfig(long n) {
-    return new GeneratorConfig(
-        NexmarkConfiguration.DEFAULT, System.currentTimeMillis(), 0, n, 0);
-  }
-
-  private <T> long consume(long n, Iterator<T> itr) {
-    for (long i = 0; i < n; i++) {
-      assertTrue(itr.hasNext());
-      itr.next();
-    }
-    return n;
-  }
-
-  private <T> long consume(Iterator<T> itr) {
-    long n = 0;
-    while (itr.hasNext()) {
-      itr.next();
-      n++;
-    }
-    return n;
-  }
-
-  @Test
-  public void splitAtFractionPreservesOverallEventCount() {
-    long n = 55729L;
-    GeneratorConfig initialConfig = makeConfig(n);
-    long expected = initialConfig.getStopEventId() - initialConfig.getStartEventId();
-
-    long actual = 0;
-
-    Generator initialGenerator = new Generator(initialConfig);
-
-    // Consume some events.
-    actual += consume(5000, initialGenerator);
-
-
-    // Split once.
-    GeneratorConfig remainConfig1 = initialGenerator.splitAtEventId(9000L);
-    Generator remainGenerator1 = new Generator(remainConfig1);
-
-    // Consume some more events.
-    actual += consume(2000, initialGenerator);
-    actual += consume(3000, remainGenerator1);
-
-    // Split again.
-    GeneratorConfig remainConfig2 = remainGenerator1.splitAtEventId(30000L);
-    Generator remainGenerator2 = new Generator(remainConfig2);
-
-    // Run to completion.
-    actual += consume(initialGenerator);
-    actual += consume(remainGenerator1);
-    actual += consume(remainGenerator2);
-
-    assertEquals(expected, actual);
-  }
-
-  @Test
-  public void splitPreservesOverallEventCount() {
-    long n = 51237L;
-    GeneratorConfig initialConfig = makeConfig(n);
-    long expected = initialConfig.getStopEventId() - initialConfig.getStartEventId();
-
-    List<Generator> generators = new ArrayList<>();
-    for (GeneratorConfig subConfig : initialConfig.split(20)) {
-      generators.add(new Generator(subConfig));
-    }
-
-    long actual = 0;
-    for (Generator generator : generators) {
-      actual += consume(generator);
-    }
-
-    assertEquals(expected, actual);
-  }
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/f4333df7/integration/java/nexmark/src/test/java/org/apache/beam/integration/nexmark/sources/UnboundedEventSourceTest.java
----------------------------------------------------------------------
diff --git a/integration/java/nexmark/src/test/java/org/apache/beam/integration/nexmark/sources/UnboundedEventSourceTest.java b/integration/java/nexmark/src/test/java/org/apache/beam/integration/nexmark/sources/UnboundedEventSourceTest.java
deleted file mode 100644
index 1ecc33e..0000000
--- a/integration/java/nexmark/src/test/java/org/apache/beam/integration/nexmark/sources/UnboundedEventSourceTest.java
+++ /dev/null
@@ -1,107 +0,0 @@
-/*
- * 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.beam.integration.nexmark.sources;
-
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertFalse;
-import static org.junit.Assert.assertTrue;
-
-import java.io.IOException;
-import java.util.HashSet;
-import java.util.Random;
-import java.util.Set;
-
-import org.apache.beam.integration.nexmark.NexmarkConfiguration;
-import org.apache.beam.integration.nexmark.model.Event;
-import org.apache.beam.sdk.io.UnboundedSource.CheckpointMark;
-import org.apache.beam.sdk.io.UnboundedSource.UnboundedReader;
-import org.apache.beam.sdk.options.PipelineOptions;
-import org.apache.beam.sdk.testing.TestPipeline;
-
-import org.junit.Test;
-import org.junit.runner.RunWith;
-import org.junit.runners.JUnit4;
-
-/**
- * Test UnboundedEventSource.
- */
-@RunWith(JUnit4.class)
-public class UnboundedEventSourceTest {
-  private GeneratorConfig makeConfig(long n) {
-    return new GeneratorConfig(
-        NexmarkConfiguration.DEFAULT, System.currentTimeMillis(), 0, n, 0);
-  }
-
-  /**
-   * Helper for tracking which ids we've seen (so we can detect dups) and
-   * confirming reading events match the model events.
-   */
-  private static class EventIdChecker {
-    private final Set<Long> seenPersonIds = new HashSet<>();
-    private final Set<Long> seenAuctionIds = new HashSet<>();
-
-    public void add(Event event) {
-      if (event.newAuction != null) {
-        assertTrue(seenAuctionIds.add(event.newAuction.id));
-      } else if (event.newPerson != null) {
-        assertTrue(seenPersonIds.add(event.newPerson.id));
-      }
-    }
-
-    public void add(int n, UnboundedReader<Event> reader, Generator modelGenerator)
-        throws IOException {
-      for (int i = 0; i < n; i++) {
-        assertTrue(modelGenerator.hasNext());
-        Event modelEvent = modelGenerator.next().getValue();
-        assertTrue(reader.advance());
-        Event actualEvent = reader.getCurrent();
-        assertEquals(modelEvent.toString(), actualEvent.toString());
-        add(actualEvent);
-      }
-    }
-  }
-
-  /**
-   * Check aggressively checkpointing and resuming a reader gives us exactly the
-   * same event stream as reading directly.
-   */
-  @Test
-  public void resumeFromCheckpoint() throws IOException {
-    Random random = new Random(297);
-    int n = 47293;
-    GeneratorConfig config = makeConfig(n);
-    Generator modelGenerator = new Generator(config);
-
-    EventIdChecker checker = new EventIdChecker();
-    PipelineOptions options = TestPipeline.testingPipelineOptions();
-    UnboundedEventSource source = new UnboundedEventSource(config, 1, 0, false);
-    UnboundedReader<Event> reader = source.createReader(options, null);
-
-    while (n > 0) {
-      int m = Math.min(459 + random.nextInt(455), n);
-      System.out.printf("reading %d...%n", m);
-      checker.add(m, reader, modelGenerator);
-      n -= m;
-      System.out.printf("splitting with %d remaining...%n", n);
-      CheckpointMark checkpointMark = reader.getCheckpointMark();
-      reader = source.createReader(options, (Generator.Checkpoint) checkpointMark);
-    }
-
-    assertFalse(reader.advance());
-  }
-}


[07/55] [abbrv] beam git commit: Remove Accumulators and switch to the Metrics API

Posted by ie...@apache.org.
Remove Accumulators and switch to the Metrics API

Fix compile after sideOutput and split refactor


Project: http://git-wip-us.apache.org/repos/asf/beam/repo
Commit: http://git-wip-us.apache.org/repos/asf/beam/commit/b438fa7d
Tree: http://git-wip-us.apache.org/repos/asf/beam/tree/b438fa7d
Diff: http://git-wip-us.apache.org/repos/asf/beam/diff/b438fa7d

Branch: refs/heads/master
Commit: b438fa7df16e5181f73b6103ac2f57430cd9e6f3
Parents: e10d578
Author: Ismaël Mejía <ie...@apache.org>
Authored: Wed Apr 19 11:22:42 2017 +0200
Committer: Ismaël Mejía <ie...@gmail.com>
Committed: Wed Aug 23 19:07:27 2017 +0200

----------------------------------------------------------------------
 integration/java/nexmark/pom.xml                |   6 +-
 .../beam/integration/nexmark/Monitor.java       |  77 ++--
 .../beam/integration/nexmark/NexmarkQuery.java  |  16 +-
 .../beam/integration/nexmark/NexmarkRunner.java | 129 +++++--
 .../beam/integration/nexmark/NexmarkUtils.java  | 107 +++---
 .../beam/integration/nexmark/WinningBids.java   | 102 +++---
 .../nexmark/drivers/NexmarkGoogleRunner.java    |   4 +-
 .../integration/nexmark/queries/Query0.java     |  10 +-
 .../integration/nexmark/queries/Query10.java    | 363 +++++++++----------
 .../integration/nexmark/queries/Query3.java     |  73 ++--
 .../nexmark/sources/BoundedEventSource.java     |   2 +-
 .../nexmark/sources/UnboundedEventSource.java   |   2 +-
 .../nexmark/sources/BoundedEventSourceTest.java |   2 +-
 13 files changed, 448 insertions(+), 445 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/beam/blob/b438fa7d/integration/java/nexmark/pom.xml
----------------------------------------------------------------------
diff --git a/integration/java/nexmark/pom.xml b/integration/java/nexmark/pom.xml
index 67d6117..103c18f 100644
--- a/integration/java/nexmark/pom.xml
+++ b/integration/java/nexmark/pom.xml
@@ -29,7 +29,6 @@
 
   <artifactId>beam-integration-java-nexmark</artifactId>
   <name>Apache Beam :: Integration Tests :: Java :: Nexmark</name>
-
   <packaging>jar</packaging>
 
   <properties>
@@ -227,6 +226,11 @@
       <artifactId>beam-sdks-java-io-google-cloud-platform</artifactId>
     </dependency>
 
+    <dependency>
+      <groupId>org.apache.beam</groupId>
+      <artifactId>beam-sdks-java-extensions-gcp-core</artifactId>
+    </dependency>
+
     <!-- Extra libraries -->
     <dependency>
       <groupId>com.google.apis</groupId>

http://git-wip-us.apache.org/repos/asf/beam/blob/b438fa7d/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/Monitor.java
----------------------------------------------------------------------
diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/Monitor.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/Monitor.java
index 6370e41..cb4d71c 100644
--- a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/Monitor.java
+++ b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/Monitor.java
@@ -20,54 +20,55 @@ package org.apache.beam.integration.nexmark;
 import java.io.Serializable;
 
 import org.apache.beam.integration.nexmark.model.KnownSize;
-import org.apache.beam.sdk.transforms.Aggregator;
+import org.apache.beam.sdk.metrics.Counter;
+import org.apache.beam.sdk.metrics.Distribution;
+import org.apache.beam.sdk.metrics.Metrics;
 import org.apache.beam.sdk.transforms.DoFn;
-import org.apache.beam.sdk.transforms.Max;
-import org.apache.beam.sdk.transforms.Min;
 import org.apache.beam.sdk.transforms.PTransform;
 import org.apache.beam.sdk.transforms.ParDo;
-import org.apache.beam.sdk.transforms.Sum;
 import org.apache.beam.sdk.values.PCollection;
 
 /**
- * A monitor of elements with support for later retrieving their aggregators.
+ * A monitor of elements with support for later retrieving their metrics.
  *
  * @param <T> Type of element we are monitoring.
  */
 public class Monitor<T extends KnownSize> implements Serializable {
   private class MonitorDoFn extends DoFn<T, T> {
-    public final Aggregator<Long, Long> elementCounter =
-        createAggregator(counterNamePrefix + "_elements", Sum.ofLongs());
-    public final Aggregator<Long, Long> bytesCounter =
-        createAggregator(counterNamePrefix + "_bytes", Sum.ofLongs());
-    public final Aggregator<Long, Long> startTime =
-        createAggregator(counterNamePrefix + "_startTime", Min.ofLongs());
-    public final Aggregator<Long, Long> endTime =
-        createAggregator(counterNamePrefix + "_endTime", Max.ofLongs());
-    public final Aggregator<Long, Long> startTimestamp =
-        createAggregator("startTimestamp", Min.ofLongs());
-    public final Aggregator<Long, Long> endTimestamp =
-        createAggregator("endTimestamp", Max.ofLongs());
+    final Counter elementCounter =
+      Metrics.counter(name , prefix + ".elements");
+    final Counter bytesCounter =
+      Metrics.counter(name , prefix + ".bytes");
+    final Distribution startTime =
+      Metrics.distribution(name , prefix + ".startTime");
+    final Distribution endTime =
+      Metrics.distribution(name , prefix + ".endTime");
+    final Distribution startTimestamp =
+      Metrics.distribution(name , prefix + ".startTimestamp");
+    final Distribution endTimestamp =
+      Metrics.distribution(name , prefix + ".endTimestamp");
 
     @ProcessElement
     public void processElement(ProcessContext c) {
-      elementCounter.addValue(1L);
-      bytesCounter.addValue(c.element().sizeInBytes());
+      elementCounter.inc();
+      bytesCounter.inc(c.element().sizeInBytes());
       long now = System.currentTimeMillis();
-      startTime.addValue(now);
-      endTime.addValue(now);
-      startTimestamp.addValue(c.timestamp().getMillis());
-      endTimestamp.addValue(c.timestamp().getMillis());
+      startTime.update(now);
+      endTime.update(now);
+      startTimestamp.update(c.timestamp().getMillis());
+      endTimestamp.update(c.timestamp().getMillis());
       c.output(c.element());
     }
   }
 
+  public final String name;
+  public final String prefix;
   final MonitorDoFn doFn;
   final PTransform<PCollection<? extends T>, PCollection<T>> transform;
-  private String counterNamePrefix;
 
-  public Monitor(String name, String counterNamePrefix) {
-    this.counterNamePrefix = counterNamePrefix;
+  public Monitor(String name, String prefix) {
+    this.name = name;
+    this.prefix = prefix;
     doFn = new MonitorDoFn();
     transform = ParDo.of(doFn);
   }
@@ -75,28 +76,4 @@ public class Monitor<T extends KnownSize> implements Serializable {
   public PTransform<PCollection<? extends T>, PCollection<T>> getTransform() {
     return transform;
   }
-
-  public Aggregator<Long, Long> getElementCounter() {
-    return doFn.elementCounter;
-  }
-
-  public Aggregator<Long, Long> getBytesCounter() {
-    return doFn.bytesCounter;
-  }
-
-  public Aggregator<Long, Long> getStartTime() {
-    return doFn.startTime;
-  }
-
-  public Aggregator<Long, Long> getEndTime() {
-    return doFn.endTime;
-  }
-
-  public Aggregator<Long, Long> getStartTimestamp() {
-    return doFn.startTimestamp;
-  }
-
-  public Aggregator<Long, Long> getEndTimestamp() {
-    return doFn.endTimestamp;
-  }
 }

http://git-wip-us.apache.org/repos/asf/beam/blob/b438fa7d/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkQuery.java
----------------------------------------------------------------------
diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkQuery.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkQuery.java
index e1cd493..ab1c305 100644
--- a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkQuery.java
+++ b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkQuery.java
@@ -17,13 +17,13 @@
  */
 package org.apache.beam.integration.nexmark;
 
-import javax.annotation.Nullable;
 import org.apache.beam.integration.nexmark.model.Auction;
 import org.apache.beam.integration.nexmark.model.Bid;
 import org.apache.beam.integration.nexmark.model.Event;
 import org.apache.beam.integration.nexmark.model.KnownSize;
 import org.apache.beam.integration.nexmark.model.Person;
-import org.apache.beam.sdk.transforms.Aggregator;
+import org.apache.beam.sdk.metrics.Counter;
+import org.apache.beam.sdk.metrics.Metrics;
 import org.apache.beam.sdk.transforms.DoFn;
 import org.apache.beam.sdk.transforms.Filter;
 import org.apache.beam.sdk.transforms.PTransform;
@@ -206,6 +206,7 @@ public abstract class NexmarkQuery
   public final Monitor<Event> eventMonitor;
   public final Monitor<KnownSize> resultMonitor;
   public final Monitor<Event> endOfStreamMonitor;
+  protected final Counter fatalCounter;
 
   protected NexmarkQuery(NexmarkConfiguration configuration, String name) {
     super(name);
@@ -214,23 +215,16 @@ public abstract class NexmarkQuery
       eventMonitor = new Monitor<>(name + ".Events", "event");
       resultMonitor = new Monitor<>(name + ".Results", "result");
       endOfStreamMonitor = new Monitor<>(name + ".EndOfStream", "end");
+      fatalCounter = Metrics.counter(name , "fatal");
     } else {
       eventMonitor = null;
       resultMonitor = null;
       endOfStreamMonitor = null;
+      fatalCounter = null;
     }
   }
 
   /**
-   * Return the aggregator which counts fatal errors in this query. Return null if no such
-   * aggregator.
-   */
-  @Nullable
-  public Aggregator<Long, Long> getFatalCount() {
-    return null;
-  }
-
-  /**
    * Implement the actual query. All we know about the result is it has a known encoded size.
    */
   protected abstract PCollection<KnownSize> applyPrim(PCollection<Event> events);

http://git-wip-us.apache.org/repos/asf/beam/blob/b438fa7d/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkRunner.java
----------------------------------------------------------------------
diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkRunner.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkRunner.java
index ef5f0e2..87314ce 100644
--- a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkRunner.java
+++ b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkRunner.java
@@ -24,14 +24,13 @@ import com.google.api.services.bigquery.model.TableRow;
 import com.google.api.services.bigquery.model.TableSchema;
 import com.google.common.base.Strings;
 import com.google.common.collect.ImmutableList;
-import com.google.common.collect.Iterables;
 import com.google.common.collect.Lists;
 import java.io.IOException;
 import java.util.ArrayList;
 import java.util.Arrays;
-import java.util.Collection;
 import java.util.Collections;
 import java.util.List;
+import java.util.NoSuchElementException;
 import java.util.concurrent.ThreadLocalRandom;
 import javax.annotation.Nullable;
 import org.apache.beam.integration.nexmark.io.PubsubHelper;
@@ -63,15 +62,18 @@ import org.apache.beam.integration.nexmark.queries.Query8;
 import org.apache.beam.integration.nexmark.queries.Query8Model;
 import org.apache.beam.integration.nexmark.queries.Query9;
 import org.apache.beam.integration.nexmark.queries.Query9Model;
-import org.apache.beam.sdk.AggregatorRetrievalException;
 import org.apache.beam.sdk.Pipeline;
 import org.apache.beam.sdk.PipelineResult;
 import org.apache.beam.sdk.io.AvroIO;
 import org.apache.beam.sdk.io.TextIO;
 import org.apache.beam.sdk.io.gcp.bigquery.BigQueryIO;
 import org.apache.beam.sdk.io.gcp.pubsub.PubsubIO;
+import org.apache.beam.sdk.metrics.DistributionResult;
+import org.apache.beam.sdk.metrics.MetricNameFilter;
+import org.apache.beam.sdk.metrics.MetricQueryResults;
+import org.apache.beam.sdk.metrics.MetricResult;
+import org.apache.beam.sdk.metrics.MetricsFilter;
 import org.apache.beam.sdk.testing.PAssert;
-import org.apache.beam.sdk.transforms.Aggregator;
 import org.apache.beam.sdk.transforms.DoFn;
 import org.apache.beam.sdk.transforms.ParDo;
 import org.apache.beam.sdk.values.PCollection;
@@ -186,38 +188,59 @@ public abstract class NexmarkRunner<OptionT extends NexmarkOptions> {
   protected abstract int maxNumWorkers();
 
   /**
-   * Return the current value for a long counter, or -1 if can't be retrieved.
+   * Return the current value for a long counter, or a default value if can't be retrieved.
+   * Note this uses only attempted metrics because some runners don't support committed metrics.
    */
-  protected long getLong(PipelineResult job, Aggregator<Long, Long> aggregator) {
+  protected long getCounterMetric(PipelineResult result, String namespace, String name,
+    long defaultValue) {
+    //TODO Ismael calc this only once
+    MetricQueryResults metrics = result.metrics().queryMetrics(
+        MetricsFilter.builder().addNameFilter(MetricNameFilter.named(namespace, name)).build());
+    Iterable<MetricResult<Long>> counters = metrics.counters();
     try {
-      Collection<Long> values = job.getAggregatorValues(aggregator).getValues();
-      if (values.size() != 1) {
-        return -1;
-      }
-      return Iterables.getOnlyElement(values);
-    } catch (AggregatorRetrievalException e) {
-      return -1;
+      MetricResult<Long> metricResult = counters.iterator().next();
+      return metricResult.attempted();
+    } catch (NoSuchElementException e) {
+      //TODO Ismael
     }
+    return defaultValue;
   }
 
   /**
-   * Return the current value for a time counter, or -1 if can't be retrieved.
+   * Return the current value for a long counter, or a default value if can't be retrieved.
+   * Note this uses only attempted metrics because some runners don't support committed metrics.
    */
-  protected long getTimestamp(
-    long now, PipelineResult job, Aggregator<Long, Long> aggregator) {
+  protected long getDistributionMetric(PipelineResult result, String namespace, String name,
+      DistributionType distType, long defaultValue) {
+    MetricQueryResults metrics = result.metrics().queryMetrics(
+        MetricsFilter.builder().addNameFilter(MetricNameFilter.named(namespace, name)).build());
+    Iterable<MetricResult<DistributionResult>> distributions = metrics.distributions();
     try {
-      Collection<Long> values = job.getAggregatorValues(aggregator).getValues();
-      if (values.size() != 1) {
-        return -1;
-      }
-      long value = Iterables.getOnlyElement(values);
-      if (Math.abs(value - now) > Duration.standardDays(10000).getMillis()) {
-        return -1;
+      MetricResult<DistributionResult> distributionResult = distributions.iterator().next();
+      if (distType.equals(DistributionType.MIN)) {
+        return distributionResult.attempted().min();
+      } else if (distType.equals(DistributionType.MAX)) {
+        return distributionResult.attempted().max();
+      } else {
+        //TODO Ismael
       }
-      return value;
-    } catch (AggregatorRetrievalException e) {
+    } catch (NoSuchElementException e) {
+      //TODO Ismael
+    }
+    return defaultValue;
+  }
+
+  private enum DistributionType {MIN, MAX}
+
+  /**
+   * Return the current value for a time counter, or -1 if can't be retrieved.
+   */
+  protected long getTimestampMetric(long now, long value) {
+    //TODO Ismael improve doc
+    if (Math.abs(value - now) > Duration.standardDays(10000).getMillis()) {
       return -1;
     }
+    return value;
   }
 
   /**
@@ -294,21 +317,46 @@ public abstract class NexmarkRunner<OptionT extends NexmarkOptions> {
    * Return the current performance given {@code eventMonitor} and {@code resultMonitor}.
    */
   private NexmarkPerf currentPerf(
-      long startMsSinceEpoch, long now, PipelineResult job,
+      long startMsSinceEpoch, long now, PipelineResult result,
       List<NexmarkPerf.ProgressSnapshot> snapshots, Monitor<?> eventMonitor,
       Monitor<?> resultMonitor) {
     NexmarkPerf perf = new NexmarkPerf();
 
-    long numEvents = getLong(job, eventMonitor.getElementCounter());
-    long numEventBytes = getLong(job, eventMonitor.getBytesCounter());
-    long eventStart = getTimestamp(now, job, eventMonitor.getStartTime());
-    long eventEnd = getTimestamp(now, job, eventMonitor.getEndTime());
-    long numResults = getLong(job, resultMonitor.getElementCounter());
-    long numResultBytes = getLong(job, resultMonitor.getBytesCounter());
-    long resultStart = getTimestamp(now, job, resultMonitor.getStartTime());
-    long resultEnd = getTimestamp(now, job, resultMonitor.getEndTime());
-    long timestampStart = getTimestamp(now, job, resultMonitor.getStartTimestamp());
-    long timestampEnd = getTimestamp(now, job, resultMonitor.getEndTimestamp());
+    long numEvents =
+      getCounterMetric(result, eventMonitor.name, eventMonitor.prefix + ".elements", -1);
+    long numEventBytes =
+      getCounterMetric(result, eventMonitor.name, eventMonitor.prefix + ".bytes", -1);
+    long eventStart =
+      getTimestampMetric(now,
+        getDistributionMetric(result, eventMonitor.name, eventMonitor.prefix + ".startTime",
+          DistributionType.MIN, -1));
+    long eventEnd =
+      getTimestampMetric(now,
+        getDistributionMetric(result, eventMonitor.name, eventMonitor.prefix + ".endTime",
+          DistributionType.MAX, -1));
+
+    long numResults =
+      getCounterMetric(result, resultMonitor.name, resultMonitor.prefix + ".elements", -1);
+    long numResultBytes =
+      getCounterMetric(result, resultMonitor.name, resultMonitor.prefix + ".bytes", -1);
+    long resultStart =
+      getTimestampMetric(now,
+        getDistributionMetric(result, resultMonitor.name, resultMonitor.prefix + ".startTime",
+          DistributionType.MIN, -1));
+    long resultEnd =
+      getTimestampMetric(now,
+        getDistributionMetric(result, resultMonitor.name, resultMonitor.prefix + ".endTime",
+          DistributionType.MAX, -1));
+    long timestampStart =
+      getTimestampMetric(now,
+        getDistributionMetric(result,
+          resultMonitor.name, resultMonitor.prefix + ".startTimestamp",
+          DistributionType.MIN, -1));
+    long timestampEnd =
+      getTimestampMetric(now,
+        getDistributionMetric(result,
+          resultMonitor.name, resultMonitor.prefix + ".endTimestamp",
+          DistributionType.MAX, -1));
 
     long effectiveEnd = -1;
     if (eventEnd >= 0 && resultEnd >= 0) {
@@ -372,7 +420,7 @@ public abstract class NexmarkRunner<OptionT extends NexmarkOptions> {
       perf.shutdownDelaySec = (now - resultEnd) / 1000.0;
     }
 
-    perf.jobId = getJobId(job);
+    perf.jobId = getJobId(result);
     // As soon as available, try to capture cumulative cost at this point too.
 
     NexmarkPerf.ProgressSnapshot snapshot = new NexmarkPerf.ProgressSnapshot();
@@ -574,9 +622,10 @@ public abstract class NexmarkRunner<OptionT extends NexmarkOptions> {
 
       if (options.isStreaming() && !waitingForShutdown) {
         Duration quietFor = new Duration(lastActivityMsSinceEpoch, now);
-        if (query.getFatalCount() != null && getLong(job, query.getFatalCount()) > 0) {
+        long fatalCount = getCounterMetric(job, query.getName(), "fatal", 0);
+        if (fatalCount > 0) {
           NexmarkUtils.console("job has fatal errors, cancelling.");
-          errors.add(String.format("Pipeline reported %s fatal errors", query.getFatalCount()));
+          errors.add(String.format("Pipeline reported %s fatal errors", fatalCount));
           waitingForShutdown = true;
         } else if (configuration.debug && configuration.numEvents > 0
                    && currPerf.numEvents == configuration.numEvents
@@ -1033,7 +1082,7 @@ public abstract class NexmarkRunner<OptionT extends NexmarkOptions> {
       if (c.element().hashCode() % 2 == 0) {
         c.output(c.element());
       } else {
-        c.sideOutput(SIDE, c.element());
+        c.output(SIDE, c.element());
       }
     }
   }

http://git-wip-us.apache.org/repos/asf/beam/blob/b438fa7d/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkUtils.java
----------------------------------------------------------------------
diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkUtils.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkUtils.java
index a47ebcc..18589c4 100644
--- a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkUtils.java
+++ b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkUtils.java
@@ -53,12 +53,12 @@ import org.apache.beam.sdk.coders.CoderRegistry;
 import org.apache.beam.sdk.coders.CustomCoder;
 import org.apache.beam.sdk.coders.SerializableCoder;
 import org.apache.beam.sdk.io.Read;
-import org.apache.beam.sdk.transforms.Aggregator;
+import org.apache.beam.sdk.metrics.Counter;
+import org.apache.beam.sdk.metrics.Metrics;
 import org.apache.beam.sdk.transforms.Combine;
 import org.apache.beam.sdk.transforms.DoFn;
 import org.apache.beam.sdk.transforms.PTransform;
 import org.apache.beam.sdk.transforms.ParDo;
-import org.apache.beam.sdk.transforms.Sum;
 import org.apache.beam.sdk.transforms.windowing.AfterPane;
 import org.apache.beam.sdk.transforms.windowing.BoundedWindow;
 import org.apache.beam.sdk.transforms.windowing.GlobalWindows;
@@ -419,48 +419,42 @@ public class NexmarkUtils {
    */
   public static ParDo.SingleOutput<Event, Event> snoop(final String name) {
     return ParDo.of(new DoFn<Event, Event>() {
-                  final Aggregator<Long, Long> eventCounter =
-                      createAggregator("events", Sum.ofLongs());
-                  final Aggregator<Long, Long> newPersonCounter =
-                      createAggregator("newPersons", Sum.ofLongs());
-                  final Aggregator<Long, Long> newAuctionCounter =
-                      createAggregator("newAuctions", Sum.ofLongs());
-                  final Aggregator<Long, Long> bidCounter =
-                      createAggregator("bids", Sum.ofLongs());
-                  final Aggregator<Long, Long> endOfStreamCounter =
-                      createAggregator("endOfStream", Sum.ofLongs());
-
-                  @ProcessElement
-                  public void processElement(ProcessContext c) {
-                    eventCounter.addValue(1L);
-                    if (c.element().newPerson != null) {
-                      newPersonCounter.addValue(1L);
-                    } else if (c.element().newAuction != null) {
-                      newAuctionCounter.addValue(1L);
-                    } else if (c.element().bid != null) {
-                      bidCounter.addValue(1L);
-                    } else {
-                      endOfStreamCounter.addValue(1L);
-                    }
-                    info("%s snooping element %s", name, c.element());
-                    c.output(c.element());
-                  }
-                });
+      final Counter eventCounter = Metrics.counter(name, "events");
+      final Counter newPersonCounter = Metrics.counter(name, "newPersons");
+      final Counter newAuctionCounter = Metrics.counter(name, "newAuctions");
+      final Counter bidCounter = Metrics.counter(name, "bids");
+      final Counter endOfStreamCounter = Metrics.counter(name, "endOfStream");
+
+      @ProcessElement
+      public void processElement(ProcessContext c) {
+        eventCounter.inc();
+        if (c.element().newPerson != null) {
+          newPersonCounter.inc();
+        } else if (c.element().newAuction != null) {
+          newAuctionCounter.inc();
+        } else if (c.element().bid != null) {
+          bidCounter.inc();
+        } else {
+          endOfStreamCounter.inc();
+        }
+        info("%s snooping element %s", name, c.element());
+        c.output(c.element());
+      }
+    });
   }
 
   /**
    * Return a transform to count and discard each element.
    */
-  public static <T> ParDo.SingleOutput<T, Void> devNull(String name) {
+  public static <T> ParDo.SingleOutput<T, Void> devNull(final String name) {
     return ParDo.of(new DoFn<T, Void>() {
-                  final Aggregator<Long, Long> discardCounter =
-                      createAggregator("discarded", Sum.ofLongs());
+      final Counter discardedCounterMetric = Metrics.counter(name, "discarded");
 
-                  @ProcessElement
-                  public void processElement(ProcessContext c) {
-                    discardCounter.addValue(1L);
-                  }
-                });
+      @ProcessElement
+      public void processElement(ProcessContext c) {
+        discardedCounterMetric.inc();
+      }
+    });
   }
 
   /**
@@ -468,28 +462,27 @@ public class NexmarkUtils {
    */
   public static <T> ParDo.SingleOutput<T, T> log(final String name) {
     return ParDo.of(new DoFn<T, T>() {
-                  @ProcessElement
-                  public void processElement(ProcessContext c) {
-                    LOG.info("%s: %s", name, c.element());
-                    c.output(c.element());
-                  }
-                });
+      @ProcessElement
+      public void processElement(ProcessContext c) {
+        LOG.info("%s: %s", name, c.element());
+        c.output(c.element());
+      }
+    });
   }
 
   /**
    * Return a transform to format each element as a string.
    */
-  public static <T> ParDo.SingleOutput<T, String> format(String name) {
+  public static <T> ParDo.SingleOutput<T, String> format(final String name) {
     return ParDo.of(new DoFn<T, String>() {
-                  final Aggregator<Long, Long> recordCounter =
-                      createAggregator("records", Sum.ofLongs());
+      final Counter recordCounterMetric = Metrics.counter(name, "records");
 
-                  @ProcessElement
-                  public void processElement(ProcessContext c) {
-                    recordCounter.addValue(1L);
-                    c.output(c.element().toString());
-                  }
-                });
+      @ProcessElement
+      public void processElement(ProcessContext c) {
+        recordCounterMetric.inc();
+        c.output(c.element().toString());
+      }
+    });
   }
 
   /**
@@ -497,11 +490,11 @@ public class NexmarkUtils {
    */
   public static <T> ParDo.SingleOutput<T, TimestampedValue<T>> stamp(String name) {
     return ParDo.of(new DoFn<T, TimestampedValue<T>>() {
-                  @ProcessElement
-                  public void processElement(ProcessContext c) {
-                    c.output(TimestampedValue.of(c.element(), c.timestamp()));
-                  }
-                });
+      @ProcessElement
+      public void processElement(ProcessContext c) {
+        c.output(TimestampedValue.of(c.element(), c.timestamp()));
+      }
+    });
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/beam/blob/b438fa7d/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/WinningBids.java
----------------------------------------------------------------------
diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/WinningBids.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/WinningBids.java
index 9f1ddf8..f2566b8 100644
--- a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/WinningBids.java
+++ b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/WinningBids.java
@@ -40,11 +40,11 @@ import org.apache.beam.sdk.coders.Coder;
 import org.apache.beam.sdk.coders.CoderException;
 import org.apache.beam.sdk.coders.VarIntCoder;
 import org.apache.beam.sdk.coders.VarLongCoder;
-import org.apache.beam.sdk.transforms.Aggregator;
+import org.apache.beam.sdk.metrics.Counter;
+import org.apache.beam.sdk.metrics.Metrics;
 import org.apache.beam.sdk.transforms.DoFn;
 import org.apache.beam.sdk.transforms.PTransform;
 import org.apache.beam.sdk.transforms.ParDo;
-import org.apache.beam.sdk.transforms.Sum;
 import org.apache.beam.sdk.transforms.join.CoGbkResult;
 import org.apache.beam.sdk.transforms.join.CoGroupByKey;
 import org.apache.beam.sdk.transforms.join.KeyedPCollectionTuple;
@@ -323,56 +323,52 @@ public class WinningBids extends PTransform<PCollection<Event>, PCollection<Auct
 
     // Find the highest price valid bid for each closed auction.
     return
-        // Join auctions and bids.
-        KeyedPCollectionTuple.of(NexmarkQuery.AUCTION_TAG, auctionsById)
-            .and(NexmarkQuery.BID_TAG, bidsByAuctionId)
-            .apply(CoGroupByKey.<Long>create())
-
-            // Filter and select.
-            .apply(name + ".Join",
-                ParDo.of(new DoFn<KV<Long, CoGbkResult>, AuctionBid>() {
-                      final Aggregator<Long, Long> noAuctionCounter =
-                          createAggregator("noAuction", Sum.ofLongs());
-                      final Aggregator<Long, Long> underReserveCounter =
-                          createAggregator("underReserve", Sum.ofLongs());
-                      final Aggregator<Long, Long> noValidBidsCounter =
-                          createAggregator("noValidBids", Sum.ofLongs());
-
-
-                      @ProcessElement
-                      public void processElement(ProcessContext c) {
-                        Auction auction =
-                            c.element().getValue().getOnly(NexmarkQuery.AUCTION_TAG, null);
-                        if (auction == null) {
-                          // We have bids without a matching auction. Give up.
-                          noAuctionCounter.addValue(1L);
-                          return;
-                        }
-                        // Find the current winning bid for auction.
-                        // The earliest bid with the maximum price above the reserve wins.
-                        Bid bestBid = null;
-                        for (Bid bid : c.element().getValue().getAll(NexmarkQuery.BID_TAG)) {
-                          // Bids too late for their auction will have been
-                          // filtered out by the window merge function.
-                          checkState(bid.dateTime < auction.expires);
-                          if (bid.price < auction.reserve) {
-                            // Bid price is below auction reserve.
-                            underReserveCounter.addValue(1L);
-                            continue;
-                          }
-
-                          if (bestBid == null
-                              || Bid.PRICE_THEN_DESCENDING_TIME.compare(bid, bestBid) > 0) {
-                            bestBid = bid;
-                          }
-                        }
-                        if (bestBid == null) {
-                          // We don't have any valid bids for auction.
-                          noValidBidsCounter.addValue(1L);
-                          return;
-                        }
-                        c.output(new AuctionBid(auction, bestBid));
-                      }
-                    }));
+      // Join auctions and bids.
+      KeyedPCollectionTuple.of(NexmarkQuery.AUCTION_TAG, auctionsById)
+        .and(NexmarkQuery.BID_TAG, bidsByAuctionId)
+        .apply(CoGroupByKey.<Long>create())
+        // Filter and select.
+        .apply(name + ".Join",
+          ParDo.of(new DoFn<KV<Long, CoGbkResult>, AuctionBid>() {
+            private final Counter noAuctionCounter = Metrics.counter(name, "noAuction");
+            private final Counter underReserveCounter = Metrics.counter(name, "underReserve");
+            private final Counter noValidBidsCounter = Metrics.counter(name, "noValidBids");
+
+            @ProcessElement
+            public void processElement(ProcessContext c) {
+              Auction auction =
+                  c.element().getValue().getOnly(NexmarkQuery.AUCTION_TAG, null);
+              if (auction == null) {
+                // We have bids without a matching auction. Give up.
+                noAuctionCounter.inc();
+                return;
+              }
+              // Find the current winning bid for auction.
+              // The earliest bid with the maximum price above the reserve wins.
+              Bid bestBid = null;
+              for (Bid bid : c.element().getValue().getAll(NexmarkQuery.BID_TAG)) {
+                // Bids too late for their auction will have been
+                // filtered out by the window merge function.
+                checkState(bid.dateTime < auction.expires);
+                if (bid.price < auction.reserve) {
+                  // Bid price is below auction reserve.
+                  underReserveCounter.inc();
+                  continue;
+                }
+
+                if (bestBid == null
+                    || Bid.PRICE_THEN_DESCENDING_TIME.compare(bid, bestBid) > 0) {
+                  bestBid = bid;
+                }
+              }
+              if (bestBid == null) {
+                // We don't have any valid bids for auction.
+                noValidBidsCounter.inc();
+                return;
+              }
+              c.output(new AuctionBid(auction, bestBid));
+            }
+          }
+        ));
   }
 }

http://git-wip-us.apache.org/repos/asf/beam/blob/b438fa7d/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/drivers/NexmarkGoogleRunner.java
----------------------------------------------------------------------
diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/drivers/NexmarkGoogleRunner.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/drivers/NexmarkGoogleRunner.java
index 7ffd47a..935bf0d 100644
--- a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/drivers/NexmarkGoogleRunner.java
+++ b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/drivers/NexmarkGoogleRunner.java
@@ -130,7 +130,9 @@ class NexmarkGoogleRunner extends NexmarkRunner<NexmarkGoogleDriver.NexmarkGoogl
           NexmarkUtils.console("%s publisher (%d events)", state, numEvents);
           return;
         case RUNNING:
-          numEvents = getLong(job, publisherMonitor.getElementCounter());
+          //TODO Ismael Validate that this counter is ok
+          numEvents =
+            getCounterMetric(job, publisherMonitor.name, publisherMonitor.prefix + ".elements", -1);
           if (startMsSinceEpoch < 0 && numEvents > 0) {
             startMsSinceEpoch = System.currentTimeMillis();
             endMsSinceEpoch = startMsSinceEpoch

http://git-wip-us.apache.org/repos/asf/beam/blob/b438fa7d/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query0.java
----------------------------------------------------------------------
diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query0.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query0.java
index f60d5de..84696c4 100644
--- a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query0.java
+++ b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query0.java
@@ -28,10 +28,10 @@ import org.apache.beam.integration.nexmark.model.Event;
 import org.apache.beam.integration.nexmark.model.KnownSize;
 import org.apache.beam.sdk.coders.Coder;
 import org.apache.beam.sdk.coders.CoderException;
-import org.apache.beam.sdk.transforms.Aggregator;
+import org.apache.beam.sdk.metrics.Counter;
+import org.apache.beam.sdk.metrics.Metrics;
 import org.apache.beam.sdk.transforms.DoFn;
 import org.apache.beam.sdk.transforms.ParDo;
-import org.apache.beam.sdk.transforms.Sum;
 import org.apache.beam.sdk.values.PCollection;
 
 /**
@@ -49,15 +49,15 @@ public class Query0 extends NexmarkQuery {
         // Force round trip through coder.
         .apply(name + ".Serialize",
             ParDo.of(new DoFn<Event, Event>() {
-                  private final Aggregator<Long, Long> bytes =
-                      createAggregator("bytes", Sum.ofLongs());
+                  private final Counter bytesMetric =
+                    Metrics.counter(name , "bytes");
 
                   @ProcessElement
                   public void processElement(ProcessContext c) throws CoderException, IOException {
                     ByteArrayOutputStream outStream = new ByteArrayOutputStream();
                     coder.encode(c.element(), outStream, Coder.Context.OUTER);
                     byte[] byteArray = outStream.toByteArray();
-                    bytes.addValue((long) byteArray.length);
+                    bytesMetric.inc((long) byteArray.length);
                     ByteArrayInputStream inStream = new ByteArrayInputStream(byteArray);
                     Event event = coder.decode(inStream, Coder.Context.OUTER);
                     c.output(event);

http://git-wip-us.apache.org/repos/asf/beam/blob/b438fa7d/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query10.java
----------------------------------------------------------------------
diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query10.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query10.java
index 5246427..d9b3557 100644
--- a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query10.java
+++ b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query10.java
@@ -34,12 +34,12 @@ import org.apache.beam.integration.nexmark.model.Done;
 import org.apache.beam.integration.nexmark.model.Event;
 import org.apache.beam.integration.nexmark.model.KnownSize;
 import org.apache.beam.sdk.coders.Coder;
+import org.apache.beam.sdk.metrics.Counter;
+import org.apache.beam.sdk.metrics.Metrics;
 import org.apache.beam.sdk.options.GcsOptions;
-import org.apache.beam.sdk.transforms.Aggregator;
 import org.apache.beam.sdk.transforms.DoFn;
 import org.apache.beam.sdk.transforms.GroupByKey;
 import org.apache.beam.sdk.transforms.ParDo;
-import org.apache.beam.sdk.transforms.Sum;
 import org.apache.beam.sdk.transforms.windowing.AfterEach;
 import org.apache.beam.sdk.transforms.windowing.AfterFirst;
 import org.apache.beam.sdk.transforms.windowing.AfterPane;
@@ -184,196 +184,189 @@ public class Query10 extends NexmarkQuery {
   private PCollection<Done> applyTyped(PCollection<Event> events) {
     final int numLogShards = maxNumWorkers * NUM_SHARDS_PER_WORKER;
 
-    return events.apply(name + ".ShardEvents",
-            ParDo.of(new DoFn<Event, KV<String, Event>>() {
-                      final Aggregator<Long, Long> lateCounter =
-                          createAggregator("actuallyLateEvent", Sum.ofLongs());
-                      final Aggregator<Long, Long> onTimeCounter =
-                          createAggregator("actuallyOnTimeEvent", Sum.ofLongs());
+    return events
+      .apply(name + ".ShardEvents",
+        ParDo.of(new DoFn<Event, KV<String, Event>>() {
+          private final Counter lateCounter = Metrics.counter(name , "actuallyLateEvent");
+          private final Counter onTimeCounter = Metrics.counter(name , "onTimeCounter");
 
-                      @ProcessElement
-                      public void processElement(ProcessContext c) {
-                        if (c.element().hasAnnotation("LATE")) {
-                          lateCounter.addValue(1L);
-                          LOG.error("Observed late: %s", c.element());
-                        } else {
-                          onTimeCounter.addValue(1L);
-                        }
-                        int shardNum = (int) Math.abs((long) c.element().hashCode() % numLogShards);
-                        String shard = String.format("shard-%05d-of-%05d", shardNum, numLogShards);
-                        c.output(KV.of(shard, c.element()));
-                      }
-                    }))
-        .apply(name + ".WindowEvents",
-                Window.<KV<String, Event>>into(
-            FixedWindows.of(Duration.standardSeconds(configuration.windowSizeSec)))
-            .triggering(AfterEach.inOrder(
-                Repeatedly
-                    .forever(AfterPane.elementCountAtLeast(configuration.maxLogEvents))
-                    .orFinally(AfterWatermark.pastEndOfWindow()),
-                Repeatedly.forever(
-                    AfterFirst.of(AfterPane.elementCountAtLeast(configuration.maxLogEvents),
-                        AfterProcessingTime.pastFirstElementInPane()
-                                           .plusDelayOf(LATE_BATCHING_PERIOD)))))
-            .discardingFiredPanes()
-            // Use a 1 day allowed lateness so that any forgotten hold will stall the
-            // pipeline for that period and be very noticeable.
-            .withAllowedLateness(Duration.standardDays(1)))
-        .apply(name + ".GroupByKey", GroupByKey.<String, Event>create())
-        .apply(name + ".CheckForLateEvents",
-            ParDo.of(new DoFn<KV<String, Iterable<Event>>,
-                     KV<String, Iterable<Event>>>() {
-                   final Aggregator<Long, Long> earlyCounter =
-                       createAggregator("earlyShard", Sum.ofLongs());
-                   final Aggregator<Long, Long> onTimeCounter =
-                       createAggregator("onTimeShard", Sum.ofLongs());
-                   final Aggregator<Long, Long> lateCounter =
-                       createAggregator("lateShard", Sum.ofLongs());
-                   final Aggregator<Long, Long> unexpectedLatePaneCounter =
-                       createAggregator("ERROR_unexpectedLatePane", Sum.ofLongs());
-                   final Aggregator<Long, Long> unexpectedOnTimeElementCounter =
-                       createAggregator("ERROR_unexpectedOnTimeElement", Sum.ofLongs());
+          @ProcessElement
+          public void processElement(ProcessContext c) {
+            if (c.element().hasAnnotation("LATE")) {
+              lateCounter.inc();
+              LOG.error("Observed late: %s", c.element());
+            } else {
+              onTimeCounter.inc();
+            }
+            int shardNum = (int) Math.abs((long) c.element().hashCode() % numLogShards);
+            String shard = String.format("shard-%05d-of-%05d", shardNum, numLogShards);
+            c.output(KV.of(shard, c.element()));
+          }
+        }))
+      .apply(name + ".WindowEvents",
+        Window.<KV<String, Event>>into(
+          FixedWindows.of(Duration.standardSeconds(configuration.windowSizeSec)))
+          .triggering(AfterEach.inOrder(
+              Repeatedly
+                  .forever(AfterPane.elementCountAtLeast(configuration.maxLogEvents))
+                  .orFinally(AfterWatermark.pastEndOfWindow()),
+              Repeatedly.forever(
+                  AfterFirst.of(AfterPane.elementCountAtLeast(configuration.maxLogEvents),
+                      AfterProcessingTime.pastFirstElementInPane()
+                                         .plusDelayOf(LATE_BATCHING_PERIOD)))))
+          .discardingFiredPanes()
+          // Use a 1 day allowed lateness so that any forgotten hold will stall the
+          // pipeline for that period and be very noticeable.
+          .withAllowedLateness(Duration.standardDays(1)))
+      .apply(name + ".GroupByKey", GroupByKey.<String, Event>create())
+      .apply(name + ".CheckForLateEvents",
+        ParDo.of(new DoFn<KV<String, Iterable<Event>>,
+                 KV<String, Iterable<Event>>>() {
+          private final Counter earlyCounter = Metrics.counter(name , "earlyShard");
+          private final Counter onTimeCounter = Metrics.counter(name , "onTimeShard");
+          private final Counter lateCounter = Metrics.counter(name , "lateShard");
+          private final Counter unexpectedLatePaneCounter =
+            Metrics.counter(name , "ERROR_unexpectedLatePane");
+          private final Counter unexpectedOnTimeElementCounter =
+            Metrics.counter(name , "ERROR_unexpectedOnTimeElement");
 
-                   @ProcessElement
-                   public void processElement(ProcessContext c, BoundedWindow window) {
-                     int numLate = 0;
-                     int numOnTime = 0;
-                     for (Event event : c.element().getValue()) {
-                       if (event.hasAnnotation("LATE")) {
-                         numLate++;
-                       } else {
-                         numOnTime++;
-                       }
-                     }
-                     String shard = c.element().getKey();
-                     LOG.error(
-                         "%s with timestamp %s has %d actually late and %d on-time "
-                             + "elements in pane %s for window %s",
-                         shard, c.timestamp(), numLate, numOnTime, c.pane(),
-                         window.maxTimestamp());
-                     if (c.pane().getTiming() == PaneInfo.Timing.LATE) {
-                       if (numLate == 0) {
-                         LOG.error(
-                             "ERROR! No late events in late pane for %s", shard);
-                         unexpectedLatePaneCounter.addValue(1L);
-                       }
-                       if (numOnTime > 0) {
-                         LOG.error(
-                             "ERROR! Have %d on-time events in late pane for %s",
-                             numOnTime, shard);
-                         unexpectedOnTimeElementCounter.addValue(1L);
-                       }
-                       lateCounter.addValue(1L);
-                     } else if (c.pane().getTiming() == PaneInfo.Timing.EARLY) {
-                       if (numOnTime + numLate < configuration.maxLogEvents) {
-                         LOG.error(
-                             "ERROR! Only have %d events in early pane for %s",
-                             numOnTime + numLate, shard);
-                       }
-                       earlyCounter.addValue(1L);
-                     } else {
-                       onTimeCounter.addValue(1L);
-                     }
-                     c.output(c.element());
-                   }
-                 }))
-        .apply(name + ".UploadEvents",
-            ParDo.of(new DoFn<KV<String, Iterable<Event>>,
-                     KV<Void, OutputFile>>() {
-                   final Aggregator<Long, Long> savedFileCounter =
-                       createAggregator("savedFile", Sum.ofLongs());
-                   final Aggregator<Long, Long> writtenRecordsCounter =
-                       createAggregator("writtenRecords", Sum.ofLongs());
+          @ProcessElement
+          public void processElement(ProcessContext c, BoundedWindow window) {
+            int numLate = 0;
+            int numOnTime = 0;
+            for (Event event : c.element().getValue()) {
+              if (event.hasAnnotation("LATE")) {
+                numLate++;
+              } else {
+                numOnTime++;
+              }
+            }
+            String shard = c.element().getKey();
+            LOG.error(
+                "%s with timestamp %s has %d actually late and %d on-time "
+                    + "elements in pane %s for window %s",
+                shard, c.timestamp(), numLate, numOnTime, c.pane(),
+                window.maxTimestamp());
+            if (c.pane().getTiming() == PaneInfo.Timing.LATE) {
+              if (numLate == 0) {
+                LOG.error(
+                    "ERROR! No late events in late pane for %s", shard);
+                unexpectedLatePaneCounter.inc();
+              }
+              if (numOnTime > 0) {
+                LOG.error(
+                    "ERROR! Have %d on-time events in late pane for %s",
+                    numOnTime, shard);
+                unexpectedOnTimeElementCounter.inc();
+              }
+              lateCounter.inc();
+            } else if (c.pane().getTiming() == PaneInfo.Timing.EARLY) {
+              if (numOnTime + numLate < configuration.maxLogEvents) {
+                LOG.error(
+                    "ERROR! Only have %d events in early pane for %s",
+                    numOnTime + numLate, shard);
+              }
+              earlyCounter.inc();
+            } else {
+              onTimeCounter.inc();
+            }
+            c.output(c.element());
+          }
+        }))
+      .apply(name + ".UploadEvents",
+        ParDo.of(new DoFn<KV<String, Iterable<Event>>,
+                 KV<Void, OutputFile>>() {
+          private final Counter savedFileCounter = Metrics.counter(name , "savedFile");
+          private final Counter writtenRecordsCounter = Metrics.counter(name , "writtenRecords");
 
-                   @ProcessElement
-                   public void processElement(ProcessContext c, BoundedWindow window)
-                           throws IOException {
-                     String shard = c.element().getKey();
-                     GcsOptions options = c.getPipelineOptions().as(GcsOptions.class);
-                     OutputFile outputFile = outputFileFor(window, shard, c.pane());
-                     LOG.error(
-                         "Writing %s with record timestamp %s, window timestamp %s, pane %s",
-                         shard, c.timestamp(), window.maxTimestamp(), c.pane());
-                     if (outputFile.filename != null) {
-                       LOG.error("Beginning write to '%s'", outputFile.filename);
-                       int n = 0;
-                       try (OutputStream output =
-                                Channels.newOutputStream(openWritableGcsFile(options, outputFile
-                                    .filename))) {
-                         for (Event event : c.element().getValue()) {
-                           Event.CODER.encode(event, output, Coder.Context.OUTER);
-                           writtenRecordsCounter.addValue(1L);
-                           if (++n % 10000 == 0) {
-                             LOG.error("So far written %d records to '%s'", n,
-                                 outputFile.filename);
-                           }
-                         }
-                       }
-                       LOG.error("Written all %d records to '%s'", n, outputFile.filename);
-                     }
-                     savedFileCounter.addValue(1L);
-                     c.output(KV.<Void, OutputFile>of(null, outputFile));
-                   }
-                 }))
-        // Clear fancy triggering from above.
-        .apply(name + ".WindowLogFiles", Window.<KV<Void, OutputFile>>into(
-            FixedWindows.of(Duration.standardSeconds(configuration.windowSizeSec)))
-            .triggering(AfterWatermark.pastEndOfWindow())
-            // We expect no late data here, but we'll assume the worst so we can detect any.
-            .withAllowedLateness(Duration.standardDays(1))
-            .discardingFiredPanes())
+            @ProcessElement
+            public void processElement(ProcessContext c, BoundedWindow window)
+                    throws IOException {
+              String shard = c.element().getKey();
+              GcsOptions options = c.getPipelineOptions().as(GcsOptions.class);
+              OutputFile outputFile = outputFileFor(window, shard, c.pane());
+              LOG.error(
+                  "Writing %s with record timestamp %s, window timestamp %s, pane %s",
+                  shard, c.timestamp(), window.maxTimestamp(), c.pane());
+              if (outputFile.filename != null) {
+                LOG.error("Beginning write to '%s'", outputFile.filename);
+                int n = 0;
+                try (OutputStream output =
+                         Channels.newOutputStream(openWritableGcsFile(options, outputFile
+                             .filename))) {
+                  for (Event event : c.element().getValue()) {
+                    Event.CODER.encode(event, output, Coder.Context.OUTER);
+                    writtenRecordsCounter.inc();
+                    if (++n % 10000 == 0) {
+                      LOG.error("So far written %d records to '%s'", n,
+                          outputFile.filename);
+                    }
+                  }
+                }
+                LOG.error("Written all %d records to '%s'", n, outputFile.filename);
+              }
+              savedFileCounter.inc();
+              c.output(KV.<Void, OutputFile>of(null, outputFile));
+            }
+          }))
+      // Clear fancy triggering from above.
+      .apply(name + ".WindowLogFiles", Window.<KV<Void, OutputFile>>into(
+        FixedWindows.of(Duration.standardSeconds(configuration.windowSizeSec)))
+        .triggering(AfterWatermark.pastEndOfWindow())
+        // We expect no late data here, but we'll assume the worst so we can detect any.
+        .withAllowedLateness(Duration.standardDays(1))
+        .discardingFiredPanes())
       // this GroupByKey allows to have one file per window
       .apply(name + ".GroupByKey2", GroupByKey.<Void, OutputFile>create())
-        .apply(name + ".Index",
-            ParDo.of(new DoFn<KV<Void, Iterable<OutputFile>>, Done>() {
-                   final Aggregator<Long, Long> unexpectedLateCounter =
-                       createAggregator("ERROR_unexpectedLate", Sum.ofLongs());
-                   final Aggregator<Long, Long> unexpectedEarlyCounter =
-                       createAggregator("ERROR_unexpectedEarly", Sum.ofLongs());
-                   final Aggregator<Long, Long> unexpectedIndexCounter =
-                       createAggregator("ERROR_unexpectedIndex", Sum.ofLongs());
-                   final Aggregator<Long, Long> finalizedCounter =
-                       createAggregator("indexed", Sum.ofLongs());
+      .apply(name + ".Index",
+        ParDo.of(new DoFn<KV<Void, Iterable<OutputFile>>, Done>() {
+          private final Counter unexpectedLateCounter =
+            Metrics.counter(name , "ERROR_unexpectedLate");
+          private final Counter unexpectedEarlyCounter =
+              Metrics.counter(name , "ERROR_unexpectedEarly");
+          private final Counter unexpectedIndexCounter =
+              Metrics.counter(name , "ERROR_unexpectedIndex");
+          private final Counter finalizedCounter = Metrics.counter(name , "indexed");
 
-                   @ProcessElement
-                   public void processElement(ProcessContext c, BoundedWindow window)
-                           throws IOException {
-                     if (c.pane().getTiming() == Timing.LATE) {
-                       unexpectedLateCounter.addValue(1L);
-                       LOG.error("ERROR! Unexpected LATE pane: %s", c.pane());
-                     } else if (c.pane().getTiming() == Timing.EARLY) {
-                       unexpectedEarlyCounter.addValue(1L);
-                       LOG.error("ERROR! Unexpected EARLY pane: %s", c.pane());
-                     } else if (c.pane().getTiming() == Timing.ON_TIME
-                         && c.pane().getIndex() != 0) {
-                       unexpectedIndexCounter.addValue(1L);
-                       LOG.error("ERROR! Unexpected ON_TIME pane index: %s", c.pane());
-                     } else {
-                       GcsOptions options = c.getPipelineOptions().as(GcsOptions.class);
-                       LOG.error(
-                           "Index with record timestamp %s, window timestamp %s, pane %s",
-                           c.timestamp(), window.maxTimestamp(), c.pane());
+          @ProcessElement
+          public void processElement(ProcessContext c, BoundedWindow window)
+                  throws IOException {
+            if (c.pane().getTiming() == Timing.LATE) {
+              unexpectedLateCounter.inc();
+              LOG.error("ERROR! Unexpected LATE pane: %s", c.pane());
+            } else if (c.pane().getTiming() == Timing.EARLY) {
+              unexpectedEarlyCounter.inc();
+              LOG.error("ERROR! Unexpected EARLY pane: %s", c.pane());
+            } else if (c.pane().getTiming() == Timing.ON_TIME
+                && c.pane().getIndex() != 0) {
+              unexpectedIndexCounter.inc();
+              LOG.error("ERROR! Unexpected ON_TIME pane index: %s", c.pane());
+            } else {
+              GcsOptions options = c.getPipelineOptions().as(GcsOptions.class);
+              LOG.error(
+                  "Index with record timestamp %s, window timestamp %s, pane %s",
+                  c.timestamp(), window.maxTimestamp(), c.pane());
 
-                       @Nullable String filename = indexPathFor(window);
-                       if (filename != null) {
-                         LOG.error("Beginning write to '%s'", filename);
-                         int n = 0;
-                         try (OutputStream output =
-                                  Channels.newOutputStream(
-                                      openWritableGcsFile(options, filename))) {
-                           for (OutputFile outputFile : c.element().getValue()) {
-                             output.write(outputFile.toString().getBytes());
-                             n++;
-                           }
-                         }
-                         LOG.error("Written all %d lines to '%s'", n, filename);
-                       }
-                       c.output(
-                           new Done("written for timestamp " + window.maxTimestamp()));
-                       finalizedCounter.addValue(1L);
-                     }
-                   }
-                 }));
+              @Nullable String filename = indexPathFor(window);
+              if (filename != null) {
+                LOG.error("Beginning write to '%s'", filename);
+                int n = 0;
+                try (OutputStream output =
+                         Channels.newOutputStream(
+                             openWritableGcsFile(options, filename))) {
+                  for (OutputFile outputFile : c.element().getValue()) {
+                    output.write(outputFile.toString().getBytes());
+                    n++;
+                  }
+                }
+                LOG.error("Written all %d lines to '%s'", n, filename);
+              }
+              c.output(
+                  new Done("written for timestamp " + window.maxTimestamp()));
+              finalizedCounter.inc();
+            }
+          }
+        }));
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/beam/blob/b438fa7d/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query3.java
----------------------------------------------------------------------
diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query3.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query3.java
index ba31e9f..12b16f1 100644
--- a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query3.java
+++ b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query3.java
@@ -20,7 +20,6 @@ package org.apache.beam.integration.nexmark.queries;
 import java.io.IOException;
 import java.util.ArrayList;
 import java.util.List;
-import javax.annotation.Nullable;
 import org.apache.beam.integration.nexmark.NexmarkConfiguration;
 import org.apache.beam.integration.nexmark.NexmarkQuery;
 import org.apache.beam.integration.nexmark.NexmarkUtils;
@@ -30,12 +29,12 @@ import org.apache.beam.integration.nexmark.model.KnownSize;
 import org.apache.beam.integration.nexmark.model.NameCityStateId;
 import org.apache.beam.integration.nexmark.model.Person;
 import org.apache.beam.sdk.coders.ListCoder;
-import org.apache.beam.sdk.transforms.Aggregator;
+import org.apache.beam.sdk.metrics.Counter;
+import org.apache.beam.sdk.metrics.Metrics;
 import org.apache.beam.sdk.transforms.DoFn;
 import org.apache.beam.sdk.transforms.Filter;
 import org.apache.beam.sdk.transforms.ParDo;
 import org.apache.beam.sdk.transforms.SerializableFunction;
-import org.apache.beam.sdk.transforms.Sum;
 import org.apache.beam.sdk.transforms.join.CoGbkResult;
 import org.apache.beam.sdk.transforms.join.CoGroupByKey;
 import org.apache.beam.sdk.transforms.join.KeyedPCollectionTuple;
@@ -81,14 +80,7 @@ public class Query3 extends NexmarkQuery {
 
   public Query3(NexmarkConfiguration configuration) {
     super(configuration, "Query3");
-    joinDoFn = new JoinDoFn(configuration.maxAuctionsWaitingTime);
-
-  }
-
-  @Override
-  @Nullable
-  public Aggregator<Long, Long> getFatalCount() {
-    return joinDoFn.fatalCounter;
+    joinDoFn = new JoinDoFn(name, configuration.maxAuctionsWaitingTime);
   }
 
   private PCollection<NameCityStateId> applyTyped(PCollection<Event> events) {
@@ -195,8 +187,6 @@ public class Query3 extends NexmarkQuery {
 
     private static final String PERSON_STATE_EXPIRING = "personStateExpiring";
 
-    public final Aggregator<Long, Long> fatalCounter = createAggregator("fatal", Sum.ofLongs());
-
     @StateId(AUCTIONS)
     private final StateSpec<Object, ValueState<List<Auction>>> auctionsSpec =
         StateSpecs.value(ListCoder.of(Auction.CODER));
@@ -204,19 +194,25 @@ public class Query3 extends NexmarkQuery {
     @TimerId(PERSON_STATE_EXPIRING)
     private final TimerSpec timerSpec = TimerSpecs.timer(TimeDomain.EVENT_TIME);
 
-    private final Aggregator<Long, Long> newAuctionCounter =
-        createAggregator("newAuction", Sum.ofLongs());
-    private final Aggregator<Long, Long> newPersonCounter =
-        createAggregator("newPerson", Sum.ofLongs());
-    private final Aggregator<Long, Long> newNewOutputCounter =
-        createAggregator("newNewOutput", Sum.ofLongs());
-    private final Aggregator<Long, Long> newOldOutputCounter =
-        createAggregator("newOldOutput", Sum.ofLongs());
-    private final Aggregator<Long, Long> oldNewOutputCounter =
-        createAggregator("oldNewOutput", Sum.ofLongs());
+    // Used to refer the metrics namespace
+    private final String name;
 
-    private JoinDoFn(int maxAuctionsWaitingTime) {
+    private final Counter newAuctionCounter;
+    private final Counter newPersonCounter;
+    private final Counter newNewOutputCounter;
+    private final Counter newOldOutputCounter;
+    private final Counter oldNewOutputCounter;
+    private final Counter fatalCounter;
+
+    private JoinDoFn(String name, int maxAuctionsWaitingTime) {
+      this.name = name;
       this.maxAuctionsWaitingTime = maxAuctionsWaitingTime;
+      newAuctionCounter = Metrics.counter(name, "newAuction");
+      newPersonCounter = Metrics.counter(name, "newPerson");
+      newNewOutputCounter = Metrics.counter(name, "newNewOutput");
+      newOldOutputCounter = Metrics.counter(name, "newOldOutput");
+      oldNewOutputCounter = Metrics.counter(name, "oldNewOutput");
+      fatalCounter = Metrics.counter(name , "fatal");
     }
 
     @ProcessElement
@@ -232,14 +228,13 @@ public class Query3 extends NexmarkQuery {
       // we need to wait for the pending ReduceFn API.
 
       Person existingPerson = personState.read();
-
       if (existingPerson != null) {
         // We've already seen the new person event for this person id.
         // We can join with any new auctions on-the-fly without needing any
         // additional persistent state.
         for (Auction newAuction : c.element().getValue().getAll(AUCTION_TAG)) {
-          newAuctionCounter.addValue(1L);
-          newOldOutputCounter.addValue(1L);
+          newAuctionCounter.inc();
+          newOldOutputCounter.inc();
           c.output(KV.of(newAuction, existingPerson));
         }
         return;
@@ -255,24 +250,24 @@ public class Query3 extends NexmarkQuery {
           } else {
             LOG.error("**** conflicting persons {} and {} ****", theNewPerson, newPerson);
           }
-          fatalCounter.addValue(1L);
+          fatalCounter.inc();
           continue;
         }
-        newPersonCounter.addValue(1L);
+        newPersonCounter.inc();
         // We've now seen the person for this person id so can flush any
         // pending auctions for the same seller id (an auction is done by only one seller).
         List<Auction> pendingAuctions = auctionsState.read();
         if (pendingAuctions != null) {
           for (Auction pendingAuction : pendingAuctions) {
-            oldNewOutputCounter.addValue(1L);
+            oldNewOutputCounter.inc();
             c.output(KV.of(pendingAuction, newPerson));
           }
           auctionsState.clear();
         }
         // Also deal with any new auctions.
         for (Auction newAuction : c.element().getValue().getAll(AUCTION_TAG)) {
-          newAuctionCounter.addValue(1L);
-          newNewOutputCounter.addValue(1L);
+          newAuctionCounter.inc();
+          newNewOutputCounter.inc();
           c.output(KV.of(newAuction, newPerson));
         }
         // Remember this person for any future auctions.
@@ -293,17 +288,17 @@ public class Query3 extends NexmarkQuery {
         pendingAuctions = new ArrayList<>();
       }
       for (Auction newAuction : c.element().getValue().getAll(AUCTION_TAG)) {
-        newAuctionCounter.addValue(1L);
+        newAuctionCounter.inc();
         pendingAuctions.add(newAuction);
       }
       auctionsState.write(pendingAuctions);
     }
-  @OnTimer(PERSON_STATE_EXPIRING)
-  public void onTimerCallback(
-      OnTimerContext context,
-      @StateId(PERSON) ValueState<Person> personState) {
-      personState.clear();
-  }
 
+    @OnTimer(PERSON_STATE_EXPIRING)
+    public void onTimerCallback(
+        OnTimerContext context,
+        @StateId(PERSON) ValueState<Person> personState) {
+        personState.clear();
+    }
   }
 }

http://git-wip-us.apache.org/repos/asf/beam/blob/b438fa7d/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/sources/BoundedEventSource.java
----------------------------------------------------------------------
diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/sources/BoundedEventSource.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/sources/BoundedEventSource.java
index be74151..43d6690 100644
--- a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/sources/BoundedEventSource.java
+++ b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/sources/BoundedEventSource.java
@@ -156,7 +156,7 @@ public class BoundedEventSource extends BoundedSource<Event> {
   }
 
   @Override
-  public List<BoundedEventSource> splitIntoBundles(
+  public List<BoundedEventSource> split(
       long desiredBundleSizeBytes, PipelineOptions options) {
     NexmarkUtils.info("slitting bounded source %s into %d sub-sources", config, numEventGenerators);
     List<BoundedEventSource> results = new ArrayList<>();

http://git-wip-us.apache.org/repos/asf/beam/blob/b438fa7d/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/sources/UnboundedEventSource.java
----------------------------------------------------------------------
diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/sources/UnboundedEventSource.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/sources/UnboundedEventSource.java
index 286c576..c3c6eb0 100644
--- a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/sources/UnboundedEventSource.java
+++ b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/sources/UnboundedEventSource.java
@@ -289,7 +289,7 @@ public class UnboundedEventSource extends UnboundedSource<Event, Generator.Check
   }
 
   @Override
-  public List<UnboundedEventSource> generateInitialSplits(
+  public List<UnboundedEventSource> split(
       int desiredNumSplits, PipelineOptions options) {
     LOG.trace("splitting unbounded source into {} sub-sources", numEventGenerators);
     List<UnboundedEventSource> results = new ArrayList<>();

http://git-wip-us.apache.org/repos/asf/beam/blob/b438fa7d/integration/java/nexmark/src/test/java/org/apache/beam/integration/nexmark/sources/BoundedEventSourceTest.java
----------------------------------------------------------------------
diff --git a/integration/java/nexmark/src/test/java/org/apache/beam/integration/nexmark/sources/BoundedEventSourceTest.java b/integration/java/nexmark/src/test/java/org/apache/beam/integration/nexmark/sources/BoundedEventSourceTest.java
index 3f85bab..c5d7725 100644
--- a/integration/java/nexmark/src/test/java/org/apache/beam/integration/nexmark/sources/BoundedEventSourceTest.java
+++ b/integration/java/nexmark/src/test/java/org/apache/beam/integration/nexmark/sources/BoundedEventSourceTest.java
@@ -66,6 +66,6 @@ public class BoundedEventSourceTest {
     long n = 200L;
     BoundedEventSource source = new BoundedEventSource(makeConfig(n), 1);
     SourceTestUtils.assertSourcesEqualReferenceSource(
-        source, source.splitIntoBundles(10, options), options);
+        source, source.split(10, options), options);
   }
 }


[27/55] [abbrv] beam git commit: Remove NexmarkDrivers and make execution runner-agnostic

Posted by ie...@apache.org.
Remove NexmarkDrivers and make execution runner-agnostic

This configuration should be external to the benchmark to avoid
unexpected dependencies and to have a more Beam like
(runner-independent) apprach.

Add maven profiles to execute NexMark in the different runners

Fix compile after extra PubSubIO refactor and remove PubsubClient.
PubsubClient was used to create and reuse topics, this logic should not
be part of Nexmark because this add extra complexity/dependencies. A
simple script should do this, or the user should provide the
corresponding topics.


Project: http://git-wip-us.apache.org/repos/asf/beam/repo
Commit: http://git-wip-us.apache.org/repos/asf/beam/commit/a6dbdfa5
Tree: http://git-wip-us.apache.org/repos/asf/beam/tree/a6dbdfa5
Diff: http://git-wip-us.apache.org/repos/asf/beam/diff/a6dbdfa5

Branch: refs/heads/master
Commit: a6dbdfa5457344191ebba383174063270239d9fa
Parents: 8b96949
Author: Ismaël Mejía <ie...@apache.org>
Authored: Sat Apr 29 16:50:22 2017 +0200
Committer: Ismaël Mejía <ie...@gmail.com>
Committed: Wed Aug 23 19:07:28 2017 +0200

----------------------------------------------------------------------
 integration/java/nexmark/pom.xml                | 294 +++++-----
 .../beam/integration/nexmark/NexmarkDriver.java |   9 +
 .../beam/integration/nexmark/NexmarkRunner.java | 106 ++--
 .../nexmark/drivers/NexmarkApexDriver.java      |  50 --
 .../nexmark/drivers/NexmarkApexRunner.java      |  65 ---
 .../nexmark/drivers/NexmarkDirectDriver.java    |  49 --
 .../nexmark/drivers/NexmarkDirectRunner.java    |  60 --
 .../nexmark/drivers/NexmarkFlinkDriver.java     |  50 --
 .../nexmark/drivers/NexmarkFlinkRunner.java     |  55 --
 .../nexmark/drivers/NexmarkGoogleDriver.java    |  67 ---
 .../nexmark/drivers/NexmarkGoogleRunner.java    | 165 ------
 .../nexmark/drivers/NexmarkSparkDriver.java     |  48 --
 .../nexmark/drivers/NexmarkSparkRunner.java     |  56 --
 .../nexmark/drivers/package-info.java           |  22 -
 .../integration/nexmark/io/PubsubClient.java    | 543 -------------------
 .../integration/nexmark/io/PubsubHelper.java    | 215 --------
 .../nexmark/io/PubsubJsonClient.java            | 318 -----------
 .../nexmark/io/PubsubTestClient.java            | 436 ---------------
 .../integration/nexmark/io/package-info.java    |  22 -
 .../nexmark/sources/BoundedEventSourceTest.java |   8 +-
 20 files changed, 172 insertions(+), 2466 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/beam/blob/a6dbdfa5/integration/java/nexmark/pom.xml
----------------------------------------------------------------------
diff --git a/integration/java/nexmark/pom.xml b/integration/java/nexmark/pom.xml
index 103c18f..fb213e9 100644
--- a/integration/java/nexmark/pom.xml
+++ b/integration/java/nexmark/pom.xml
@@ -31,44 +31,114 @@
   <name>Apache Beam :: Integration Tests :: Java :: Nexmark</name>
   <packaging>jar</packaging>
 
-  <properties>
-    <flink.version>1.2.0</flink.version>
-    <spark.version>1.6.3</spark.version>
-    <apex.codehaus.jackson.version>1.9.3</apex.codehaus.jackson.version>
-    <skipITs>true</skipITs>
-  </properties>
+  <profiles>
+
+    <!--
+      The direct runner is available by default.
+      You can also include it on the classpath explicitly with -P direct-runner
+    -->
+    <profile>
+      <id>direct-runner</id>
+      <activation>
+        <activeByDefault>true</activeByDefault>
+      </activation>
+      <dependencies>
+        <dependency>
+          <groupId>org.apache.beam</groupId>
+          <artifactId>beam-runners-direct-java</artifactId>
+          <scope>runtime</scope>
+        </dependency>
+      </dependencies>
+    </profile>
+
+    <!-- Include the Apache Apex runner with -P apex-runner -->
+    <profile>
+      <id>apex-runner</id>
+      <dependencies>
+        <dependency>
+          <groupId>org.apache.beam</groupId>
+          <artifactId>beam-runners-apex</artifactId>
+          <scope>runtime</scope>
+        </dependency>
+        <!--
+          Apex depends on httpclient version 4.3.5, project has a transitive dependency to httpclient 4.0.1 from
+          google-http-client. Apex dependency version being specified explicitly so that it gets picked up. This
+          can be removed when the project no longer has a dependency on a different httpclient version.
+        -->
+        <dependency>
+          <groupId>org.apache.httpcomponents</groupId>
+          <artifactId>httpclient</artifactId>
+          <version>4.3.5</version>
+          <scope>runtime</scope>
+          <exclusions>
+            <exclusion>
+              <groupId>commons-codec</groupId>
+              <artifactId>commons-codec</artifactId>
+            </exclusion>
+          </exclusions>
+        </dependency>
+      </dependencies>
+    </profile>
+
+    <!-- Include the Apache Flink runner with -P flink-runner -->
+    <profile>
+      <id>flink-runner</id>
+      <dependencies>
+        <dependency>
+          <groupId>org.apache.beam</groupId>
+          <artifactId>beam-runners-flink_2.10</artifactId>
+          <scope>runtime</scope>
+        </dependency>
+      </dependencies>
+    </profile>
+
+    <!-- Include the Apache Spark runner -P spark-runner -->
+    <profile>
+      <id>spark-runner</id>
+      <dependencies>
+        <dependency>
+          <groupId>org.apache.beam</groupId>
+          <artifactId>beam-runners-spark</artifactId>
+          <scope>runtime</scope>
+        </dependency>
+        <dependency>
+          <groupId>org.apache.spark</groupId>
+          <artifactId>spark-streaming_2.10</artifactId>
+          <version>${spark.version}</version>
+          <scope>runtime</scope>
+        </dependency>
+        <dependency>
+          <groupId>org.apache.spark</groupId>
+          <artifactId>spark-core_2.10</artifactId>
+          <version>${spark.version}</version>
+          <scope>runtime</scope>
+          <exclusions>
+            <exclusion>
+              <groupId>org.slf4j</groupId>
+              <artifactId>jul-to-slf4j</artifactId>
+            </exclusion>
+          </exclusions>
+        </dependency>
+      </dependencies>
+    </profile>
+
+    <!-- Include the Google Cloud Dataflow runner -P dataflow-runner -->
+    <profile>
+      <id>dataflow-runner</id>
+      <dependencies>
+        <dependency>
+          <groupId>org.apache.beam</groupId>
+          <artifactId>beam-runners-google-cloud-dataflow-java</artifactId>
+          <scope>runtime</scope>
+        </dependency>
+      </dependencies>
+    </profile>
+  </profiles>
 
   <build>
     <plugins>
       <plugin>
         <groupId>org.apache.maven.plugins</groupId>
-        <artifactId>maven-compiler-plugin</artifactId>
-      </plugin>
-
-      <plugin>
-        <groupId>org.apache.maven.plugins</groupId>
-        <artifactId>maven-surefire-plugin</artifactId>
-        <configuration>
-          <systemPropertyVariables>
-            <beamTestPipelineOptions>
-            </beamTestPipelineOptions>
-          </systemPropertyVariables>
-        </configuration>
-      </plugin>
-
-      <plugin>
-        <groupId>org.apache.maven.plugins</groupId>
-        <artifactId>maven-checkstyle-plugin</artifactId>
-      </plugin>
-
-      <!-- Source plugin for generating source and test-source JARs. -->
-      <plugin>
-        <groupId>org.apache.maven.plugins</groupId>
-        <artifactId>maven-source-plugin</artifactId>
-      </plugin>
-
-      <plugin>
-        <groupId>org.apache.maven.plugins</groupId>
         <artifactId>maven-shade-plugin</artifactId>
         <executions>
           <execution>
@@ -98,11 +168,6 @@
         </executions>
       </plugin>
 
-      <plugin>
-        <groupId>org.apache.maven.plugins</groupId>
-        <artifactId>maven-jar-plugin</artifactId>
-      </plugin>
-
       <!-- Avro plugin for automatic code generation -->
       <plugin>
         <groupId>org.apache.avro</groupId>
@@ -127,22 +192,6 @@
         <groupId>org.jacoco</groupId>
         <artifactId>jacoco-maven-plugin</artifactId>
       </plugin>
-
-      <plugin>
-        <groupId>org.apache.maven.plugins</groupId>
-        <artifactId>maven-dependency-plugin</artifactId>
-        <executions>
-          <execution>
-            <goals><goal>analyze-only</goal></goals>
-            <configuration>
-              <!-- Ignore runtime-only dependencies in analysis -->
-              <ignoreNonCompile>true</ignoreNonCompile>
-              <failOnWarning>false</failOnWarning>
-            </configuration>
-          </execution>
-        </executions>
-      </plugin>
-
     </plugins>
   </build>
 
@@ -153,73 +202,6 @@
       <artifactId>beam-sdks-java-core</artifactId>
     </dependency>
 
-    <!-- Java runner for Google Cloud Dataflow -->
-    <dependency>
-      <groupId>org.apache.beam</groupId>
-      <artifactId>beam-runners-google-cloud-dataflow-java</artifactId>
-    </dependency>
-
-    <!-- Direct runner -->
-    <dependency>
-      <groupId>org.apache.beam</groupId>
-      <artifactId>beam-runners-direct-java</artifactId>
-    </dependency>
-
-    <!-- Flink runner -->
-    <dependency>
-      <groupId>org.apache.beam</groupId>
-      <artifactId>beam-runners-flink_2.10</artifactId>
-    </dependency>
-
-    <dependency>
-      <groupId>org.apache.flink</groupId>
-      <artifactId>flink-shaded-hadoop2</artifactId>
-      <version>${flink.version}</version>
-      <scope>provided</scope>
-    </dependency>
-
-    <!-- Spark runner -->
-    <dependency>
-      <groupId>org.apache.beam</groupId>
-      <artifactId>beam-runners-spark</artifactId>
-    </dependency>
-    <dependency>
-      <groupId>org.apache.spark</groupId>
-      <artifactId>spark-core_2.10</artifactId>
-      <version>${spark.version}</version>
-      <scope>runtime</scope>
-    </dependency>
-    <dependency>
-      <groupId>org.apache.spark</groupId>
-      <artifactId>spark-streaming_2.10</artifactId>
-      <version>${spark.version}</version>
-      <scope>runtime</scope>
-    </dependency>
-
-    <!-- Apex runner -->
-    <dependency>
-      <groupId>org.apache.beam</groupId>
-      <artifactId>beam-runners-apex</artifactId>
-    </dependency>
-    <dependency>
-      <groupId>com.esotericsoftware.kryo</groupId>
-      <artifactId>kryo</artifactId>
-      <version>${apex.kryo.version}</version>
-      <scope>runtime</scope>
-    </dependency>
-    <dependency>
-      <groupId>org.codehaus.jackson</groupId>
-      <artifactId>jackson-mapper-asl</artifactId>
-      <version>${apex.codehaus.jackson.version}</version>
-      <scope>runtime</scope>
-    </dependency>
-    <dependency>
-      <groupId>org.codehaus.jackson</groupId>
-      <artifactId>jackson-core-asl</artifactId>
-      <version>${apex.codehaus.jackson.version}</version>
-      <scope>runtime</scope>
-    </dependency>
-
     <!-- IOs -->
     <dependency>
       <groupId>org.apache.beam</groupId>
@@ -231,57 +213,20 @@
       <artifactId>beam-sdks-java-extensions-gcp-core</artifactId>
     </dependency>
 
-    <!-- Extra libraries -->
-    <dependency>
-      <groupId>com.google.apis</groupId>
-      <artifactId>google-api-services-dataflow</artifactId>
-      <version>${dataflow.version}</version>
-      <scope>runtime</scope>
-    </dependency>
-
-    <dependency>
-      <groupId>joda-time</groupId>
-      <artifactId>joda-time</artifactId>
-    </dependency>
-
-    <dependency>
-      <groupId>com.fasterxml.jackson.core</groupId>
-      <artifactId>jackson-core</artifactId>
-    </dependency>
-
     <dependency>
       <groupId>com.google.apis</groupId>
       <artifactId>google-api-services-bigquery</artifactId>
     </dependency>
 
     <dependency>
-      <groupId>com.google.apis</groupId>
-      <artifactId>google-api-services-pubsub</artifactId>
-    </dependency>
-
-    <dependency>
-      <groupId>com.google.auth</groupId>
-      <artifactId>google-auth-library-credentials</artifactId>
-    </dependency>
-
-    <dependency>
-      <groupId>com.google.auth</groupId>
-      <artifactId>google-auth-library-oauth2-http</artifactId>
-    </dependency>
-
-    <dependency>
       <groupId>com.google.cloud.bigdataoss</groupId>
       <artifactId>gcsio</artifactId>
     </dependency>
 
+    <!-- Extra libraries -->
     <dependency>
-      <groupId>com.google.cloud.bigdataoss</groupId>
-      <artifactId>util</artifactId>
-    </dependency>
-
-    <dependency>
-      <groupId>com.google.http-client</groupId>
-      <artifactId>google-http-client</artifactId>
+      <groupId>com.fasterxml.jackson.core</groupId>
+      <artifactId>jackson-core</artifactId>
     </dependency>
 
     <dependency>
@@ -300,19 +245,18 @@
     </dependency>
 
     <dependency>
-      <groupId>com.google.guava</groupId>
-      <artifactId>guava</artifactId>
+      <groupId>joda-time</groupId>
+      <artifactId>joda-time</artifactId>
     </dependency>
 
     <dependency>
-      <groupId>org.hamcrest</groupId>
-      <artifactId>hamcrest-all</artifactId>
+      <groupId>com.google.guava</groupId>
+      <artifactId>guava</artifactId>
     </dependency>
 
     <dependency>
       <groupId>org.slf4j</groupId>
       <artifactId>slf4j-api</artifactId>
-      <scope>compile</scope>
     </dependency>
 
     <dependency>
@@ -325,5 +269,23 @@
       <artifactId>junit</artifactId>
       <scope>compile</scope>
     </dependency>
+
+    <dependency>
+      <groupId>org.hamcrest</groupId>
+      <artifactId>hamcrest-core</artifactId>
+      <scope>compile</scope>
+    </dependency>
+
+    <dependency>
+      <groupId>org.hamcrest</groupId>
+      <artifactId>hamcrest-all</artifactId>
+    </dependency>
+
+    <!-- Test -->
+    <dependency>
+      <groupId>org.apache.beam</groupId>
+      <artifactId>beam-runners-direct-java</artifactId>
+      <scope>test</scope>
+    </dependency>
   </dependencies>
 </project>

http://git-wip-us.apache.org/repos/asf/beam/blob/a6dbdfa5/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkDriver.java
----------------------------------------------------------------------
diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkDriver.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkDriver.java
index 4714124..7d532cc 100644
--- a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkDriver.java
+++ b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkDriver.java
@@ -31,6 +31,7 @@ import javax.annotation.Nullable;
 import org.apache.beam.integration.nexmark.model.Auction;
 import org.apache.beam.integration.nexmark.model.Bid;
 import org.apache.beam.integration.nexmark.model.Person;
+import org.apache.beam.sdk.options.PipelineOptionsFactory;
 import org.joda.time.Duration;
 import org.joda.time.Instant;
 
@@ -294,4 +295,12 @@ public class NexmarkDriver<OptionT extends NexmarkOptions> {
     }
     NexmarkUtils.console("saved javascript to file %s.", javascriptFilename);
   }
+
+  public static void main(String[] args) {
+    NexmarkOptions options = PipelineOptionsFactory.fromArgs(args)
+      .withValidation()
+      .as(NexmarkOptions.class);
+    NexmarkRunner runner = new NexmarkRunner(options);
+    new NexmarkDriver().runAll(options, runner);
+  }
 }

http://git-wip-us.apache.org/repos/asf/beam/blob/a6dbdfa5/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkRunner.java
----------------------------------------------------------------------
diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkRunner.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkRunner.java
index 87314ce..ebfd196 100644
--- a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkRunner.java
+++ b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkRunner.java
@@ -33,7 +33,6 @@ import java.util.List;
 import java.util.NoSuchElementException;
 import java.util.concurrent.ThreadLocalRandom;
 import javax.annotation.Nullable;
-import org.apache.beam.integration.nexmark.io.PubsubHelper;
 import org.apache.beam.integration.nexmark.model.Auction;
 import org.apache.beam.integration.nexmark.model.Bid;
 import org.apache.beam.integration.nexmark.model.Event;
@@ -86,7 +85,7 @@ import org.joda.time.Duration;
 /**
  * Run a single Nexmark query using a given configuration.
  */
-public abstract class NexmarkRunner<OptionT extends NexmarkOptions> {
+public class NexmarkRunner<OptionT extends NexmarkOptions> {
   /**
    * Minimum number of samples needed for 'stead-state' rate calculation.
    */
@@ -125,12 +124,6 @@ public abstract class NexmarkRunner<OptionT extends NexmarkOptions> {
   protected NexmarkConfiguration configuration;
 
   /**
-   * Accumulate the pub/sub subscriptions etc which should be cleaned up on end of run.
-   */
-  @Nullable
-  protected PubsubHelper pubsub;
-
-  /**
    * If in --pubsubMode=COMBINED, the event monitor for the publisher pipeline. Otherwise null.
    */
   @Nullable
@@ -158,16 +151,6 @@ public abstract class NexmarkRunner<OptionT extends NexmarkOptions> {
     this.options = options;
   }
 
-  /**
-   * Return a Pubsub helper.
-   */
-  private PubsubHelper getPubsub() {
-    if (pubsub == null) {
-      pubsub = PubsubHelper.create(options);
-    }
-    return pubsub;
-  }
-
   // ================================================================================
   // Overridden by each runner.
   // ================================================================================
@@ -175,17 +158,23 @@ public abstract class NexmarkRunner<OptionT extends NexmarkOptions> {
   /**
    * Is this query running in streaming mode?
    */
-  protected abstract boolean isStreaming();
+  protected boolean isStreaming() {
+    return options.isStreaming();
+  }
 
   /**
    * Return number of cores per worker.
    */
-  protected abstract int coresPerWorker();
+  protected int coresPerWorker() {
+    return 4;
+  }
 
   /**
    * Return maximum number of workers.
    */
-  protected abstract int maxNumWorkers();
+  protected int maxNumWorkers() {
+    return 5;
+  }
 
   /**
    * Return the current value for a long counter, or a default value if can't be retrieved.
@@ -544,13 +533,20 @@ public abstract class NexmarkRunner<OptionT extends NexmarkOptions> {
   /**
    * Invoke the builder with options suitable for running a publish-only child pipeline.
    */
-  protected abstract void invokeBuilderForPublishOnlyPipeline(PipelineBuilder builder);
+  protected void invokeBuilderForPublishOnlyPipeline(
+      PipelineBuilder builder) {
+    builder.build(options);
+//    throw new UnsupportedOperationException(
+//        "Cannot use --pubSubMode=COMBINED with DirectRunner");
+  }
 
   /**
    * If monitoring, wait until the publisher pipeline has run long enough to establish
    * a backlog on the Pubsub topic. Otherwise, return immediately.
    */
-  protected abstract void waitForPublisherPreload();
+  protected void waitForPublisherPreload() {
+    throw new UnsupportedOperationException();
+  }
 
   /**
    * Monitor the performance and progress of a running job. Return final performance if
@@ -841,24 +837,14 @@ public abstract class NexmarkRunner<OptionT extends NexmarkOptions> {
    * Return source of events from Pubsub.
    */
   private PCollection<Event> sourceEventsFromPubsub(Pipeline p, long now) {
-    String shortTopic = shortTopic(now);
     String shortSubscription = shortSubscription(now);
-
-    // Create/confirm the subscription.
-    String subscription = null;
-    if (!options.getManageResources()) {
-      // The subscription should already have been created by the user.
-      subscription = getPubsub().reuseSubscription(shortTopic, shortSubscription).getPath();
-    } else {
-      subscription = getPubsub().createSubscription(shortTopic, shortSubscription).getPath();
-    }
-    NexmarkUtils.console("Reading events from Pubsub %s", subscription);
+    NexmarkUtils.console("Reading events from Pubsub %s", shortSubscription);
     PubsubIO.Read<Event> io =
-        PubsubIO.<Event>read().subscription(subscription)
-            .idLabel(NexmarkUtils.PUBSUB_ID)
+        PubsubIO.<Event>read().fromSubscription(shortSubscription)
+            .withIdAttribute(NexmarkUtils.PUBSUB_ID)
             .withCoder(Event.CODER);
     if (!configuration.usePubsubPublishTime) {
-      io = io.timestampLabel(NexmarkUtils.PUBSUB_TIMESTAMP);
+      io = io.withTimestampAttribute(NexmarkUtils.PUBSUB_TIMESTAMP);
     }
     return p.apply(queryName + ".ReadPubsubEvents", io);
   }
@@ -884,26 +870,13 @@ public abstract class NexmarkRunner<OptionT extends NexmarkOptions> {
    */
   private void sinkEventsToPubsub(PCollection<Event> events, long now) {
     String shortTopic = shortTopic(now);
-
-    // Create/confirm the topic.
-    String topic;
-    if (!options.getManageResources()
-        || configuration.pubSubMode == NexmarkUtils.PubSubMode.SUBSCRIBE_ONLY) {
-      // The topic should already have been created by the user or
-      // a companion 'PUBLISH_ONLY' process.
-      topic = getPubsub().reuseTopic(shortTopic).getPath();
-    } else {
-      // Create a fresh topic to loopback via. It will be destroyed when the
-      // (necessarily blocking) job is done.
-      topic = getPubsub().createTopic(shortTopic).getPath();
-    }
-    NexmarkUtils.console("Writing events to Pubsub %s", topic);
+    NexmarkUtils.console("Writing events to Pubsub %s", shortTopic);
     PubsubIO.Write<Event> io =
-        PubsubIO.<Event>write().topic(topic)
-                      .idLabel(NexmarkUtils.PUBSUB_ID)
+        PubsubIO.<Event>write().to(shortTopic)
+                      .withIdAttribute(NexmarkUtils.PUBSUB_ID)
                       .withCoder(Event.CODER);
     if (!configuration.usePubsubPublishTime) {
-      io = io.timestampLabel(NexmarkUtils.PUBSUB_TIMESTAMP);
+      io = io.withTimestampAttribute(NexmarkUtils.PUBSUB_TIMESTAMP);
     }
     events.apply(queryName + ".WritePubsubEvents", io);
   }
@@ -913,18 +886,12 @@ public abstract class NexmarkRunner<OptionT extends NexmarkOptions> {
    */
   private void sinkResultsToPubsub(PCollection<String> formattedResults, long now) {
     String shortTopic = shortTopic(now);
-    String topic;
-    if (!options.getManageResources()) {
-      topic = getPubsub().reuseTopic(shortTopic).getPath();
-    } else {
-      topic = getPubsub().createTopic(shortTopic).getPath();
-    }
-    NexmarkUtils.console("Writing results to Pubsub %s", topic);
+    NexmarkUtils.console("Writing results to Pubsub %s", shortTopic);
     PubsubIO.Write<String> io =
-        PubsubIO.<String>write().topic(topic)
-            .idLabel(NexmarkUtils.PUBSUB_ID);
+        PubsubIO.<String>write().to(shortTopic)
+            .withIdAttribute(NexmarkUtils.PUBSUB_ID);
     if (!configuration.usePubsubPublishTime) {
-      io = io.timestampLabel(NexmarkUtils.PUBSUB_TIMESTAMP);
+      io = io.withTimestampAttribute(NexmarkUtils.PUBSUB_TIMESTAMP);
     }
     formattedResults.apply(queryName + ".WritePubsubResults", io);
   }
@@ -1168,7 +1135,6 @@ public abstract class NexmarkRunner<OptionT extends NexmarkOptions> {
     // Setup per-run state.
     //
     checkState(configuration == null);
-    checkState(pubsub == null);
     checkState(queryName == null);
     configuration = runConfiguration;
 
@@ -1282,19 +1248,9 @@ public abstract class NexmarkRunner<OptionT extends NexmarkOptions> {
       mainResult.waitUntilFinish(Duration.standardSeconds(configuration.streamTimeout));
       return monitor(query);
     } finally {
-      //
-      // Cleanup per-run state.
-      //
-      if (pubsub != null) {
-        // Delete any subscriptions and topics we created.
-        pubsub.close();
-        pubsub = null;
-      }
       configuration = null;
       queryName = null;
       // TODO: Cleanup pathsToDelete
     }
-
   }
-
 }

http://git-wip-us.apache.org/repos/asf/beam/blob/a6dbdfa5/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/drivers/NexmarkApexDriver.java
----------------------------------------------------------------------
diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/drivers/NexmarkApexDriver.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/drivers/NexmarkApexDriver.java
deleted file mode 100644
index 265ccf7..0000000
--- a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/drivers/NexmarkApexDriver.java
+++ /dev/null
@@ -1,50 +0,0 @@
-/*
- * 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.beam.integration.nexmark.drivers;
-
-import org.apache.beam.integration.nexmark.NexmarkDriver;
-import org.apache.beam.integration.nexmark.NexmarkOptions;
-import org.apache.beam.runners.apex.ApexPipelineOptions;
-import org.apache.beam.runners.apex.ApexRunner;
-import org.apache.beam.sdk.options.PipelineOptionsFactory;
-
-/**
- * Run NexMark queries using the Apex runner.
- */
-public class NexmarkApexDriver extends NexmarkDriver<NexmarkApexDriver.NexmarkApexOptions> {
-  /**
-   * Command line flags.
-   */
-  public interface NexmarkApexOptions extends NexmarkOptions, ApexPipelineOptions {
-  }
-
-  /**
-   * Entry point.
-   */
-  public static void main(String[] args) {
-    // Gather command line args, baseline, configurations, etc.
-    NexmarkApexOptions options = PipelineOptionsFactory.fromArgs(args)
-                                                        .withValidation()
-                                                        .as(NexmarkApexOptions.class);
-    options.setRunner(ApexRunner.class);
-    NexmarkApexRunner runner = new NexmarkApexRunner(options);
-    new NexmarkApexDriver().runAll(options, runner);
-  }
-}
-
-

http://git-wip-us.apache.org/repos/asf/beam/blob/a6dbdfa5/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/drivers/NexmarkApexRunner.java
----------------------------------------------------------------------
diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/drivers/NexmarkApexRunner.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/drivers/NexmarkApexRunner.java
deleted file mode 100644
index 2bcf82d..0000000
--- a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/drivers/NexmarkApexRunner.java
+++ /dev/null
@@ -1,65 +0,0 @@
-/*
- * 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.beam.integration.nexmark.drivers;
-
-import javax.annotation.Nullable;
-
-import org.apache.beam.integration.nexmark.NexmarkPerf;
-import org.apache.beam.integration.nexmark.NexmarkQuery;
-import org.apache.beam.integration.nexmark.NexmarkRunner;
-
-/**
- * Run a query using the Apex runner.
- */
-public class NexmarkApexRunner extends NexmarkRunner<NexmarkApexDriver.NexmarkApexOptions> {
-  @Override
-  protected boolean isStreaming() {
-    return options.isStreaming();
-  }
-
-  @Override
-  protected int coresPerWorker() {
-    return 4;
-  }
-
-  @Override
-  protected int maxNumWorkers() {
-    return 5;
-  }
-
-  @Override
-  protected void invokeBuilderForPublishOnlyPipeline(
-      PipelineBuilder builder) {
-    builder.build(options);
-  }
-
-  @Override
-  protected void waitForPublisherPreload() {
-    throw new UnsupportedOperationException();
-  }
-
-  @Override
-  @Nullable
-  protected NexmarkPerf monitor(NexmarkQuery query) {
-    return null;
-  }
-
-  public NexmarkApexRunner(NexmarkApexDriver.NexmarkApexOptions options) {
-    super(options);
-  }
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/a6dbdfa5/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/drivers/NexmarkDirectDriver.java
----------------------------------------------------------------------
diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/drivers/NexmarkDirectDriver.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/drivers/NexmarkDirectDriver.java
deleted file mode 100644
index 2b825f3..0000000
--- a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/drivers/NexmarkDirectDriver.java
+++ /dev/null
@@ -1,49 +0,0 @@
-/*
- * 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.beam.integration.nexmark.drivers;
-
-import org.apache.beam.integration.nexmark.NexmarkDriver;
-import org.apache.beam.integration.nexmark.NexmarkOptions;
-import org.apache.beam.runners.direct.DirectOptions;
-import org.apache.beam.runners.direct.DirectRunner;
-import org.apache.beam.sdk.options.PipelineOptionsFactory;
-
-/**
- * An implementation of the 'NEXMark queries' using the Direct Runner.
- */
-class NexmarkDirectDriver extends NexmarkDriver<NexmarkDirectDriver.NexmarkDirectOptions> {
-  /**
-   * Command line flags.
-   */
-  public interface NexmarkDirectOptions extends NexmarkOptions, DirectOptions {
-  }
-
-  /**
-   * Entry point.
-   */
-  public static void main(String[] args) {
-    NexmarkDirectOptions options =
-        PipelineOptionsFactory.fromArgs(args)
-                              .withValidation()
-                              .as(NexmarkDirectOptions.class);
-    options.setRunner(DirectRunner.class);
-    NexmarkDirectRunner runner = new NexmarkDirectRunner(options);
-    new NexmarkDirectDriver().runAll(options, runner);
-  }
-}
-

http://git-wip-us.apache.org/repos/asf/beam/blob/a6dbdfa5/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/drivers/NexmarkDirectRunner.java
----------------------------------------------------------------------
diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/drivers/NexmarkDirectRunner.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/drivers/NexmarkDirectRunner.java
deleted file mode 100644
index 1391040..0000000
--- a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/drivers/NexmarkDirectRunner.java
+++ /dev/null
@@ -1,60 +0,0 @@
-/*
- * 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.beam.integration.nexmark.drivers;
-
-import org.apache.beam.integration.nexmark.NexmarkRunner;
-
-/**
- * Run a single query using the Direct Runner.
- */
-class NexmarkDirectRunner extends NexmarkRunner<NexmarkDirectDriver.NexmarkDirectOptions> {
-  public NexmarkDirectRunner(NexmarkDirectDriver.NexmarkDirectOptions options) {
-    super(options);
-  }
-
-  @Override
-  protected boolean isStreaming() {
-    return options.isStreaming();
-  }
-
-  @Override
-  protected int coresPerWorker() {
-    return 4;
-  }
-
-  @Override
-  protected int maxNumWorkers() {
-    return 1;
-  }
-
-  @Override
-  protected void invokeBuilderForPublishOnlyPipeline(PipelineBuilder builder) {
-    throw new UnsupportedOperationException(
-        "Cannot use --pubSubMode=COMBINED with DirectRunner");
-  }
-
-  /**
-   * Monitor the progress of the publisher job. Return when it has been generating events for
-   * at least {@code configuration.preloadSeconds}.
-   */
-  @Override
-  protected void waitForPublisherPreload() {
-    throw new UnsupportedOperationException(
-        "Cannot use --pubSubMode=COMBINED with DirectRunner");
-  }
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/a6dbdfa5/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/drivers/NexmarkFlinkDriver.java
----------------------------------------------------------------------
diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/drivers/NexmarkFlinkDriver.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/drivers/NexmarkFlinkDriver.java
deleted file mode 100644
index bf0b115..0000000
--- a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/drivers/NexmarkFlinkDriver.java
+++ /dev/null
@@ -1,50 +0,0 @@
-/*
- * 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.beam.integration.nexmark.drivers;
-
-import org.apache.beam.integration.nexmark.NexmarkDriver;
-import org.apache.beam.integration.nexmark.NexmarkOptions;
-import org.apache.beam.runners.flink.FlinkPipelineOptions;
-import org.apache.beam.runners.flink.FlinkRunner;
-import org.apache.beam.sdk.options.PipelineOptionsFactory;
-
-/**
- * Run NexMark queries using the Flink runner.
- */
-public class NexmarkFlinkDriver extends NexmarkDriver<NexmarkFlinkDriver.NexmarkFlinkOptions> {
-  /**
-   * Command line flags.
-   */
-  public interface NexmarkFlinkOptions extends NexmarkOptions, FlinkPipelineOptions {
-  }
-
-  /**
-   * Entry point.
-   */
-  public static void main(String[] args) {
-    // Gather command line args, baseline, configurations, etc.
-    NexmarkFlinkOptions options = PipelineOptionsFactory.fromArgs(args)
-                                                        .withValidation()
-                                                        .as(NexmarkFlinkOptions.class);
-    options.setRunner(FlinkRunner.class);
-    NexmarkFlinkRunner runner = new NexmarkFlinkRunner(options);
-    new NexmarkFlinkDriver().runAll(options, runner);
-  }
-}
-
-

http://git-wip-us.apache.org/repos/asf/beam/blob/a6dbdfa5/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/drivers/NexmarkFlinkRunner.java
----------------------------------------------------------------------
diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/drivers/NexmarkFlinkRunner.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/drivers/NexmarkFlinkRunner.java
deleted file mode 100644
index 9d547ef..0000000
--- a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/drivers/NexmarkFlinkRunner.java
+++ /dev/null
@@ -1,55 +0,0 @@
-/*
- * 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.beam.integration.nexmark.drivers;
-
-import org.apache.beam.integration.nexmark.NexmarkRunner;
-
-/**
- * Run a query using the Flink runner.
- */
-public class NexmarkFlinkRunner extends NexmarkRunner<NexmarkFlinkDriver.NexmarkFlinkOptions> {
-  @Override
-  protected boolean isStreaming() {
-    return options.isStreaming();
-  }
-
-  @Override
-  protected int coresPerWorker() {
-    return 4;
-  }
-
-  @Override
-  protected int maxNumWorkers() {
-    return 5;
-  }
-
-  @Override
-  protected void invokeBuilderForPublishOnlyPipeline(
-      PipelineBuilder builder) {
-    builder.build(options);
-  }
-
-  @Override
-  protected void waitForPublisherPreload() {
-    throw new UnsupportedOperationException();
-  }
-
-  public NexmarkFlinkRunner(NexmarkFlinkDriver.NexmarkFlinkOptions options) {
-    super(options);
-  }
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/a6dbdfa5/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/drivers/NexmarkGoogleDriver.java
----------------------------------------------------------------------
diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/drivers/NexmarkGoogleDriver.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/drivers/NexmarkGoogleDriver.java
deleted file mode 100644
index f5a9751..0000000
--- a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/drivers/NexmarkGoogleDriver.java
+++ /dev/null
@@ -1,67 +0,0 @@
-/*
- * 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.beam.integration.nexmark.drivers;
-
-import org.apache.beam.integration.nexmark.NexmarkDriver;
-import org.apache.beam.integration.nexmark.NexmarkOptions;
-import org.apache.beam.integration.nexmark.model.Auction;
-import org.apache.beam.integration.nexmark.model.Bid;
-import org.apache.beam.integration.nexmark.model.Person;
-import org.apache.beam.runners.dataflow.DataflowRunner;
-import org.apache.beam.runners.dataflow.options.DataflowPipelineOptions;
-import org.apache.beam.sdk.options.PipelineOptionsFactory;
-
-/**
- * An implementation of the 'NEXMark queries' for Google Dataflow.
- * These are multiple queries over a three table schema representing an online auction system:
- * <ul>
- * <li>{@link Person} represents a person submitting an item for auction and/or making a bid
- * on an auction.
- * <li>{@link Auction} represents an item under auction.
- * <li>{@link Bid} represents a bid for an item under auction.
- * </ul>
- * The queries exercise many aspects of streaming dataflow.
- *
- * <p>We synthesize the creation of people, auctions and bids in real-time. The data is not
- * particularly sensible.
- *
- * <p>See <a href="http://datalab.cs.pdx.edu/niagaraST/NEXMark/">
- * http://datalab.cs.pdx.edu/niagaraST/NEXMark/</a>
- */
-class NexmarkGoogleDriver extends NexmarkDriver<NexmarkGoogleDriver.NexmarkGoogleOptions> {
-  /**
-   * Command line flags.
-   */
-  public interface NexmarkGoogleOptions extends NexmarkOptions, DataflowPipelineOptions {
-
-  }
-
-  /**
-   * Entry point.
-   */
-  public static void main(String[] args) {
-    // Gather command line args, baseline, configurations, etc.
-    NexmarkGoogleOptions options = PipelineOptionsFactory.fromArgs(args)
-                                                         .withValidation()
-                                                         .as(NexmarkGoogleOptions.class);
-    options.setRunner(DataflowRunner.class);
-    NexmarkGoogleRunner runner = new NexmarkGoogleRunner(options);
-    new NexmarkGoogleDriver().runAll(options, runner);
-  }
-}
-

http://git-wip-us.apache.org/repos/asf/beam/blob/a6dbdfa5/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/drivers/NexmarkGoogleRunner.java
----------------------------------------------------------------------
diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/drivers/NexmarkGoogleRunner.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/drivers/NexmarkGoogleRunner.java
deleted file mode 100644
index 935bf0d..0000000
--- a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/drivers/NexmarkGoogleRunner.java
+++ /dev/null
@@ -1,165 +0,0 @@
-/*
- * 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.beam.integration.nexmark.drivers;
-
-import static com.google.common.base.Preconditions.checkNotNull;
-
-import org.apache.beam.integration.nexmark.Monitor;
-import org.apache.beam.integration.nexmark.NexmarkRunner;
-import org.apache.beam.integration.nexmark.NexmarkUtils;
-import org.apache.beam.integration.nexmark.model.Event;
-import org.apache.beam.runners.dataflow.DataflowPipelineJob;
-import org.apache.beam.sdk.PipelineResult;
-import org.joda.time.Duration;
-
-/**
- * Run a singe Nexmark query using a given configuration on Google Dataflow.
- */
-class NexmarkGoogleRunner extends NexmarkRunner<NexmarkGoogleDriver.NexmarkGoogleOptions> {
-
-  public NexmarkGoogleRunner(NexmarkGoogleDriver.NexmarkGoogleOptions options) {
-    super(options);
-  }
-
-  @Override
-  protected boolean isStreaming() {
-    return options.isStreaming();
-  }
-
-  @Override
-  protected int coresPerWorker() {
-    String machineType = options.getWorkerMachineType();
-    if (machineType == null || machineType.isEmpty()) {
-      return 1;
-    }
-    String[] split = machineType.split("-");
-    if (split.length != 3) {
-      return 1;
-    }
-    try {
-      return Integer.parseInt(split[2]);
-    } catch (NumberFormatException ex) {
-      return 1;
-    }
-  }
-
-  @Override
-  protected int maxNumWorkers() {
-    return Math.max(options.getNumWorkers(), options.getMaxNumWorkers());
-  }
-
-  @Override
-  protected String getJobId(PipelineResult job) {
-    return ((DataflowPipelineJob) job).getJobId();
-  }
-
-  @Override
-  protected void invokeBuilderForPublishOnlyPipeline(PipelineBuilder builder) {
-    String jobName = options.getJobName();
-    String appName = options.getAppName();
-    options.setJobName("p-" + jobName);
-    options.setAppName("p-" + appName);
-    int coresPerWorker = coresPerWorker();
-    int eventGeneratorWorkers = (configuration.numEventGenerators + coresPerWorker - 1)
-                                / coresPerWorker;
-    options.setMaxNumWorkers(Math.min(options.getMaxNumWorkers(), eventGeneratorWorkers));
-    options.setNumWorkers(Math.min(options.getNumWorkers(), eventGeneratorWorkers));
-    publisherMonitor = new Monitor<Event>(queryName, "publisher");
-    try {
-      builder.build(options);
-    } finally {
-      options.setJobName(jobName);
-      options.setAppName(appName);
-      options.setMaxNumWorkers(options.getMaxNumWorkers());
-      options.setNumWorkers(options.getNumWorkers());
-    }
-  }
-
-  /**
-   * Monitor the progress of the publisher job. Return when it has been generating events for
-   * at least {@code configuration.preloadSeconds}.
-   */
-  @Override
-  protected void waitForPublisherPreload() {
-    checkNotNull(publisherMonitor);
-    checkNotNull(publisherResult);
-    if (!options.getMonitorJobs()) {
-      return;
-    }
-    if (!(publisherResult instanceof DataflowPipelineJob)) {
-      return;
-    }
-    if (configuration.preloadSeconds <= 0) {
-      return;
-    }
-
-    NexmarkUtils.console("waiting for publisher to pre-load");
-
-    DataflowPipelineJob job = (DataflowPipelineJob) publisherResult;
-
-    long numEvents = 0;
-    long startMsSinceEpoch = -1;
-    long endMsSinceEpoch = -1;
-    while (true) {
-      PipelineResult.State state = job.getState();
-      switch (state) {
-        case UNKNOWN:
-          // Keep waiting.
-          NexmarkUtils.console("%s publisher (%d events)", state, numEvents);
-          break;
-        case STOPPED:
-        case DONE:
-        case CANCELLED:
-        case FAILED:
-        case UPDATED:
-          NexmarkUtils.console("%s publisher (%d events)", state, numEvents);
-          return;
-        case RUNNING:
-          //TODO Ismael Validate that this counter is ok
-          numEvents =
-            getCounterMetric(job, publisherMonitor.name, publisherMonitor.prefix + ".elements", -1);
-          if (startMsSinceEpoch < 0 && numEvents > 0) {
-            startMsSinceEpoch = System.currentTimeMillis();
-            endMsSinceEpoch = startMsSinceEpoch
-                              + Duration.standardSeconds(configuration.preloadSeconds).getMillis();
-          }
-          if (endMsSinceEpoch < 0) {
-            NexmarkUtils.console("%s publisher (%d events)", state, numEvents);
-          } else {
-            long remainMs = endMsSinceEpoch - System.currentTimeMillis();
-            if (remainMs > 0) {
-              NexmarkUtils.console("%s publisher (%d events, waiting for %ds)", state, numEvents,
-                  remainMs / 1000);
-            } else {
-              NexmarkUtils.console("publisher preloaded %d events", numEvents);
-              return;
-            }
-          }
-          break;
-      }
-
-      try {
-        Thread.sleep(PERF_DELAY.getMillis());
-      } catch (InterruptedException e) {
-        Thread.interrupted();
-        throw new RuntimeException("Interrupted: publisher still running.");
-      }
-    }
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/a6dbdfa5/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/drivers/NexmarkSparkDriver.java
----------------------------------------------------------------------
diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/drivers/NexmarkSparkDriver.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/drivers/NexmarkSparkDriver.java
deleted file mode 100644
index c7c32c2..0000000
--- a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/drivers/NexmarkSparkDriver.java
+++ /dev/null
@@ -1,48 +0,0 @@
-/*
- * 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.beam.integration.nexmark.drivers;
-
-import org.apache.beam.integration.nexmark.NexmarkDriver;
-import org.apache.beam.integration.nexmark.NexmarkOptions;
-import org.apache.beam.runners.spark.SparkPipelineOptions;
-import org.apache.beam.runners.spark.SparkRunner;
-import org.apache.beam.sdk.options.PipelineOptionsFactory;
-
-/**
- * Run NexMark queries using the Spark runner.
- */
-class NexmarkSparkDriver extends NexmarkDriver<NexmarkSparkDriver.NexmarkSparkOptions> {
-    /**
-     * Command line flags.
-     */
-    public interface NexmarkSparkOptions extends NexmarkOptions, SparkPipelineOptions {
-    }
-
-    /**
-     * Entry point.
-     */
-    public static void main(String[] args) {
-        NexmarkSparkOptions options =
-                PipelineOptionsFactory.fromArgs(args)
-                        .withValidation()
-                        .as(NexmarkSparkOptions.class);
-        options.setRunner(SparkRunner.class);
-        NexmarkSparkRunner runner = new NexmarkSparkRunner(options);
-        new NexmarkSparkDriver().runAll(options, runner);
-    }
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/a6dbdfa5/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/drivers/NexmarkSparkRunner.java
----------------------------------------------------------------------
diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/drivers/NexmarkSparkRunner.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/drivers/NexmarkSparkRunner.java
deleted file mode 100644
index 1d49a3a..0000000
--- a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/drivers/NexmarkSparkRunner.java
+++ /dev/null
@@ -1,56 +0,0 @@
-/*
- * 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.beam.integration.nexmark.drivers;
-
-import org.apache.beam.integration.nexmark.NexmarkRunner;
-
-/**
- * Run a query using the Spark runner.
- */
-public class NexmarkSparkRunner extends NexmarkRunner<NexmarkSparkDriver.NexmarkSparkOptions> {
-    @Override
-    protected boolean isStreaming() {
-        return options.isStreaming();
-    }
-
-    @Override
-    protected int coresPerWorker() {
-        return 4;
-    }
-
-    @Override
-    protected int maxNumWorkers() {
-        return 5;
-    }
-
-    @Override
-    protected void invokeBuilderForPublishOnlyPipeline(
-            PipelineBuilder builder) {
-        builder.build(options);
-    }
-
-    @Override
-    protected void waitForPublisherPreload() {
-        throw new UnsupportedOperationException();
-    }
-
-
-    public NexmarkSparkRunner(NexmarkSparkDriver.NexmarkSparkOptions options) {
-        super(options);
-    }
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/a6dbdfa5/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/drivers/package-info.java
----------------------------------------------------------------------
diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/drivers/package-info.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/drivers/package-info.java
deleted file mode 100644
index c8aa144..0000000
--- a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/drivers/package-info.java
+++ /dev/null
@@ -1,22 +0,0 @@
-/*
- * 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.
- */
-
-/**
- * Nexmark Benchmark Execution Drivers.
- */
-package org.apache.beam.integration.nexmark.drivers;

http://git-wip-us.apache.org/repos/asf/beam/blob/a6dbdfa5/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/io/PubsubClient.java
----------------------------------------------------------------------
diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/io/PubsubClient.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/io/PubsubClient.java
deleted file mode 100644
index 931fe6e..0000000
--- a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/io/PubsubClient.java
+++ /dev/null
@@ -1,543 +0,0 @@
-/*
- * 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.beam.integration.nexmark.io;
-
-import static com.google.common.base.Preconditions.checkArgument;
-import static com.google.common.base.Preconditions.checkState;
-
-import com.google.api.client.util.DateTime;
-import com.google.common.base.Objects;
-import com.google.common.base.Strings;
-import java.io.Closeable;
-import java.io.IOException;
-import java.io.Serializable;
-import java.util.Arrays;
-import java.util.List;
-import java.util.Map;
-import java.util.concurrent.ThreadLocalRandom;
-import javax.annotation.Nullable;
-import org.apache.beam.sdk.io.gcp.pubsub.PubsubOptions;
-
-/**
- * An (abstract) helper class for talking to Pubsub via an underlying transport.
- */
-abstract class PubsubClient implements Closeable {
-  /**
-   * Factory for creating clients.
-   */
-  public interface PubsubClientFactory extends Serializable {
-    /**
-     * Construct a new Pubsub client. It should be closed via {@link #close} in order
-     * to ensure tidy cleanup of underlying netty resources (or use the try-with-resources
-     * construct). Uses {@code options} to derive pubsub endpoints and application credentials.
-     * If non-{@literal null}, use {@code timestampLabel} and {@code idLabel} to store custom
-     * timestamps/ids within message metadata.
-     */
-    PubsubClient newClient(@Nullable String timestampLabel,
-        @Nullable String idLabel, PubsubOptions options) throws IOException;
-
-    /**
-     * Return the display name for this factory. Eg "Json", "gRPC".
-     */
-    String getKind();
-  }
-
-  /**
-   * Return timestamp as ms-since-unix-epoch corresponding to {@code timestamp}.
-   * Return {@literal null} if no timestamp could be found. Throw {@link IllegalArgumentException}
-   * if timestamp cannot be recognized.
-   */
-  @Nullable
-  private static Long asMsSinceEpoch(@Nullable String timestamp) {
-    if (Strings.isNullOrEmpty(timestamp)) {
-      return null;
-    }
-    try {
-      // Try parsing as milliseconds since epoch. Note there is no way to parse a
-      // string in RFC 3339 format here.
-      // Expected IllegalArgumentException if parsing fails; we use that to fall back
-      // to RFC 3339.
-      return Long.parseLong(timestamp);
-    } catch (IllegalArgumentException e1) {
-      // Try parsing as RFC3339 string. DateTime.parseRfc3339 will throw an
-      // IllegalArgumentException if parsing fails, and the caller should handle.
-      return DateTime.parseRfc3339(timestamp).getValue();
-    }
-  }
-
-  /**
-   * Return the timestamp (in ms since unix epoch) to use for a Pubsub message with {@code
-   * attributes} and {@code pubsubTimestamp}.
-   *
-   * <p>If {@code timestampLabel} is non-{@literal null} then the message attributes must contain
-   * that label, and the value of that label will be taken as the timestamp.
-   * Otherwise the timestamp will be taken from the Pubsub publish timestamp {@code
-   * pubsubTimestamp}.
-   *
-   * @throws IllegalArgumentException if the timestamp cannot be recognized as a ms-since-unix-epoch
-   * or RFC3339 time.
-   */
-  protected static long extractTimestamp(
-      @Nullable String timestampLabel,
-      @Nullable String pubsubTimestamp,
-      @Nullable Map<String, String> attributes) {
-    Long timestampMsSinceEpoch;
-    if (Strings.isNullOrEmpty(timestampLabel)) {
-      timestampMsSinceEpoch = asMsSinceEpoch(pubsubTimestamp);
-      checkArgument(timestampMsSinceEpoch != null,
-                    "Cannot interpret PubSub publish timestamp: %s",
-                    pubsubTimestamp);
-    } else {
-      String value = attributes == null ? null : attributes.get(timestampLabel);
-      checkArgument(value != null,
-                    "PubSub message is missing a value for timestamp label %s",
-                    timestampLabel);
-      timestampMsSinceEpoch = asMsSinceEpoch(value);
-      checkArgument(timestampMsSinceEpoch != null,
-                    "Cannot interpret value of label %s as timestamp: %s",
-                    timestampLabel, value);
-    }
-    return timestampMsSinceEpoch;
-  }
-
-  /**
-   * Path representing a cloud project id.
-   */
-  static class ProjectPath implements Serializable {
-    private final String projectId;
-
-    /**
-     * Creates a {@link ProjectPath} from a {@link String} representation, which
-     * must be of the form {@code "projects/" + projectId}.
-     */
-    ProjectPath(String path) {
-      String[] splits = path.split("/");
-      checkArgument(
-          splits.length == 2 && splits[0].equals("projects"),
-          "Malformed project path \"%s\": must be of the form \"projects/\" + <project id>",
-          path);
-      this.projectId = splits[1];
-    }
-
-    public String getPath() {
-      return String.format("projects/%s", projectId);
-    }
-
-    public String getId() {
-      return projectId;
-    }
-
-    @Override
-    public boolean equals(Object o) {
-      if (this == o) {
-        return true;
-      }
-      if (o == null || getClass() != o.getClass()) {
-        return false;
-      }
-
-      ProjectPath that = (ProjectPath) o;
-
-      return projectId.equals(that.projectId);
-    }
-
-    @Override
-    public int hashCode() {
-      return projectId.hashCode();
-    }
-
-    @Override
-    public String toString() {
-      return getPath();
-    }
-  }
-
-  public static ProjectPath projectPathFromPath(String path) {
-    return new ProjectPath(path);
-  }
-
-  public static ProjectPath projectPathFromId(String projectId) {
-    return new ProjectPath(String.format("projects/%s", projectId));
-  }
-
-  /**
-   * Path representing a Pubsub subscription.
-   */
-  public static class SubscriptionPath implements Serializable {
-    private final String projectId;
-    private final String subscriptionName;
-
-    SubscriptionPath(String path) {
-      String[] splits = path.split("/");
-      checkState(
-          splits.length == 4 && splits[0].equals("projects") && splits[2].equals("subscriptions"),
-          "Malformed subscription path %s: "
-          + "must be of the form \"projects/\" + <project id> + \"subscriptions\"", path);
-      this.projectId = splits[1];
-      this.subscriptionName = splits[3];
-    }
-
-    public String getPath() {
-      return String.format("projects/%s/subscriptions/%s", projectId, subscriptionName);
-    }
-
-    public String getName() {
-      return subscriptionName;
-    }
-
-    public String getV1Beta1Path() {
-      return String.format("/subscriptions/%s/%s", projectId, subscriptionName);
-    }
-
-    @Override
-    public boolean equals(Object o) {
-      if (this == o) {
-        return true;
-      }
-      if (o == null || getClass() != o.getClass()) {
-        return false;
-      }
-      SubscriptionPath that = (SubscriptionPath) o;
-      return this.subscriptionName.equals(that.subscriptionName)
-          && this.projectId.equals(that.projectId);
-    }
-
-    @Override
-    public int hashCode() {
-      return Objects.hashCode(projectId, subscriptionName);
-    }
-
-    @Override
-    public String toString() {
-      return getPath();
-    }
-  }
-
-  public static SubscriptionPath subscriptionPathFromPath(String path) {
-    return new SubscriptionPath(path);
-  }
-
-  public static SubscriptionPath subscriptionPathFromName(
-      String projectId, String subscriptionName) {
-    return new SubscriptionPath(String.format("projects/%s/subscriptions/%s",
-                                              projectId, subscriptionName));
-  }
-
-  /**
-   * Path representing a Pubsub topic.
-   */
-  public static class TopicPath implements Serializable {
-    private final String path;
-
-    TopicPath(String path) {
-      this.path = path;
-    }
-
-    public String getPath() {
-      return path;
-    }
-
-    public String getName() {
-      String[] splits = path.split("/");
-      checkState(splits.length == 4, "Malformed topic path %s", path);
-      return splits[3];
-    }
-
-    public String getV1Beta1Path() {
-      String[] splits = path.split("/");
-      checkState(splits.length == 4, "Malformed topic path %s", path);
-      return String.format("/topics/%s/%s", splits[1], splits[3]);
-    }
-
-    @Override
-    public boolean equals(Object o) {
-      if (this == o) {
-        return true;
-      }
-      if (o == null || getClass() != o.getClass()) {
-        return false;
-      }
-      TopicPath topicPath = (TopicPath) o;
-      return path.equals(topicPath.path);
-    }
-
-    @Override
-    public int hashCode() {
-      return path.hashCode();
-    }
-
-    @Override
-    public String toString() {
-      return path;
-    }
-  }
-
-  public static TopicPath topicPathFromPath(String path) {
-    return new TopicPath(path);
-  }
-
-  public static TopicPath topicPathFromName(String projectId, String topicName) {
-    return new TopicPath(String.format("projects/%s/topics/%s", projectId, topicName));
-  }
-
-  /**
-   * A message to be sent to Pubsub.
-   *
-   * <p>NOTE: This class is {@link Serializable} only to support the {@link PubsubTestClient}.
-   * Java serialization is never used for non-test clients.
-   */
-  static class OutgoingMessage implements Serializable {
-    /**
-     * Underlying (encoded) element.
-     */
-    public final byte[] elementBytes;
-
-    public final Map<String, String> attributes;
-
-    /**
-     * Timestamp for element (ms since epoch).
-     */
-    public final long timestampMsSinceEpoch;
-
-    /**
-     * If using an id label, the record id to associate with this record's metadata so the receiver
-     * can reject duplicates. Otherwise {@literal null}.
-     */
-    @Nullable
-    public final String recordId;
-
-    public OutgoingMessage(byte[] elementBytes, Map<String, String> attributes,
-                           long timestampMsSinceEpoch, @Nullable String recordId) {
-      this.elementBytes = elementBytes;
-      this.attributes = attributes;
-      this.timestampMsSinceEpoch = timestampMsSinceEpoch;
-      this.recordId = recordId;
-    }
-
-    @Override
-    public String toString() {
-      return String.format("OutgoingMessage(%db, %dms)",
-                           elementBytes.length, timestampMsSinceEpoch);
-    }
-
-    @Override
-    public boolean equals(Object o) {
-      if (this == o) {
-        return true;
-      }
-      if (o == null || getClass() != o.getClass()) {
-        return false;
-      }
-
-      OutgoingMessage that = (OutgoingMessage) o;
-
-      return timestampMsSinceEpoch == that.timestampMsSinceEpoch
-              && Arrays.equals(elementBytes, that.elementBytes)
-              && Objects.equal(attributes, that.attributes)
-              && Objects.equal(recordId, that.recordId);
-    }
-
-    @Override
-    public int hashCode() {
-      return Objects.hashCode(Arrays.hashCode(elementBytes), attributes, timestampMsSinceEpoch,
-              recordId);
-    }
-  }
-
-  /**
-   * A message received from Pubsub.
-   *
-   * <p>NOTE: This class is {@link Serializable} only to support the {@link PubsubTestClient}.
-   * Java serialization is never used for non-test clients.
-   */
-  static class IncomingMessage implements Serializable {
-    /**
-     * Underlying (encoded) element.
-     */
-    public final byte[] elementBytes;
-
-    public Map<String, String> attributes;
-
-    /**
-     * Timestamp for element (ms since epoch). Either Pubsub's processing time,
-     * or the custom timestamp associated with the message.
-     */
-    public final long timestampMsSinceEpoch;
-
-    /**
-     * Timestamp (in system time) at which we requested the message (ms since epoch).
-     */
-    public final long requestTimeMsSinceEpoch;
-
-    /**
-     * Id to pass back to Pubsub to acknowledge receipt of this message.
-     */
-    public final String ackId;
-
-    /**
-     * Id to pass to the runner to distinguish this message from all others.
-     */
-    public final String recordId;
-
-    public IncomingMessage(
-        byte[] elementBytes,
-        Map<String, String> attributes,
-        long timestampMsSinceEpoch,
-        long requestTimeMsSinceEpoch,
-        String ackId,
-        String recordId) {
-      this.elementBytes = elementBytes;
-      this.attributes = attributes;
-      this.timestampMsSinceEpoch = timestampMsSinceEpoch;
-      this.requestTimeMsSinceEpoch = requestTimeMsSinceEpoch;
-      this.ackId = ackId;
-      this.recordId = recordId;
-    }
-
-    public IncomingMessage withRequestTime(long requestTimeMsSinceEpoch) {
-      return new IncomingMessage(elementBytes, attributes, timestampMsSinceEpoch,
-              requestTimeMsSinceEpoch, ackId, recordId);
-    }
-
-    @Override
-    public String toString() {
-      return String.format("IncomingMessage(%db, %dms)",
-                           elementBytes.length, timestampMsSinceEpoch);
-    }
-
-    @Override
-    public boolean equals(Object o) {
-      if (this == o) {
-        return true;
-      }
-      if (o == null || getClass() != o.getClass()) {
-        return false;
-      }
-
-      IncomingMessage that = (IncomingMessage) o;
-
-      return timestampMsSinceEpoch == that.timestampMsSinceEpoch
-             && requestTimeMsSinceEpoch == that.requestTimeMsSinceEpoch
-             && ackId.equals(that.ackId)
-             && recordId.equals(that.recordId)
-             && Arrays.equals(elementBytes, that.elementBytes)
-              && Objects.equal(attributes, that.attributes);
-    }
-
-    @Override
-    public int hashCode() {
-      return Objects.hashCode(Arrays.hashCode(elementBytes), attributes, timestampMsSinceEpoch,
-                              requestTimeMsSinceEpoch,
-                              ackId, recordId);
-    }
-  }
-
-  /**
-   * Publish {@code outgoingMessages} to Pubsub {@code topic}. Return number of messages
-   * published.
-   */
-  public abstract int publish(TopicPath topic, List<OutgoingMessage> outgoingMessages)
-      throws IOException;
-
-  /**
-   * Request the next batch of up to {@code batchSize} messages from {@code subscription}.
-   * Return the received messages, or empty collection if none were available. Does not
-   * wait for messages to arrive if {@code returnImmediately} is {@literal true}.
-   * Returned messages will record their request time as {@code requestTimeMsSinceEpoch}.
-   */
-  public abstract List<IncomingMessage> pull(
-      long requestTimeMsSinceEpoch,
-      SubscriptionPath subscription,
-      int batchSize,
-      boolean returnImmediately)
-      throws IOException;
-
-  /**
-   * Acknowldege messages from {@code subscription} with {@code ackIds}.
-   */
-  public abstract void acknowledge(SubscriptionPath subscription, List<String> ackIds)
-      throws IOException;
-
-  /**
-   * Modify the ack deadline for messages from {@code subscription} with {@code ackIds} to
-   * be {@code deadlineSeconds} from now.
-   */
-  public abstract void modifyAckDeadline(
-      SubscriptionPath subscription, List<String> ackIds,
-      int deadlineSeconds) throws IOException;
-
-  /**
-   * Create {@code topic}.
-   */
-  public abstract void createTopic(TopicPath topic) throws IOException;
-
-  /*
-   * Delete {@code topic}.
-   */
-  public abstract void deleteTopic(TopicPath topic) throws IOException;
-
-  /**
-   * Return a list of topics for {@code project}.
-   */
-  public abstract List<TopicPath> listTopics(ProjectPath project) throws IOException;
-
-  /**
-   * Create {@code subscription} to {@code topic}.
-   */
-  public abstract void createSubscription(
-      TopicPath topic, SubscriptionPath subscription, int ackDeadlineSeconds) throws IOException;
-
-  /**
-   * Create a random subscription for {@code topic}. Return the {@link SubscriptionPath}. It
-   * is the responsibility of the caller to later delete the subscription.
-   */
-  public SubscriptionPath createRandomSubscription(
-      ProjectPath project, TopicPath topic, int ackDeadlineSeconds) throws IOException {
-    // Create a randomized subscription derived from the topic name.
-    String subscriptionName = topic.getName() + "_beam_" + ThreadLocalRandom.current().nextLong();
-    SubscriptionPath subscription =
-        PubsubClient
-            .subscriptionPathFromName(project.getId(), subscriptionName);
-    createSubscription(topic, subscription, ackDeadlineSeconds);
-    return subscription;
-  }
-
-  /**
-   * Delete {@code subscription}.
-   */
-  public abstract void deleteSubscription(SubscriptionPath subscription) throws IOException;
-
-  /**
-   * Return a list of subscriptions for {@code topic} in {@code project}.
-   */
-  public abstract List<SubscriptionPath> listSubscriptions(ProjectPath project, TopicPath topic)
-      throws IOException;
-
-  /**
-   * Return the ack deadline, in seconds, for {@code subscription}.
-   */
-  public abstract int ackDeadlineSeconds(SubscriptionPath subscription) throws IOException;
-
-  /**
-   * Return {@literal true} if {@link #pull} will always return empty list. Actual clients
-   * will return {@literal false}. Test clients may return {@literal true} to signal that all
-   * expected messages have been pulled and the test may complete.
-   */
-  public abstract boolean isEOF();
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/a6dbdfa5/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/io/PubsubHelper.java
----------------------------------------------------------------------
diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/io/PubsubHelper.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/io/PubsubHelper.java
deleted file mode 100644
index bcc5b1c..0000000
--- a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/io/PubsubHelper.java
+++ /dev/null
@@ -1,215 +0,0 @@
-/*
- * 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.beam.integration.nexmark.io;
-
-import java.io.IOException;
-import java.util.ArrayList;
-import java.util.Collection;
-import java.util.List;
-
-import org.apache.beam.integration.nexmark.NexmarkUtils;
-import org.apache.beam.sdk.io.gcp.pubsub.PubsubOptions;
-
-/**
- * Helper for working with pubsub.
- */
-public class PubsubHelper implements AutoCloseable {
-  /**
-   * Underlying pub/sub client.
-   */
-  private final PubsubClient pubsubClient;
-
-  /**
-   * Project id.
-   */
-  private final String projectId;
-
-  /**
-   * Topics we should delete on close.
-   */
-  private final List<PubsubClient.TopicPath> createdTopics;
-
-  /**
-   * Subscriptions we should delete on close.
-   */
-  private final List<PubsubClient.SubscriptionPath> createdSubscriptions;
-
-  private PubsubHelper(PubsubClient pubsubClient, String projectId) {
-    this.pubsubClient = pubsubClient;
-    this.projectId = projectId;
-    createdTopics = new ArrayList<>();
-    createdSubscriptions = new ArrayList<>();
-  }
-
-  /**
-   * Create a helper.
-   */
-  public static PubsubHelper create(PubsubOptions options) {
-    try {
-      return new PubsubHelper(
-          PubsubJsonClient.FACTORY.newClient(null, null, options),
-          options.getProject());
-    } catch (IOException e) {
-      throw new RuntimeException("Unable to create Pubsub client: ", e);
-    }
-  }
-
-  /**
-   * Create a topic from short name. Delete it if it already exists. Ensure the topic will be
-   * deleted on cleanup. Return full topic name.
-   */
-  public PubsubClient.TopicPath createTopic(String shortTopic) {
-    PubsubClient.TopicPath topic = PubsubClient.topicPathFromName(projectId, shortTopic);
-    try {
-      if (topicExists(shortTopic)) {
-        NexmarkUtils.console("attempting to cleanup topic %s", topic);
-        pubsubClient.deleteTopic(topic);
-      }
-      NexmarkUtils.console("create topic %s", topic);
-      pubsubClient.createTopic(topic);
-      createdTopics.add(topic);
-      return topic;
-    } catch (IOException e) {
-      throw new RuntimeException("Unable to create Pubsub topic " + topic + ": ", e);
-    }
-  }
-
-  /**
-   * Create a topic from short name if it does not already exist. The topic will not be
-   * deleted on cleanup. Return full topic name.
-   */
-  public PubsubClient.TopicPath createOrReuseTopic(String shortTopic) {
-    PubsubClient.TopicPath topic = PubsubClient.topicPathFromName(projectId, shortTopic);
-    try {
-      if (topicExists(shortTopic)) {
-        NexmarkUtils.console("topic %s already exists", topic);
-        return topic;
-      }
-      NexmarkUtils.console("create topic %s", topic);
-      pubsubClient.createTopic(topic);
-      return topic;
-    } catch (IOException e) {
-      throw new RuntimeException("Unable to create or reuse Pubsub topic " + topic + ": ", e);
-    }
-  }
-
-  /**
-   * Check a topic corresponding to short name exists, and throw exception if not. The
-   * topic will not be deleted on cleanup. Return full topic name.
-   */
-  public PubsubClient.TopicPath reuseTopic(String shortTopic) {
-    PubsubClient.TopicPath topic = PubsubClient.topicPathFromName(projectId, shortTopic);
-    if (topicExists(shortTopic)) {
-      NexmarkUtils.console("reusing existing topic %s", topic);
-      return topic;
-    }
-    throw new RuntimeException("topic '" + topic + "' does not already exist");
-  }
-
-  /**
-   * Does topic corresponding to short name exist?
-   */
-  public boolean topicExists(String shortTopic) {
-    PubsubClient.ProjectPath project = PubsubClient.projectPathFromId(projectId);
-    PubsubClient.TopicPath topic = PubsubClient.topicPathFromName(projectId, shortTopic);
-    try {
-      Collection<PubsubClient.TopicPath> existingTopics = pubsubClient.listTopics(project);
-      return existingTopics.contains(topic);
-    } catch (IOException e) {
-      throw new RuntimeException("Unable to check Pubsub topic " + topic + ": ", e);
-    }
-  }
-
-  /**
-   * Create subscription from short name. Delete subscription if it already exists. Ensure the
-   * subscription will be deleted on cleanup. Return full subscription name.
-   */
-  public PubsubClient.SubscriptionPath createSubscription(
-      String shortTopic, String shortSubscription) {
-    PubsubClient.TopicPath topic = PubsubClient.topicPathFromName(projectId, shortTopic);
-    PubsubClient.SubscriptionPath subscription =
-        PubsubClient.subscriptionPathFromName(projectId, shortSubscription);
-    try {
-      if (subscriptionExists(shortTopic, shortSubscription)) {
-        NexmarkUtils.console("attempting to cleanup subscription %s", subscription);
-        pubsubClient.deleteSubscription(subscription);
-      }
-      NexmarkUtils.console("create subscription %s", subscription);
-      pubsubClient.createSubscription(topic, subscription, 60);
-      createdSubscriptions.add(subscription);
-    } catch (IOException e) {
-      throw new RuntimeException("Unable to create Pubsub subscription " + subscription + ": ", e);
-    }
-    return subscription;
-  }
-
-  /**
-   * Check a subscription corresponding to short name exists, and throw exception if not. The
-   * subscription will not be deleted on cleanup. Return full topic name.
-   */
-  public PubsubClient.SubscriptionPath reuseSubscription(
-      String shortTopic, String shortSubscription) {
-    PubsubClient.SubscriptionPath subscription =
-        PubsubClient.subscriptionPathFromName(projectId, shortSubscription);
-    if (subscriptionExists(shortTopic, shortSubscription)) {
-      NexmarkUtils.console("reusing existing subscription %s", subscription);
-      return subscription;
-    }
-    throw new RuntimeException("subscription'" + subscription + "' does not already exist");
-  }
-
-  /**
-   * Does subscription corresponding to short name exist?
-   */
-  public boolean subscriptionExists(String shortTopic, String shortSubscription) {
-    PubsubClient.ProjectPath project = PubsubClient.projectPathFromId(projectId);
-    PubsubClient.TopicPath topic = PubsubClient.topicPathFromName(projectId, shortTopic);
-    PubsubClient.SubscriptionPath subscription =
-        PubsubClient.subscriptionPathFromName(projectId, shortSubscription);
-    try {
-      Collection<PubsubClient.SubscriptionPath> existingSubscriptions =
-          pubsubClient.listSubscriptions(project, topic);
-      return existingSubscriptions.contains(subscription);
-    } catch (IOException e) {
-      throw new RuntimeException("Unable to check Pubsub subscription" + subscription + ": ", e);
-    }
-  }
-
-  /**
-   * Delete all the subscriptions and topics we created.
-   */
-  @Override
-  public void close() {
-    for (PubsubClient.SubscriptionPath subscription : createdSubscriptions) {
-      try {
-        NexmarkUtils.console("delete subscription %s", subscription);
-        pubsubClient.deleteSubscription(subscription);
-      } catch (IOException ex) {
-        NexmarkUtils.console("could not delete subscription %s", subscription);
-      }
-    }
-    for (PubsubClient.TopicPath topic : createdTopics) {
-      try {
-        NexmarkUtils.console("delete topic %s", topic);
-        pubsubClient.deleteTopic(topic);
-      } catch (IOException ex) {
-        NexmarkUtils.console("could not delete topic %s", topic);
-      }
-    }
-  }
-}


[44/55] [abbrv] beam git commit: Move module beam-integration-java-nexmark to beam-sdks-java-nexmark

Posted by ie...@apache.org.
http://git-wip-us.apache.org/repos/asf/beam/blob/f4333df7/sdks/java/nexmark/src/main/java/org/apache/beam/sdk/nexmark/NexmarkUtils.java
----------------------------------------------------------------------
diff --git a/sdks/java/nexmark/src/main/java/org/apache/beam/sdk/nexmark/NexmarkUtils.java b/sdks/java/nexmark/src/main/java/org/apache/beam/sdk/nexmark/NexmarkUtils.java
new file mode 100644
index 0000000..fa1ef16
--- /dev/null
+++ b/sdks/java/nexmark/src/main/java/org/apache/beam/sdk/nexmark/NexmarkUtils.java
@@ -0,0 +1,674 @@
+/*
+ * 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.beam.sdk.nexmark;
+
+import com.fasterxml.jackson.databind.ObjectMapper;
+import com.google.common.hash.Hashing;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
+import java.nio.charset.StandardCharsets;
+import java.util.Iterator;
+import org.apache.beam.sdk.Pipeline;
+import org.apache.beam.sdk.coders.AvroCoder;
+import org.apache.beam.sdk.coders.ByteArrayCoder;
+import org.apache.beam.sdk.coders.Coder;
+import org.apache.beam.sdk.coders.CoderException;
+import org.apache.beam.sdk.coders.CoderRegistry;
+import org.apache.beam.sdk.coders.CustomCoder;
+import org.apache.beam.sdk.coders.SerializableCoder;
+import org.apache.beam.sdk.io.Read;
+import org.apache.beam.sdk.metrics.Counter;
+import org.apache.beam.sdk.metrics.Metrics;
+import org.apache.beam.sdk.nexmark.model.Auction;
+import org.apache.beam.sdk.nexmark.model.AuctionBid;
+import org.apache.beam.sdk.nexmark.model.AuctionCount;
+import org.apache.beam.sdk.nexmark.model.AuctionPrice;
+import org.apache.beam.sdk.nexmark.model.Bid;
+import org.apache.beam.sdk.nexmark.model.BidsPerSession;
+import org.apache.beam.sdk.nexmark.model.CategoryPrice;
+import org.apache.beam.sdk.nexmark.model.Done;
+import org.apache.beam.sdk.nexmark.model.Event;
+import org.apache.beam.sdk.nexmark.model.IdNameReserve;
+import org.apache.beam.sdk.nexmark.model.KnownSize;
+import org.apache.beam.sdk.nexmark.model.NameCityStateId;
+import org.apache.beam.sdk.nexmark.model.Person;
+import org.apache.beam.sdk.nexmark.model.SellerPrice;
+import org.apache.beam.sdk.nexmark.sources.BoundedEventSource;
+import org.apache.beam.sdk.nexmark.sources.Generator;
+import org.apache.beam.sdk.nexmark.sources.GeneratorConfig;
+import org.apache.beam.sdk.nexmark.sources.UnboundedEventSource;
+import org.apache.beam.sdk.state.StateSpec;
+import org.apache.beam.sdk.state.StateSpecs;
+import org.apache.beam.sdk.state.ValueState;
+import org.apache.beam.sdk.transforms.Combine;
+import org.apache.beam.sdk.transforms.DoFn;
+import org.apache.beam.sdk.transforms.PTransform;
+import org.apache.beam.sdk.transforms.ParDo;
+import org.apache.beam.sdk.transforms.windowing.AfterPane;
+import org.apache.beam.sdk.transforms.windowing.BoundedWindow;
+import org.apache.beam.sdk.transforms.windowing.GlobalWindows;
+import org.apache.beam.sdk.transforms.windowing.Window;
+import org.apache.beam.sdk.values.KV;
+import org.apache.beam.sdk.values.PBegin;
+import org.apache.beam.sdk.values.PCollection;
+import org.apache.beam.sdk.values.TimestampedValue;
+import org.joda.time.Duration;
+import org.joda.time.Instant;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Odd's 'n Ends used throughout queries and driver.
+ */
+public class NexmarkUtils {
+  private static final Logger LOG = LoggerFactory.getLogger(NexmarkUtils.class);
+
+  /**
+   * Mapper for (de)serializing JSON.
+   */
+  public static final ObjectMapper MAPPER = new ObjectMapper();
+
+  /**
+   * Possible sources for events.
+   */
+  public enum SourceType {
+    /**
+     * Produce events directly.
+     */
+    DIRECT,
+    /**
+     * Read events from an Avro file.
+     */
+    AVRO,
+    /**
+     * Read from a PubSub topic. It will be fed the same synthetic events by this pipeline.
+     */
+    PUBSUB
+  }
+
+  /**
+   * Possible sinks for query results.
+   */
+  public enum SinkType {
+    /**
+     * Discard all results.
+     */
+    COUNT_ONLY,
+    /**
+     * Discard all results after converting them to strings.
+     */
+    DEVNULL,
+    /**
+     * Write to a PubSub topic. It will be drained by this pipeline.
+     */
+    PUBSUB,
+    /**
+     * Write to a text file. Only works in batch mode.
+     */
+    TEXT,
+    /**
+     * Write raw Events to Avro. Only works in batch mode.
+     */
+    AVRO,
+    /**
+     * Write raw Events to BigQuery.
+     */
+    BIGQUERY,
+  }
+
+  /**
+   * Pub/sub mode to run in.
+   */
+  public enum PubSubMode {
+    /**
+     * Publish events to pub/sub, but don't run the query.
+     */
+    PUBLISH_ONLY,
+    /**
+     * Consume events from pub/sub and run the query, but don't publish.
+     */
+    SUBSCRIBE_ONLY,
+    /**
+     * Both publish and consume, but as separate jobs.
+     */
+    COMBINED
+  }
+
+  /**
+   * Coder strategies.
+   */
+  public enum CoderStrategy {
+    /**
+     * Hand-written.
+     */
+    HAND,
+    /**
+     * Avro.
+     */
+    AVRO,
+    /**
+     * Java serialization.
+     */
+    JAVA
+  }
+
+  /**
+   * How to determine resource names.
+   */
+  public enum ResourceNameMode {
+    /** Names are used as provided. */
+    VERBATIM,
+    /** Names are suffixed with the query being run. */
+    QUERY,
+    /** Names are suffixed with the query being run and a random number. */
+    QUERY_AND_SALT
+  }
+
+  /**
+   * Units for rates.
+   */
+  public enum RateUnit {
+    PER_SECOND(1_000_000L),
+    PER_MINUTE(60_000_000L);
+
+    RateUnit(long usPerUnit) {
+      this.usPerUnit = usPerUnit;
+    }
+
+    /**
+     * Number of microseconds per unit.
+     */
+    private final long usPerUnit;
+
+    /**
+     * Number of microseconds between events at given rate.
+     */
+    public long rateToPeriodUs(long rate) {
+      return (usPerUnit + rate / 2) / rate;
+    }
+  }
+
+  /**
+   * Shape of event rate.
+   */
+  public enum RateShape {
+    SQUARE,
+    SINE;
+
+    /**
+     * Number of steps used to approximate sine wave.
+     */
+    private static final int N = 10;
+
+    /**
+     * Return inter-event delay, in microseconds, for each generator
+     * to follow in order to achieve {@code rate} at {@code unit} using {@code numGenerators}.
+     */
+    public long interEventDelayUs(int rate, RateUnit unit, int numGenerators) {
+      return unit.rateToPeriodUs(rate) * numGenerators;
+    }
+
+    /**
+     * Return array of successive inter-event delays, in microseconds, for each generator
+     * to follow in order to achieve this shape with {@code firstRate/nextRate} at
+     * {@code unit} using {@code numGenerators}.
+     */
+    public long[] interEventDelayUs(
+        int firstRate, int nextRate, RateUnit unit, int numGenerators) {
+      if (firstRate == nextRate) {
+        long[] interEventDelayUs = new long[1];
+        interEventDelayUs[0] = unit.rateToPeriodUs(firstRate) * numGenerators;
+        return interEventDelayUs;
+      }
+
+      switch (this) {
+        case SQUARE: {
+          long[] interEventDelayUs = new long[2];
+          interEventDelayUs[0] = unit.rateToPeriodUs(firstRate) * numGenerators;
+          interEventDelayUs[1] = unit.rateToPeriodUs(nextRate) * numGenerators;
+          return interEventDelayUs;
+        }
+        case SINE: {
+          double mid = (firstRate + nextRate) / 2.0;
+          double amp = (firstRate - nextRate) / 2.0; // may be -ve
+          long[] interEventDelayUs = new long[N];
+          for (int i = 0; i < N; i++) {
+            double r = (2.0 * Math.PI * i) / N;
+            double rate = mid + amp * Math.cos(r);
+            interEventDelayUs[i] = unit.rateToPeriodUs(Math.round(rate)) * numGenerators;
+          }
+          return interEventDelayUs;
+        }
+      }
+      throw new RuntimeException(); // switch should be exhaustive
+    }
+
+    /**
+     * Return delay between steps, in seconds, for result of {@link #interEventDelayUs}, so
+     * as to cycle through the entire sequence every {@code ratePeriodSec}.
+     */
+    public int stepLengthSec(int ratePeriodSec) {
+      int n = 0;
+      switch (this) {
+        case SQUARE:
+          n = 2;
+          break;
+        case SINE:
+          n = N;
+          break;
+      }
+      return (ratePeriodSec + n - 1) / n;
+    }
+  }
+
+  /**
+   * Set to true to capture all info messages. The logging level flags don't currently work.
+   */
+  private static final boolean LOG_INFO = false;
+
+  /**
+   * Set to true to capture all error messages. The logging level flags don't currently work.
+   */
+  private static final boolean LOG_ERROR = true;
+
+  /**
+   * Set to true to log directly to stdout. If run using Google Dataflow, you can watch the results
+   * in real-time with: tail -f /var/log/dataflow/streaming-harness/harness-stdout.log
+   */
+  private static final boolean LOG_TO_CONSOLE = false;
+
+  /**
+   * Log info message.
+   */
+  public static void info(String format, Object... args) {
+    if (LOG_INFO) {
+      LOG.info(String.format(format, args));
+      if (LOG_TO_CONSOLE) {
+        System.out.println(String.format(format, args));
+      }
+    }
+  }
+
+  /**
+   * Log message to console. For client side only.
+   */
+  public static void console(String format, Object... args) {
+    System.out.printf("%s %s%n", Instant.now(), String.format(format, args));
+  }
+
+  /**
+   * Label to use for timestamps on pub/sub messages.
+   */
+  public static final String PUBSUB_TIMESTAMP = "timestamp";
+
+  /**
+   * Label to use for windmill ids on pub/sub messages.
+   */
+  public static final String PUBSUB_ID = "id";
+
+  /**
+   * All events will be given a timestamp relative to this time (ms since epoch).
+   */
+  private static final long BASE_TIME = Instant.parse("2015-07-15T00:00:00.000Z").getMillis();
+
+  /**
+   * Instants guaranteed to be strictly before and after all event timestamps, and which won't
+   * be subject to underflow/overflow.
+   */
+  public static final Instant BEGINNING_OF_TIME = new Instant(0).plus(Duration.standardDays(365));
+  public static final Instant END_OF_TIME =
+      BoundedWindow.TIMESTAMP_MAX_VALUE.minus(Duration.standardDays(365));
+
+  /**
+   * Setup pipeline with codes and some other options.
+   */
+  public static void setupPipeline(CoderStrategy coderStrategy, Pipeline p) {
+    CoderRegistry registry = p.getCoderRegistry();
+    switch (coderStrategy) {
+      case HAND:
+        registry.registerCoderForClass(Auction.class, Auction.CODER);
+        registry.registerCoderForClass(AuctionBid.class, AuctionBid.CODER);
+        registry.registerCoderForClass(AuctionCount.class, AuctionCount.CODER);
+        registry.registerCoderForClass(AuctionPrice.class, AuctionPrice.CODER);
+        registry.registerCoderForClass(Bid.class, Bid.CODER);
+        registry.registerCoderForClass(CategoryPrice.class, CategoryPrice.CODER);
+        registry.registerCoderForClass(Event.class, Event.CODER);
+        registry.registerCoderForClass(IdNameReserve.class, IdNameReserve.CODER);
+        registry.registerCoderForClass(NameCityStateId.class, NameCityStateId.CODER);
+        registry.registerCoderForClass(Person.class, Person.CODER);
+        registry.registerCoderForClass(SellerPrice.class, SellerPrice.CODER);
+        registry.registerCoderForClass(Done.class, Done.CODER);
+        registry.registerCoderForClass(BidsPerSession.class, BidsPerSession.CODER);
+        break;
+      case AVRO:
+        registry.registerCoderProvider(AvroCoder.getCoderProvider());
+        break;
+      case JAVA:
+        registry.registerCoderProvider(SerializableCoder.getCoderProvider());
+        break;
+    }
+  }
+
+  /**
+   * Return a generator config to match the given {@code options}.
+   */
+  private static GeneratorConfig standardGeneratorConfig(NexmarkConfiguration configuration) {
+    return new GeneratorConfig(configuration,
+                               configuration.useWallclockEventTime ? System.currentTimeMillis()
+                                                                   : BASE_TIME, 0,
+                               configuration.numEvents, 0);
+  }
+
+  /**
+   * Return an iterator of events using the 'standard' generator config.
+   */
+  public static Iterator<TimestampedValue<Event>> standardEventIterator(
+      NexmarkConfiguration configuration) {
+    return new Generator(standardGeneratorConfig(configuration));
+  }
+
+  /**
+   * Return a transform which yields a finite number of synthesized events generated
+   * as a batch.
+   */
+  public static PTransform<PBegin, PCollection<Event>> batchEventsSource(
+          NexmarkConfiguration configuration) {
+    return Read.from(new BoundedEventSource(standardGeneratorConfig(configuration),
+      configuration.numEventGenerators));
+  }
+
+  /**
+   * Return a transform which yields a finite number of synthesized events generated
+   * on-the-fly in real time.
+   */
+  public static PTransform<PBegin, PCollection<Event>> streamEventsSource(
+          NexmarkConfiguration configuration) {
+    return Read.from(new UnboundedEventSource(NexmarkUtils.standardGeneratorConfig(configuration),
+                                              configuration.numEventGenerators,
+                                              configuration.watermarkHoldbackSec,
+                                              configuration.isRateLimited));
+  }
+
+  /**
+   * Return a transform to pass-through events, but count them as they go by.
+   */
+  public static ParDo.SingleOutput<Event, Event> snoop(final String name) {
+    return ParDo.of(new DoFn<Event, Event>() {
+      final Counter eventCounter = Metrics.counter(name, "events");
+      final Counter newPersonCounter = Metrics.counter(name, "newPersons");
+      final Counter newAuctionCounter = Metrics.counter(name, "newAuctions");
+      final Counter bidCounter = Metrics.counter(name, "bids");
+      final Counter endOfStreamCounter = Metrics.counter(name, "endOfStream");
+
+      @ProcessElement
+      public void processElement(ProcessContext c) {
+        eventCounter.inc();
+        if (c.element().newPerson != null) {
+          newPersonCounter.inc();
+        } else if (c.element().newAuction != null) {
+          newAuctionCounter.inc();
+        } else if (c.element().bid != null) {
+          bidCounter.inc();
+        } else {
+          endOfStreamCounter.inc();
+        }
+        info("%s snooping element %s", name, c.element());
+        c.output(c.element());
+      }
+    });
+  }
+
+  /**
+   * Return a transform to count and discard each element.
+   */
+  public static <T> ParDo.SingleOutput<T, Void> devNull(final String name) {
+    return ParDo.of(new DoFn<T, Void>() {
+      final Counter discardedCounterMetric = Metrics.counter(name, "discarded");
+
+      @ProcessElement
+      public void processElement(ProcessContext c) {
+        discardedCounterMetric.inc();
+      }
+    });
+  }
+
+  /**
+   * Return a transform to log each element, passing it through unchanged.
+   */
+  public static <T> ParDo.SingleOutput<T, T> log(final String name) {
+    return ParDo.of(new DoFn<T, T>() {
+      @ProcessElement
+      public void processElement(ProcessContext c) {
+        LOG.info("%s: %s", name, c.element());
+        c.output(c.element());
+      }
+    });
+  }
+
+  /**
+   * Return a transform to format each element as a string.
+   */
+  public static <T> ParDo.SingleOutput<T, String> format(final String name) {
+    return ParDo.of(new DoFn<T, String>() {
+      final Counter recordCounterMetric = Metrics.counter(name, "records");
+
+      @ProcessElement
+      public void processElement(ProcessContext c) {
+        recordCounterMetric.inc();
+        c.output(c.element().toString());
+      }
+    });
+  }
+
+  /**
+   * Return a transform to make explicit the timestamp of each element.
+   */
+  public static <T> ParDo.SingleOutput<T, TimestampedValue<T>> stamp(String name) {
+    return ParDo.of(new DoFn<T, TimestampedValue<T>>() {
+      @ProcessElement
+      public void processElement(ProcessContext c) {
+        c.output(TimestampedValue.of(c.element(), c.timestamp()));
+      }
+    });
+  }
+
+  /**
+   * Return a transform to reduce a stream to a single, order-invariant long hash.
+   */
+  public static <T> PTransform<PCollection<T>, PCollection<Long>> hash(
+      final long numEvents, String name) {
+    return new PTransform<PCollection<T>, PCollection<Long>>(name) {
+      @Override
+      public PCollection<Long> expand(PCollection<T> input) {
+        return input.apply(Window.<T>into(new GlobalWindows())
+                               .triggering(AfterPane.elementCountAtLeast((int) numEvents))
+                               .withAllowedLateness(Duration.standardDays(1))
+                               .discardingFiredPanes())
+
+                    .apply(name + ".Hash", ParDo.of(new DoFn<T, Long>() {
+                      @ProcessElement
+                      public void processElement(ProcessContext c) {
+                        long hash =
+                            Hashing.murmur3_128()
+                                   .newHasher()
+                                   .putLong(c.timestamp().getMillis())
+                                   .putString(c.element().toString(), StandardCharsets.UTF_8)
+                                   .hash()
+                                   .asLong();
+                        c.output(hash);
+                      }
+                    }))
+
+                    .apply(Combine.globally(new Combine.BinaryCombineFn<Long>() {
+                      @Override
+                      public Long apply(Long left, Long right) {
+                        return left ^ right;
+                      }
+                    }));
+      }
+    };
+  }
+
+  private static final long MASK = (1L << 16) - 1L;
+  private static final long HASH = 0x243F6A8885A308D3L;
+  private static final long INIT_PLAINTEXT = 50000L;
+
+  /**
+   * Return a transform to keep the CPU busy for given milliseconds on every record.
+   */
+  public static <T> ParDo.SingleOutput<T, T> cpuDelay(String name, final long delayMs) {
+    return ParDo.of(new DoFn<T, T>() {
+                  @ProcessElement
+                  public void processElement(ProcessContext c) {
+                    long now = System.currentTimeMillis();
+                    long end = now + delayMs;
+                    while (now < end) {
+                      // Find plaintext which hashes to HASH in lowest MASK bits.
+                      // Values chosen to roughly take 1ms on typical workstation.
+                      long p = INIT_PLAINTEXT;
+                      while (true) {
+                        long t = Hashing.murmur3_128().hashLong(p).asLong();
+                        if ((t & MASK) == (HASH & MASK)) {
+                          break;
+                        }
+                        p++;
+                      }
+                      now = System.currentTimeMillis();
+                    }
+                    c.output(c.element());
+                  }
+                });
+  }
+
+  private static final int MAX_BUFFER_SIZE = 1 << 24;
+
+  private static class DiskBusyTransform<T> extends PTransform<PCollection<T>, PCollection<T>>{
+
+    private long bytes;
+
+    private DiskBusyTransform(long bytes) {
+      this.bytes = bytes;
+    }
+
+    @Override public PCollection<T> expand(PCollection<T> input) {
+      // Add dummy key to be able to use State API
+      PCollection<KV<Integer, T>> kvCollection = input
+          .apply("diskBusy.keyElements", ParDo.of(new DoFn<T, KV<Integer, T>>() {
+
+            @ProcessElement public void processElement(ProcessContext context) {
+              context.output(KV.of(0, context.element()));
+        }
+      }));
+      // Apply actual transform that generates disk IO using state API
+      PCollection<T> output = kvCollection
+          .apply("diskBusy.generateIO", ParDo.of(new DoFn<KV<Integer, T>, T>() {
+
+            private static final String DISK_BUSY = "diskBusy";
+
+        @StateId(DISK_BUSY) private final StateSpec<ValueState<byte[]>> spec = StateSpecs
+            .value(ByteArrayCoder.of());
+
+        @ProcessElement public void processElement(ProcessContext c,
+            @StateId(DISK_BUSY) ValueState<byte[]> state) {
+          long remain = bytes;
+          long now = System.currentTimeMillis();
+          while (remain > 0) {
+            long thisBytes = Math.min(remain, MAX_BUFFER_SIZE);
+            remain -= thisBytes;
+            byte[] arr = new byte[(int) thisBytes];
+            for (int i = 0; i < thisBytes; i++) {
+              arr[i] = (byte) now;
+            }
+            state.write(arr);
+            now = System.currentTimeMillis();
+          }
+          c.output(c.element().getValue());
+        }
+      }));
+      return output;
+    }
+  }
+
+
+  /**
+   * Return a transform to write given number of bytes to durable store on every record.
+   */
+  public static <T> PTransform<PCollection<T>, PCollection<T>> diskBusy(final long bytes) {
+    return new DiskBusyTransform<>(bytes);
+  }
+
+  /**
+   * Return a transform to cast each element to {@link KnownSize}.
+   */
+  private static <T extends KnownSize> ParDo.SingleOutput<T, KnownSize> castToKnownSize() {
+    return ParDo.of(new DoFn<T, KnownSize>() {
+                  @ProcessElement
+                  public void processElement(ProcessContext c) {
+                    c.output(c.element());
+                  }
+                });
+  }
+
+  /**
+   * A coder for instances of {@code T} cast up to {@link KnownSize}.
+   *
+   * @param <T> True type of object.
+   */
+  private static class CastingCoder<T extends KnownSize> extends CustomCoder<KnownSize> {
+    private final Coder<T> trueCoder;
+
+    public CastingCoder(Coder<T> trueCoder) {
+      this.trueCoder = trueCoder;
+    }
+
+    @Override
+    public void encode(KnownSize value, OutputStream outStream)
+        throws CoderException, IOException {
+      @SuppressWarnings("unchecked")
+      T typedValue = (T) value;
+      trueCoder.encode(typedValue, outStream);
+    }
+
+    @Override
+    public KnownSize decode(InputStream inStream)
+        throws CoderException, IOException {
+      return trueCoder.decode(inStream);
+    }
+  }
+
+  /**
+   * Return a coder for {@code KnownSize} that are known to be exactly of type {@code T}.
+   */
+  private static <T extends KnownSize> Coder<KnownSize> makeCastingCoder(Coder<T> trueCoder) {
+    return new CastingCoder<>(trueCoder);
+  }
+
+  /**
+   * Return {@code elements} as {@code KnownSize}s.
+   */
+  public static <T extends KnownSize> PCollection<KnownSize> castToKnownSize(
+      final String name, PCollection<T> elements) {
+    return elements.apply(name + ".Forget", castToKnownSize())
+            .setCoder(makeCastingCoder(elements.getCoder()));
+  }
+
+  // Do not instantiate.
+  private NexmarkUtils() {
+  }
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/f4333df7/sdks/java/nexmark/src/main/java/org/apache/beam/sdk/nexmark/model/Auction.java
----------------------------------------------------------------------
diff --git a/sdks/java/nexmark/src/main/java/org/apache/beam/sdk/nexmark/model/Auction.java b/sdks/java/nexmark/src/main/java/org/apache/beam/sdk/nexmark/model/Auction.java
new file mode 100644
index 0000000..6a37ade
--- /dev/null
+++ b/sdks/java/nexmark/src/main/java/org/apache/beam/sdk/nexmark/model/Auction.java
@@ -0,0 +1,187 @@
+/*
+ * 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.beam.sdk.nexmark.model;
+
+import com.fasterxml.jackson.annotation.JsonProperty;
+import com.fasterxml.jackson.core.JsonProcessingException;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
+import java.io.Serializable;
+import org.apache.beam.sdk.coders.Coder;
+import org.apache.beam.sdk.coders.CoderException;
+import org.apache.beam.sdk.coders.CustomCoder;
+import org.apache.beam.sdk.coders.StringUtf8Coder;
+import org.apache.beam.sdk.coders.VarLongCoder;
+import org.apache.beam.sdk.nexmark.NexmarkUtils;
+
+/**
+ * An auction submitted by a person.
+ */
+public class Auction implements KnownSize, Serializable {
+  private static final Coder<Long> LONG_CODER = VarLongCoder.of();
+  private static final Coder<String> STRING_CODER = StringUtf8Coder.of();
+
+  public static final Coder<Auction> CODER = new CustomCoder<Auction>() {
+    @Override
+    public void encode(Auction value, OutputStream outStream)
+        throws CoderException, IOException {
+      LONG_CODER.encode(value.id, outStream);
+      STRING_CODER.encode(value.itemName, outStream);
+      STRING_CODER.encode(value.description, outStream);
+      LONG_CODER.encode(value.initialBid, outStream);
+      LONG_CODER.encode(value.reserve, outStream);
+      LONG_CODER.encode(value.dateTime, outStream);
+      LONG_CODER.encode(value.expires, outStream);
+      LONG_CODER.encode(value.seller, outStream);
+      LONG_CODER.encode(value.category, outStream);
+      STRING_CODER.encode(value.extra, outStream);
+    }
+
+    @Override
+    public Auction decode(
+        InputStream inStream)
+        throws CoderException, IOException {
+      long id = LONG_CODER.decode(inStream);
+      String itemName = STRING_CODER.decode(inStream);
+      String description = STRING_CODER.decode(inStream);
+      long initialBid = LONG_CODER.decode(inStream);
+      long reserve = LONG_CODER.decode(inStream);
+      long dateTime = LONG_CODER.decode(inStream);
+      long expires = LONG_CODER.decode(inStream);
+      long seller = LONG_CODER.decode(inStream);
+      long category = LONG_CODER.decode(inStream);
+      String extra = STRING_CODER.decode(inStream);
+      return new Auction(
+          id, itemName, description, initialBid, reserve, dateTime, expires, seller, category,
+          extra);
+    }
+  };
+
+
+  /** Id of auction. */
+  @JsonProperty
+  public final long id; // primary key
+
+  /** Extra auction properties. */
+  @JsonProperty
+  private final String itemName;
+
+  @JsonProperty
+  private final String description;
+
+  /** Initial bid price, in cents. */
+  @JsonProperty
+  private final long initialBid;
+
+  /** Reserve price, in cents. */
+  @JsonProperty
+  public final long reserve;
+
+  @JsonProperty
+  public final long dateTime;
+
+  /** When does auction expire? (ms since epoch). Bids at or after this time are ignored. */
+  @JsonProperty
+  public final long expires;
+
+  /** Id of person who instigated auction. */
+  @JsonProperty
+  public final long seller; // foreign key: Person.id
+
+  /** Id of category auction is listed under. */
+  @JsonProperty
+  public final long category; // foreign key: Category.id
+
+  /** Additional arbitrary payload for performance testing. */
+  @JsonProperty
+  private final String extra;
+
+
+  // For Avro only.
+  @SuppressWarnings("unused")
+  private Auction() {
+    id = 0;
+    itemName = null;
+    description = null;
+    initialBid = 0;
+    reserve = 0;
+    dateTime = 0;
+    expires = 0;
+    seller = 0;
+    category = 0;
+    extra = null;
+  }
+
+  public Auction(long id, String itemName, String description, long initialBid, long reserve,
+      long dateTime, long expires, long seller, long category, String extra) {
+    this.id = id;
+    this.itemName = itemName;
+    this.description = description;
+    this.initialBid = initialBid;
+    this.reserve = reserve;
+    this.dateTime = dateTime;
+    this.expires = expires;
+    this.seller = seller;
+    this.category = category;
+    this.extra = extra;
+  }
+
+  /**
+   * Return a copy of auction which capture the given annotation.
+   * (Used for debugging).
+   */
+  public Auction withAnnotation(String annotation) {
+    return new Auction(id, itemName, description, initialBid, reserve, dateTime, expires, seller,
+        category, annotation + ": " + extra);
+  }
+
+  /**
+   * Does auction have {@code annotation}? (Used for debugging.)
+   */
+  public boolean hasAnnotation(String annotation) {
+    return extra.startsWith(annotation + ": ");
+  }
+
+  /**
+   * Remove {@code annotation} from auction. (Used for debugging.)
+   */
+  public Auction withoutAnnotation(String annotation) {
+    if (hasAnnotation(annotation)) {
+      return new Auction(id, itemName, description, initialBid, reserve, dateTime, expires, seller,
+          category, extra.substring(annotation.length() + 2));
+    } else {
+      return this;
+    }
+  }
+
+  @Override
+  public long sizeInBytes() {
+    return 8 + itemName.length() + 1 + description.length() + 1 + 8 + 8 + 8 + 8 + 8 + 8
+        + extra.length() + 1;
+  }
+
+  @Override
+  public String toString() {
+    try {
+      return NexmarkUtils.MAPPER.writeValueAsString(this);
+    } catch (JsonProcessingException e) {
+      throw new RuntimeException(e);
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/f4333df7/sdks/java/nexmark/src/main/java/org/apache/beam/sdk/nexmark/model/AuctionBid.java
----------------------------------------------------------------------
diff --git a/sdks/java/nexmark/src/main/java/org/apache/beam/sdk/nexmark/model/AuctionBid.java b/sdks/java/nexmark/src/main/java/org/apache/beam/sdk/nexmark/model/AuctionBid.java
new file mode 100644
index 0000000..cb1aac5
--- /dev/null
+++ b/sdks/java/nexmark/src/main/java/org/apache/beam/sdk/nexmark/model/AuctionBid.java
@@ -0,0 +1,85 @@
+/*
+ * 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.beam.sdk.nexmark.model;
+
+import com.fasterxml.jackson.annotation.JsonProperty;
+import com.fasterxml.jackson.core.JsonProcessingException;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
+import java.io.Serializable;
+import org.apache.beam.sdk.coders.Coder;
+import org.apache.beam.sdk.coders.CoderException;
+import org.apache.beam.sdk.coders.CustomCoder;
+import org.apache.beam.sdk.nexmark.NexmarkUtils;
+import org.apache.beam.sdk.nexmark.queries.WinningBids;
+
+/**
+ * Result of {@link WinningBids} transform.
+ */
+public class AuctionBid implements KnownSize, Serializable {
+  public static final Coder<AuctionBid> CODER = new CustomCoder<AuctionBid>() {
+    @Override
+    public void encode(AuctionBid value, OutputStream outStream)
+        throws CoderException, IOException {
+      Auction.CODER.encode(value.auction, outStream);
+      Bid.CODER.encode(value.bid, outStream);
+    }
+
+    @Override
+    public AuctionBid decode(
+        InputStream inStream)
+        throws CoderException, IOException {
+      Auction auction = Auction.CODER.decode(inStream);
+      Bid bid = Bid.CODER.decode(inStream);
+      return new AuctionBid(auction, bid);
+    }
+  };
+
+  @JsonProperty
+  public final Auction auction;
+
+  @JsonProperty
+  public final Bid bid;
+
+  // For Avro only.
+  @SuppressWarnings("unused")
+  private AuctionBid() {
+    auction = null;
+    bid = null;
+  }
+
+  public AuctionBid(Auction auction, Bid bid) {
+    this.auction = auction;
+    this.bid = bid;
+  }
+
+  @Override
+  public long sizeInBytes() {
+    return auction.sizeInBytes() + bid.sizeInBytes();
+  }
+
+  @Override
+  public String toString() {
+    try {
+      return NexmarkUtils.MAPPER.writeValueAsString(this);
+    } catch (JsonProcessingException e) {
+      throw new RuntimeException(e);
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/f4333df7/sdks/java/nexmark/src/main/java/org/apache/beam/sdk/nexmark/model/AuctionCount.java
----------------------------------------------------------------------
diff --git a/sdks/java/nexmark/src/main/java/org/apache/beam/sdk/nexmark/model/AuctionCount.java b/sdks/java/nexmark/src/main/java/org/apache/beam/sdk/nexmark/model/AuctionCount.java
new file mode 100644
index 0000000..4d15d25
--- /dev/null
+++ b/sdks/java/nexmark/src/main/java/org/apache/beam/sdk/nexmark/model/AuctionCount.java
@@ -0,0 +1,84 @@
+/*
+ * 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.beam.sdk.nexmark.model;
+
+import com.fasterxml.jackson.annotation.JsonProperty;
+import com.fasterxml.jackson.core.JsonProcessingException;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
+import java.io.Serializable;
+import org.apache.beam.sdk.coders.Coder;
+import org.apache.beam.sdk.coders.CoderException;
+import org.apache.beam.sdk.coders.CustomCoder;
+import org.apache.beam.sdk.coders.VarLongCoder;
+import org.apache.beam.sdk.nexmark.NexmarkUtils;
+
+/**
+ * Result of Query5.
+ */
+public class AuctionCount implements KnownSize, Serializable {
+  private static final Coder<Long> LONG_CODER = VarLongCoder.of();
+
+  public static final Coder<AuctionCount> CODER = new CustomCoder<AuctionCount>() {
+    @Override
+    public void encode(AuctionCount value, OutputStream outStream)
+        throws CoderException, IOException {
+      LONG_CODER.encode(value.auction, outStream);
+      LONG_CODER.encode(value.count, outStream);
+    }
+
+    @Override
+    public AuctionCount decode(InputStream inStream)
+        throws CoderException, IOException {
+      long auction = LONG_CODER.decode(inStream);
+      long count = LONG_CODER.decode(inStream);
+      return new AuctionCount(auction, count);
+    }
+  };
+
+  @JsonProperty private final long auction;
+
+  @JsonProperty private final long count;
+
+  // For Avro only.
+  @SuppressWarnings("unused")
+  private AuctionCount() {
+    auction = 0;
+    count = 0;
+  }
+
+  public AuctionCount(long auction, long count) {
+    this.auction = auction;
+    this.count = count;
+  }
+
+  @Override
+  public long sizeInBytes() {
+    return 8 + 8;
+  }
+
+  @Override
+  public String toString() {
+    try {
+      return NexmarkUtils.MAPPER.writeValueAsString(this);
+    } catch (JsonProcessingException e) {
+      throw new RuntimeException(e);
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/f4333df7/sdks/java/nexmark/src/main/java/org/apache/beam/sdk/nexmark/model/AuctionPrice.java
----------------------------------------------------------------------
diff --git a/sdks/java/nexmark/src/main/java/org/apache/beam/sdk/nexmark/model/AuctionPrice.java b/sdks/java/nexmark/src/main/java/org/apache/beam/sdk/nexmark/model/AuctionPrice.java
new file mode 100644
index 0000000..f4fe881
--- /dev/null
+++ b/sdks/java/nexmark/src/main/java/org/apache/beam/sdk/nexmark/model/AuctionPrice.java
@@ -0,0 +1,88 @@
+/*
+ * 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.beam.sdk.nexmark.model;
+
+import com.fasterxml.jackson.annotation.JsonProperty;
+import com.fasterxml.jackson.core.JsonProcessingException;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
+import java.io.Serializable;
+import org.apache.beam.sdk.coders.Coder;
+import org.apache.beam.sdk.coders.CoderException;
+import org.apache.beam.sdk.coders.CustomCoder;
+import org.apache.beam.sdk.coders.VarLongCoder;
+import org.apache.beam.sdk.nexmark.NexmarkUtils;
+
+/**
+ * Result of Query2.
+ */
+public class AuctionPrice implements KnownSize, Serializable {
+  private static final Coder<Long> LONG_CODER = VarLongCoder.of();
+
+  public static final Coder<AuctionPrice> CODER = new CustomCoder<AuctionPrice>() {
+    @Override
+    public void encode(AuctionPrice value, OutputStream outStream)
+        throws CoderException, IOException {
+      LONG_CODER.encode(value.auction, outStream);
+      LONG_CODER.encode(value.price, outStream);
+    }
+
+    @Override
+    public AuctionPrice decode(
+        InputStream inStream)
+        throws CoderException, IOException {
+      long auction = LONG_CODER.decode(inStream);
+      long price = LONG_CODER.decode(inStream);
+      return new AuctionPrice(auction, price);
+    }
+  };
+
+  @JsonProperty
+  private final long auction;
+
+  /** Price in cents. */
+  @JsonProperty
+  private final long price;
+
+  // For Avro only.
+  @SuppressWarnings("unused")
+  private AuctionPrice() {
+    auction = 0;
+    price = 0;
+  }
+
+  public AuctionPrice(long auction, long price) {
+    this.auction = auction;
+    this.price = price;
+  }
+
+  @Override
+  public long sizeInBytes() {
+    return 8 + 8;
+  }
+
+  @Override
+  public String toString() {
+    try {
+      return NexmarkUtils.MAPPER.writeValueAsString(this);
+    } catch (JsonProcessingException e) {
+      throw new RuntimeException(e);
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/f4333df7/sdks/java/nexmark/src/main/java/org/apache/beam/sdk/nexmark/model/Bid.java
----------------------------------------------------------------------
diff --git a/sdks/java/nexmark/src/main/java/org/apache/beam/sdk/nexmark/model/Bid.java b/sdks/java/nexmark/src/main/java/org/apache/beam/sdk/nexmark/model/Bid.java
new file mode 100644
index 0000000..b465e62
--- /dev/null
+++ b/sdks/java/nexmark/src/main/java/org/apache/beam/sdk/nexmark/model/Bid.java
@@ -0,0 +1,177 @@
+/*
+ * 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.beam.sdk.nexmark.model;
+
+import com.fasterxml.jackson.annotation.JsonProperty;
+import com.fasterxml.jackson.core.JsonProcessingException;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
+import java.io.Serializable;
+import java.util.Comparator;
+import org.apache.beam.sdk.coders.Coder;
+import org.apache.beam.sdk.coders.CoderException;
+import org.apache.beam.sdk.coders.CustomCoder;
+import org.apache.beam.sdk.coders.StringUtf8Coder;
+import org.apache.beam.sdk.coders.VarLongCoder;
+import org.apache.beam.sdk.nexmark.NexmarkUtils;
+
+/**
+ * A bid for an item on auction.
+ */
+public class Bid implements KnownSize, Serializable {
+  private static final Coder<Long> LONG_CODER = VarLongCoder.of();
+  private static final Coder<String> STRING_CODER = StringUtf8Coder.of();
+
+  public static final Coder<Bid> CODER = new CustomCoder<Bid>() {
+    @Override
+    public void encode(Bid value, OutputStream outStream)
+        throws CoderException, IOException {
+      LONG_CODER.encode(value.auction, outStream);
+      LONG_CODER.encode(value.bidder, outStream);
+      LONG_CODER.encode(value.price, outStream);
+      LONG_CODER.encode(value.dateTime, outStream);
+      STRING_CODER.encode(value.extra, outStream);
+    }
+
+    @Override
+    public Bid decode(
+        InputStream inStream)
+        throws CoderException, IOException {
+      long auction = LONG_CODER.decode(inStream);
+      long bidder = LONG_CODER.decode(inStream);
+      long price = LONG_CODER.decode(inStream);
+      long dateTime = LONG_CODER.decode(inStream);
+      String extra = STRING_CODER.decode(inStream);
+      return new Bid(auction, bidder, price, dateTime, extra);
+    }
+
+    @Override public void verifyDeterministic() throws NonDeterministicException {}
+  };
+
+  /**
+   * Comparator to order bids by ascending price then descending time
+   * (for finding winning bids).
+   */
+  public static final Comparator<Bid> PRICE_THEN_DESCENDING_TIME = new Comparator<Bid>() {
+    @Override
+    public int compare(Bid left, Bid right) {
+      int i = Double.compare(left.price, right.price);
+      if (i != 0) {
+        return i;
+      }
+      return Long.compare(right.dateTime, left.dateTime);
+    }
+  };
+
+  /**
+   * Comparator to order bids by ascending time then ascending price.
+   * (for finding most recent bids).
+   */
+  public static final Comparator<Bid> ASCENDING_TIME_THEN_PRICE = new Comparator<Bid>() {
+    @Override
+    public int compare(Bid left, Bid right) {
+      int i = Long.compare(left.dateTime, right.dateTime);
+      if (i != 0) {
+        return i;
+      }
+      return Double.compare(left.price, right.price);
+    }
+  };
+
+  /** Id of auction this bid is for. */
+  @JsonProperty
+  public final long auction; // foreign key: Auction.id
+
+  /** Id of person bidding in auction. */
+  @JsonProperty
+  public final long bidder; // foreign key: Person.id
+
+  /** Price of bid, in cents. */
+  @JsonProperty
+  public final long price;
+
+  /**
+   * Instant at which bid was made (ms since epoch).
+   * NOTE: This may be earlier than the system's event time.
+   */
+  @JsonProperty
+  public final long dateTime;
+
+  /** Additional arbitrary payload for performance testing. */
+  @JsonProperty
+  public final String extra;
+
+  // For Avro only.
+  @SuppressWarnings("unused")
+  private Bid() {
+    auction = 0;
+    bidder = 0;
+    price = 0;
+    dateTime = 0;
+    extra = null;
+  }
+
+  public Bid(long auction, long bidder, long price, long dateTime, String extra) {
+    this.auction = auction;
+    this.bidder = bidder;
+    this.price = price;
+    this.dateTime = dateTime;
+    this.extra = extra;
+  }
+
+  /**
+   * Return a copy of bid which capture the given annotation.
+   * (Used for debugging).
+   */
+  public Bid withAnnotation(String annotation) {
+    return new Bid(auction, bidder, price, dateTime, annotation + ": " + extra);
+  }
+
+  /**
+   * Does bid have {@code annotation}? (Used for debugging.)
+   */
+  public boolean hasAnnotation(String annotation) {
+    return extra.startsWith(annotation + ": ");
+  }
+
+  /**
+   * Remove {@code annotation} from bid. (Used for debugging.)
+   */
+  public Bid withoutAnnotation(String annotation) {
+    if (hasAnnotation(annotation)) {
+      return new Bid(auction, bidder, price, dateTime, extra.substring(annotation.length() + 2));
+    } else {
+      return this;
+    }
+  }
+
+  @Override
+  public long sizeInBytes() {
+    return 8 + 8 + 8 + 8 + extra.length() + 1;
+  }
+
+  @Override
+  public String toString() {
+    try {
+      return NexmarkUtils.MAPPER.writeValueAsString(this);
+    } catch (JsonProcessingException e) {
+      throw new RuntimeException(e);
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/f4333df7/sdks/java/nexmark/src/main/java/org/apache/beam/sdk/nexmark/model/BidsPerSession.java
----------------------------------------------------------------------
diff --git a/sdks/java/nexmark/src/main/java/org/apache/beam/sdk/nexmark/model/BidsPerSession.java b/sdks/java/nexmark/src/main/java/org/apache/beam/sdk/nexmark/model/BidsPerSession.java
new file mode 100644
index 0000000..84e23e7
--- /dev/null
+++ b/sdks/java/nexmark/src/main/java/org/apache/beam/sdk/nexmark/model/BidsPerSession.java
@@ -0,0 +1,87 @@
+/*
+ * 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.beam.sdk.nexmark.model;
+
+import com.fasterxml.jackson.annotation.JsonProperty;
+import com.fasterxml.jackson.core.JsonProcessingException;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
+import java.io.Serializable;
+import org.apache.beam.sdk.coders.Coder;
+import org.apache.beam.sdk.coders.CoderException;
+import org.apache.beam.sdk.coders.CustomCoder;
+import org.apache.beam.sdk.coders.VarLongCoder;
+import org.apache.beam.sdk.nexmark.NexmarkUtils;
+
+/**
+ * Result of query 11.
+ */
+public class BidsPerSession implements KnownSize, Serializable {
+  private static final Coder<Long> LONG_CODER = VarLongCoder.of();
+
+  public static final Coder<BidsPerSession> CODER = new CustomCoder<BidsPerSession>() {
+    @Override
+    public void encode(BidsPerSession value, OutputStream outStream)
+        throws CoderException, IOException {
+      LONG_CODER.encode(value.personId, outStream);
+      LONG_CODER.encode(value.bidsPerSession, outStream);
+    }
+
+    @Override
+    public BidsPerSession decode(
+        InputStream inStream)
+        throws CoderException, IOException {
+      long personId = LONG_CODER.decode(inStream);
+      long bidsPerSession = LONG_CODER.decode(inStream);
+      return new BidsPerSession(personId, bidsPerSession);
+    }
+    @Override public void verifyDeterministic() throws NonDeterministicException {}
+  };
+
+  @JsonProperty
+  private final long personId;
+
+  @JsonProperty
+  private final long bidsPerSession;
+
+  public BidsPerSession() {
+    personId = 0;
+    bidsPerSession = 0;
+  }
+
+  public BidsPerSession(long personId, long bidsPerSession) {
+    this.personId = personId;
+    this.bidsPerSession = bidsPerSession;
+  }
+
+  @Override
+  public long sizeInBytes() {
+    // Two longs.
+    return 8 + 8;
+  }
+
+  @Override
+  public String toString() {
+    try {
+      return NexmarkUtils.MAPPER.writeValueAsString(this);
+    } catch (JsonProcessingException e) {
+      throw new RuntimeException(e);
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/f4333df7/sdks/java/nexmark/src/main/java/org/apache/beam/sdk/nexmark/model/CategoryPrice.java
----------------------------------------------------------------------
diff --git a/sdks/java/nexmark/src/main/java/org/apache/beam/sdk/nexmark/model/CategoryPrice.java b/sdks/java/nexmark/src/main/java/org/apache/beam/sdk/nexmark/model/CategoryPrice.java
new file mode 100644
index 0000000..3b33635
--- /dev/null
+++ b/sdks/java/nexmark/src/main/java/org/apache/beam/sdk/nexmark/model/CategoryPrice.java
@@ -0,0 +1,97 @@
+/*
+ * 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.beam.sdk.nexmark.model;
+
+import com.fasterxml.jackson.annotation.JsonProperty;
+import com.fasterxml.jackson.core.JsonProcessingException;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
+import java.io.Serializable;
+import org.apache.beam.sdk.coders.Coder;
+import org.apache.beam.sdk.coders.CoderException;
+import org.apache.beam.sdk.coders.CustomCoder;
+import org.apache.beam.sdk.coders.VarIntCoder;
+import org.apache.beam.sdk.coders.VarLongCoder;
+import org.apache.beam.sdk.nexmark.NexmarkUtils;
+
+/**
+ * Result of Query4.
+ */
+public class CategoryPrice implements KnownSize, Serializable {
+  private static final Coder<Long> LONG_CODER = VarLongCoder.of();
+  private static final Coder<Integer> INT_CODER = VarIntCoder.of();
+
+  public static final Coder<CategoryPrice> CODER = new CustomCoder<CategoryPrice>() {
+    @Override
+    public void encode(CategoryPrice value, OutputStream outStream)
+        throws CoderException, IOException {
+      LONG_CODER.encode(value.category, outStream);
+      LONG_CODER.encode(value.price, outStream);
+      INT_CODER.encode(value.isLast ? 1 : 0, outStream);
+    }
+
+    @Override
+    public CategoryPrice decode(InputStream inStream)
+        throws CoderException, IOException {
+      long category = LONG_CODER.decode(inStream);
+      long price = LONG_CODER.decode(inStream);
+      boolean isLast = INT_CODER.decode(inStream) != 0;
+      return new CategoryPrice(category, price, isLast);
+    }
+    @Override public void verifyDeterministic() throws NonDeterministicException {}
+  };
+
+  @JsonProperty
+  public final long category;
+
+  /** Price in cents. */
+  @JsonProperty
+  public final long price;
+
+  @JsonProperty
+  public final boolean isLast;
+
+  // For Avro only.
+  @SuppressWarnings("unused")
+  private CategoryPrice() {
+    category = 0;
+    price = 0;
+    isLast = false;
+  }
+
+  public CategoryPrice(long category, long price, boolean isLast) {
+    this.category = category;
+    this.price = price;
+    this.isLast = isLast;
+  }
+
+  @Override
+  public long sizeInBytes() {
+    return 8 + 8 + 1;
+  }
+
+  @Override
+  public String toString() {
+    try {
+      return NexmarkUtils.MAPPER.writeValueAsString(this);
+    } catch (JsonProcessingException e) {
+      throw new RuntimeException(e);
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/f4333df7/sdks/java/nexmark/src/main/java/org/apache/beam/sdk/nexmark/model/Done.java
----------------------------------------------------------------------
diff --git a/sdks/java/nexmark/src/main/java/org/apache/beam/sdk/nexmark/model/Done.java b/sdks/java/nexmark/src/main/java/org/apache/beam/sdk/nexmark/model/Done.java
new file mode 100644
index 0000000..e285041
--- /dev/null
+++ b/sdks/java/nexmark/src/main/java/org/apache/beam/sdk/nexmark/model/Done.java
@@ -0,0 +1,80 @@
+/*
+ * 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.beam.sdk.nexmark.model;
+
+import com.fasterxml.jackson.annotation.JsonProperty;
+import com.fasterxml.jackson.core.JsonProcessingException;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
+import java.io.Serializable;
+import org.apache.beam.sdk.coders.Coder;
+import org.apache.beam.sdk.coders.CoderException;
+import org.apache.beam.sdk.coders.CustomCoder;
+import org.apache.beam.sdk.coders.StringUtf8Coder;
+import org.apache.beam.sdk.nexmark.NexmarkUtils;
+
+/**
+ * Result of query 10.
+ */
+public class Done implements KnownSize, Serializable {
+  private static final Coder<String> STRING_CODER = StringUtf8Coder.of();
+
+  public static final Coder<Done> CODER = new CustomCoder<Done>() {
+    @Override
+    public void encode(Done value, OutputStream outStream)
+        throws CoderException, IOException {
+      STRING_CODER.encode(value.message, outStream);
+    }
+
+    @Override
+    public Done decode(InputStream inStream)
+        throws CoderException, IOException {
+      String message = STRING_CODER.decode(inStream);
+      return new Done(message);
+    }
+    @Override public void verifyDeterministic() throws NonDeterministicException {}
+  };
+
+  @JsonProperty
+  private final String message;
+
+  // For Avro only.
+  @SuppressWarnings("unused")
+  public Done() {
+    message = null;
+  }
+
+  public Done(String message) {
+    this.message = message;
+  }
+
+  @Override
+  public long sizeInBytes() {
+    return message.length();
+  }
+
+  @Override
+  public String toString() {
+    try {
+      return NexmarkUtils.MAPPER.writeValueAsString(this);
+    } catch (JsonProcessingException e) {
+      throw new RuntimeException(e);
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/f4333df7/sdks/java/nexmark/src/main/java/org/apache/beam/sdk/nexmark/model/Event.java
----------------------------------------------------------------------
diff --git a/sdks/java/nexmark/src/main/java/org/apache/beam/sdk/nexmark/model/Event.java b/sdks/java/nexmark/src/main/java/org/apache/beam/sdk/nexmark/model/Event.java
new file mode 100644
index 0000000..880cfe4
--- /dev/null
+++ b/sdks/java/nexmark/src/main/java/org/apache/beam/sdk/nexmark/model/Event.java
@@ -0,0 +1,171 @@
+/*
+ * 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.beam.sdk.nexmark.model;
+
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
+import java.io.Serializable;
+import javax.annotation.Nullable;
+import org.apache.beam.sdk.coders.Coder;
+import org.apache.beam.sdk.coders.CustomCoder;
+import org.apache.beam.sdk.coders.VarIntCoder;
+
+/**
+ * An event in the auction system, either a (new) {@link Person}, a (new) {@link Auction}, or a
+ * {@link Bid}.
+ */
+public class Event implements KnownSize, Serializable {
+  private enum Tag {
+    PERSON(0),
+    AUCTION(1),
+    BID(2);
+
+    private int value = -1;
+
+    Tag(int value){
+      this.value = value;
+    }
+  }
+  private static final Coder<Integer> INT_CODER = VarIntCoder.of();
+
+  public static final Coder<Event> CODER =
+      new CustomCoder<Event>() {
+        @Override
+        public void encode(Event value, OutputStream outStream) throws IOException {
+          if (value.newPerson != null) {
+            INT_CODER.encode(Tag.PERSON.value, outStream);
+            Person.CODER.encode(value.newPerson, outStream);
+          } else if (value.newAuction != null) {
+            INT_CODER.encode(Tag.AUCTION.value, outStream);
+            Auction.CODER.encode(value.newAuction, outStream);
+          } else if (value.bid != null) {
+            INT_CODER.encode(Tag.BID.value, outStream);
+            Bid.CODER.encode(value.bid, outStream);
+          } else {
+            throw new RuntimeException("invalid event");
+          }
+        }
+
+        @Override
+        public Event decode(InputStream inStream) throws IOException {
+          int tag = INT_CODER.decode(inStream);
+          if (tag == Tag.PERSON.value) {
+            Person person = Person.CODER.decode(inStream);
+            return new Event(person);
+          } else if (tag == Tag.AUCTION.value) {
+            Auction auction = Auction.CODER.decode(inStream);
+            return new Event(auction);
+          } else if (tag == Tag.BID.value) {
+            Bid bid = Bid.CODER.decode(inStream);
+            return new Event(bid);
+          } else {
+            throw new RuntimeException("invalid event encoding");
+          }
+        }
+
+        @Override
+        public void verifyDeterministic() throws NonDeterministicException {}
+      };
+
+  @Nullable
+  @org.apache.avro.reflect.Nullable
+  public final Person newPerson;
+
+  @Nullable
+  @org.apache.avro.reflect.Nullable
+  public final Auction newAuction;
+
+  @Nullable
+  @org.apache.avro.reflect.Nullable
+  public final Bid bid;
+
+  // For Avro only.
+  @SuppressWarnings("unused")
+  private Event() {
+    newPerson = null;
+    newAuction = null;
+    bid = null;
+  }
+
+  public Event(Person newPerson) {
+    this.newPerson = newPerson;
+    newAuction = null;
+    bid = null;
+  }
+
+  public Event(Auction newAuction) {
+    newPerson = null;
+    this.newAuction = newAuction;
+    bid = null;
+  }
+
+  public Event(Bid bid) {
+    newPerson = null;
+    newAuction = null;
+    this.bid = bid;
+  }
+
+  /** Return a copy of event which captures {@code annotation}. (Used for debugging). */
+  public Event withAnnotation(String annotation) {
+    if (newPerson != null) {
+      return new Event(newPerson.withAnnotation(annotation));
+    } else if (newAuction != null) {
+      return new Event(newAuction.withAnnotation(annotation));
+    } else {
+      return new Event(bid.withAnnotation(annotation));
+    }
+  }
+
+  /** Does event have {@code annotation}? (Used for debugging.) */
+  public boolean hasAnnotation(String annotation) {
+    if (newPerson != null) {
+      return newPerson.hasAnnotation(annotation);
+    } else if (newAuction != null) {
+      return newAuction.hasAnnotation(annotation);
+    } else {
+      return bid.hasAnnotation(annotation);
+    }
+  }
+
+  @Override
+  public long sizeInBytes() {
+    if (newPerson != null) {
+      return 1 + newPerson.sizeInBytes();
+    } else if (newAuction != null) {
+      return 1 + newAuction.sizeInBytes();
+    } else if (bid != null) {
+      return 1 + bid.sizeInBytes();
+    } else {
+      throw new RuntimeException("invalid event");
+    }
+  }
+
+  @Override
+  public String toString() {
+    if (newPerson != null) {
+      return newPerson.toString();
+    } else if (newAuction != null) {
+      return newAuction.toString();
+    } else if (bid != null) {
+      return bid.toString();
+    } else {
+      throw new RuntimeException("invalid event");
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/f4333df7/sdks/java/nexmark/src/main/java/org/apache/beam/sdk/nexmark/model/IdNameReserve.java
----------------------------------------------------------------------
diff --git a/sdks/java/nexmark/src/main/java/org/apache/beam/sdk/nexmark/model/IdNameReserve.java b/sdks/java/nexmark/src/main/java/org/apache/beam/sdk/nexmark/model/IdNameReserve.java
new file mode 100644
index 0000000..0519f5d
--- /dev/null
+++ b/sdks/java/nexmark/src/main/java/org/apache/beam/sdk/nexmark/model/IdNameReserve.java
@@ -0,0 +1,98 @@
+/*
+ * 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.beam.sdk.nexmark.model;
+
+import com.fasterxml.jackson.annotation.JsonProperty;
+import com.fasterxml.jackson.core.JsonProcessingException;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
+import java.io.Serializable;
+import org.apache.beam.sdk.coders.Coder;
+import org.apache.beam.sdk.coders.CoderException;
+import org.apache.beam.sdk.coders.CustomCoder;
+import org.apache.beam.sdk.coders.StringUtf8Coder;
+import org.apache.beam.sdk.coders.VarLongCoder;
+import org.apache.beam.sdk.nexmark.NexmarkUtils;
+
+/**
+ * Result type of Query8.
+ */
+public class IdNameReserve implements KnownSize, Serializable {
+  private static final Coder<Long> LONG_CODER = VarLongCoder.of();
+  private static final Coder<String> STRING_CODER = StringUtf8Coder.of();
+
+  public static final Coder<IdNameReserve> CODER = new CustomCoder<IdNameReserve>() {
+    @Override
+    public void encode(IdNameReserve value, OutputStream outStream)
+        throws CoderException, IOException {
+      LONG_CODER.encode(value.id, outStream);
+      STRING_CODER.encode(value.name, outStream);
+      LONG_CODER.encode(value.reserve, outStream);
+    }
+
+    @Override
+    public IdNameReserve decode(
+        InputStream inStream)
+        throws CoderException, IOException {
+      long id = LONG_CODER.decode(inStream);
+      String name = STRING_CODER.decode(inStream);
+      long reserve = LONG_CODER.decode(inStream);
+      return new IdNameReserve(id, name, reserve);
+    }
+    @Override public void verifyDeterministic() throws NonDeterministicException {}
+  };
+
+  @JsonProperty
+  private final long id;
+
+  @JsonProperty
+  private final String name;
+
+  /** Reserve price in cents. */
+  @JsonProperty
+  private final long reserve;
+
+  // For Avro only.
+  @SuppressWarnings("unused")
+  private IdNameReserve() {
+    id = 0;
+    name = null;
+    reserve = 0;
+  }
+
+  public IdNameReserve(long id, String name, long reserve) {
+    this.id = id;
+    this.name = name;
+    this.reserve = reserve;
+  }
+
+  @Override
+  public long sizeInBytes() {
+    return 8 + name.length() + 1 + 8;
+  }
+
+  @Override
+  public String toString() {
+    try {
+      return NexmarkUtils.MAPPER.writeValueAsString(this);
+    } catch (JsonProcessingException e) {
+      throw new RuntimeException(e);
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/f4333df7/sdks/java/nexmark/src/main/java/org/apache/beam/sdk/nexmark/model/KnownSize.java
----------------------------------------------------------------------
diff --git a/sdks/java/nexmark/src/main/java/org/apache/beam/sdk/nexmark/model/KnownSize.java b/sdks/java/nexmark/src/main/java/org/apache/beam/sdk/nexmark/model/KnownSize.java
new file mode 100644
index 0000000..45af3fc
--- /dev/null
+++ b/sdks/java/nexmark/src/main/java/org/apache/beam/sdk/nexmark/model/KnownSize.java
@@ -0,0 +1,26 @@
+/*
+ * 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.beam.sdk.nexmark.model;
+
+/**
+ * Interface for elements which can quickly estimate their encoded byte size.
+ */
+public interface KnownSize {
+  long sizeInBytes();
+}
+

http://git-wip-us.apache.org/repos/asf/beam/blob/f4333df7/sdks/java/nexmark/src/main/java/org/apache/beam/sdk/nexmark/model/NameCityStateId.java
----------------------------------------------------------------------
diff --git a/sdks/java/nexmark/src/main/java/org/apache/beam/sdk/nexmark/model/NameCityStateId.java b/sdks/java/nexmark/src/main/java/org/apache/beam/sdk/nexmark/model/NameCityStateId.java
new file mode 100644
index 0000000..55fca62
--- /dev/null
+++ b/sdks/java/nexmark/src/main/java/org/apache/beam/sdk/nexmark/model/NameCityStateId.java
@@ -0,0 +1,103 @@
+/*
+ * 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.beam.sdk.nexmark.model;
+
+import com.fasterxml.jackson.annotation.JsonProperty;
+import com.fasterxml.jackson.core.JsonProcessingException;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
+import java.io.Serializable;
+import org.apache.beam.sdk.coders.Coder;
+import org.apache.beam.sdk.coders.CoderException;
+import org.apache.beam.sdk.coders.CustomCoder;
+import org.apache.beam.sdk.coders.StringUtf8Coder;
+import org.apache.beam.sdk.coders.VarLongCoder;
+import org.apache.beam.sdk.nexmark.NexmarkUtils;
+
+/**
+ * Result of Query3.
+ */
+public class NameCityStateId implements KnownSize, Serializable {
+  private static final Coder<Long> LONG_CODER = VarLongCoder.of();
+  private static final Coder<String> STRING_CODER = StringUtf8Coder.of();
+
+  public static final Coder<NameCityStateId> CODER = new CustomCoder<NameCityStateId>() {
+    @Override
+    public void encode(NameCityStateId value, OutputStream outStream)
+        throws CoderException, IOException {
+      STRING_CODER.encode(value.name, outStream);
+      STRING_CODER.encode(value.city, outStream);
+      STRING_CODER.encode(value.state, outStream);
+      LONG_CODER.encode(value.id, outStream);
+    }
+
+    @Override
+    public NameCityStateId decode(InputStream inStream)
+        throws CoderException, IOException {
+      String name = STRING_CODER.decode(inStream);
+      String city = STRING_CODER.decode(inStream);
+      String state = STRING_CODER.decode(inStream);
+      long id = LONG_CODER.decode(inStream);
+      return new NameCityStateId(name, city, state, id);
+    }
+    @Override public void verifyDeterministic() throws NonDeterministicException {}
+  };
+
+  @JsonProperty
+  private final String name;
+
+  @JsonProperty
+  private final String city;
+
+  @JsonProperty
+  private final String state;
+
+  @JsonProperty
+  private final long id;
+
+  // For Avro only.
+  @SuppressWarnings("unused")
+  private NameCityStateId() {
+    name = null;
+    city = null;
+    state = null;
+    id = 0;
+  }
+
+  public NameCityStateId(String name, String city, String state, long id) {
+    this.name = name;
+    this.city = city;
+    this.state = state;
+    this.id = id;
+  }
+
+  @Override
+  public long sizeInBytes() {
+    return name.length() + 1 + city.length() + 1 + state.length() + 1 + 8;
+  }
+
+  @Override
+  public String toString() {
+    try {
+      return NexmarkUtils.MAPPER.writeValueAsString(this);
+    } catch (JsonProcessingException e) {
+      throw new RuntimeException(e);
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/f4333df7/sdks/java/nexmark/src/main/java/org/apache/beam/sdk/nexmark/model/Person.java
----------------------------------------------------------------------
diff --git a/sdks/java/nexmark/src/main/java/org/apache/beam/sdk/nexmark/model/Person.java b/sdks/java/nexmark/src/main/java/org/apache/beam/sdk/nexmark/model/Person.java
new file mode 100644
index 0000000..800f937
--- /dev/null
+++ b/sdks/java/nexmark/src/main/java/org/apache/beam/sdk/nexmark/model/Person.java
@@ -0,0 +1,163 @@
+/*
+ * 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.beam.sdk.nexmark.model;
+
+import com.fasterxml.jackson.annotation.JsonProperty;
+import com.fasterxml.jackson.core.JsonProcessingException;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
+import java.io.Serializable;
+import org.apache.beam.sdk.coders.Coder;
+import org.apache.beam.sdk.coders.CoderException;
+import org.apache.beam.sdk.coders.CustomCoder;
+import org.apache.beam.sdk.coders.StringUtf8Coder;
+import org.apache.beam.sdk.coders.VarLongCoder;
+import org.apache.beam.sdk.nexmark.NexmarkUtils;
+
+/**
+ * A person either creating an auction or making a bid.
+ */
+public class Person implements KnownSize, Serializable {
+  private static final Coder<Long> LONG_CODER = VarLongCoder.of();
+  private static final Coder<String> STRING_CODER = StringUtf8Coder.of();
+  public static final Coder<Person> CODER = new CustomCoder<Person>() {
+    @Override
+    public void encode(Person value, OutputStream outStream)
+        throws CoderException, IOException {
+      LONG_CODER.encode(value.id, outStream);
+      STRING_CODER.encode(value.name, outStream);
+      STRING_CODER.encode(value.emailAddress, outStream);
+      STRING_CODER.encode(value.creditCard, outStream);
+      STRING_CODER.encode(value.city, outStream);
+      STRING_CODER.encode(value.state, outStream);
+      LONG_CODER.encode(value.dateTime, outStream);
+      STRING_CODER.encode(value.extra, outStream);
+    }
+
+    @Override
+    public Person decode(InputStream inStream)
+        throws CoderException, IOException {
+      long id = LONG_CODER.decode(inStream);
+      String name = STRING_CODER.decode(inStream);
+      String emailAddress = STRING_CODER.decode(inStream);
+      String creditCard = STRING_CODER.decode(inStream);
+      String city = STRING_CODER.decode(inStream);
+      String state = STRING_CODER.decode(inStream);
+      long dateTime = LONG_CODER.decode(inStream);
+      String extra = STRING_CODER.decode(inStream);
+      return new Person(id, name, emailAddress, creditCard, city, state, dateTime, extra);
+    }
+    @Override public void verifyDeterministic() throws NonDeterministicException {}
+  };
+
+  /** Id of person. */
+  @JsonProperty
+  public final long id; // primary key
+
+  /** Extra person properties. */
+  @JsonProperty
+  public final String name;
+
+  @JsonProperty
+  private final String emailAddress;
+
+  @JsonProperty
+  private final String creditCard;
+
+  @JsonProperty
+  public final String city;
+
+  @JsonProperty
+  public final String state;
+
+  @JsonProperty
+  public final long dateTime;
+
+  /** Additional arbitrary payload for performance testing. */
+  @JsonProperty
+  private final String extra;
+
+  // For Avro only.
+  @SuppressWarnings("unused")
+  private Person() {
+    id = 0;
+    name = null;
+    emailAddress = null;
+    creditCard = null;
+    city = null;
+    state = null;
+    dateTime = 0;
+    extra = null;
+  }
+
+  public Person(long id, String name, String emailAddress, String creditCard, String city,
+      String state, long dateTime, String extra) {
+    this.id = id;
+    this.name = name;
+    this.emailAddress = emailAddress;
+    this.creditCard = creditCard;
+    this.city = city;
+    this.state = state;
+    this.dateTime = dateTime;
+    this.extra = extra;
+  }
+
+  /**
+   * Return a copy of person which capture the given annotation.
+   * (Used for debugging).
+   */
+  public Person withAnnotation(String annotation) {
+    return new Person(id, name, emailAddress, creditCard, city, state, dateTime,
+        annotation + ": " + extra);
+  }
+
+  /**
+   * Does person have {@code annotation}? (Used for debugging.)
+   */
+  public boolean hasAnnotation(String annotation) {
+    return extra.startsWith(annotation + ": ");
+  }
+
+  /**
+   * Remove {@code annotation} from person. (Used for debugging.)
+   */
+  public Person withoutAnnotation(String annotation) {
+    if (hasAnnotation(annotation)) {
+      return new Person(id, name, emailAddress, creditCard, city, state, dateTime,
+          extra.substring(annotation.length() + 2));
+    } else {
+      return this;
+    }
+  }
+
+  @Override
+  public long sizeInBytes() {
+    return 8 + name.length() + 1 + emailAddress.length() + 1 + creditCard.length() + 1
+        + city.length() + 1 + state.length() + 8 + 1 + extra.length() + 1;
+  }
+
+  @Override
+  public String toString() {
+    try {
+      return NexmarkUtils.MAPPER.writeValueAsString(this);
+    } catch (JsonProcessingException e) {
+      throw new RuntimeException(e);
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/f4333df7/sdks/java/nexmark/src/main/java/org/apache/beam/sdk/nexmark/model/SellerPrice.java
----------------------------------------------------------------------
diff --git a/sdks/java/nexmark/src/main/java/org/apache/beam/sdk/nexmark/model/SellerPrice.java b/sdks/java/nexmark/src/main/java/org/apache/beam/sdk/nexmark/model/SellerPrice.java
new file mode 100644
index 0000000..82b551c
--- /dev/null
+++ b/sdks/java/nexmark/src/main/java/org/apache/beam/sdk/nexmark/model/SellerPrice.java
@@ -0,0 +1,89 @@
+/*
+ * 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.beam.sdk.nexmark.model;
+
+import com.fasterxml.jackson.annotation.JsonProperty;
+import com.fasterxml.jackson.core.JsonProcessingException;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
+import java.io.Serializable;
+import org.apache.beam.sdk.coders.Coder;
+import org.apache.beam.sdk.coders.CoderException;
+import org.apache.beam.sdk.coders.CustomCoder;
+import org.apache.beam.sdk.coders.VarLongCoder;
+import org.apache.beam.sdk.nexmark.NexmarkUtils;
+
+/**
+ * Result of Query6.
+ */
+public class SellerPrice implements KnownSize, Serializable {
+  private static final Coder<Long> LONG_CODER = VarLongCoder.of();
+
+  public static final Coder<SellerPrice> CODER = new CustomCoder<SellerPrice>() {
+    @Override
+    public void encode(SellerPrice value, OutputStream outStream)
+        throws CoderException, IOException {
+      LONG_CODER.encode(value.seller, outStream);
+      LONG_CODER.encode(value.price, outStream);
+    }
+
+    @Override
+    public SellerPrice decode(
+        InputStream inStream)
+        throws CoderException, IOException {
+      long seller = LONG_CODER.decode(inStream);
+      long price = LONG_CODER.decode(inStream);
+      return new SellerPrice(seller, price);
+    }
+    @Override public void verifyDeterministic() throws NonDeterministicException {}
+  };
+
+  @JsonProperty
+  public final long seller;
+
+  /** Price in cents. */
+  @JsonProperty
+  private final long price;
+
+  // For Avro only.
+  @SuppressWarnings("unused")
+  private SellerPrice() {
+    seller = 0;
+    price = 0;
+  }
+
+  public SellerPrice(long seller, long price) {
+    this.seller = seller;
+    this.price = price;
+  }
+
+  @Override
+  public long sizeInBytes() {
+    return 8 + 8;
+  }
+
+  @Override
+  public String toString() {
+    try {
+      return NexmarkUtils.MAPPER.writeValueAsString(this);
+    } catch (JsonProcessingException e) {
+      throw new RuntimeException(e);
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/f4333df7/sdks/java/nexmark/src/main/java/org/apache/beam/sdk/nexmark/model/package-info.java
----------------------------------------------------------------------
diff --git a/sdks/java/nexmark/src/main/java/org/apache/beam/sdk/nexmark/model/package-info.java b/sdks/java/nexmark/src/main/java/org/apache/beam/sdk/nexmark/model/package-info.java
new file mode 100644
index 0000000..3b4bb63
--- /dev/null
+++ b/sdks/java/nexmark/src/main/java/org/apache/beam/sdk/nexmark/model/package-info.java
@@ -0,0 +1,22 @@
+/*
+ * 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.
+ */
+
+/**
+ * Nexmark Benchmark Model.
+ */
+package org.apache.beam.sdk.nexmark.model;

http://git-wip-us.apache.org/repos/asf/beam/blob/f4333df7/sdks/java/nexmark/src/main/java/org/apache/beam/sdk/nexmark/package-info.java
----------------------------------------------------------------------
diff --git a/sdks/java/nexmark/src/main/java/org/apache/beam/sdk/nexmark/package-info.java b/sdks/java/nexmark/src/main/java/org/apache/beam/sdk/nexmark/package-info.java
new file mode 100644
index 0000000..7500a24
--- /dev/null
+++ b/sdks/java/nexmark/src/main/java/org/apache/beam/sdk/nexmark/package-info.java
@@ -0,0 +1,21 @@
+/*
+ * 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.
+ */
+/**
+ * Nexmark.
+ */
+package org.apache.beam.sdk.nexmark;


[02/55] [abbrv] beam git commit: NexMark

Posted by ie...@apache.org.
http://git-wip-us.apache.org/repos/asf/beam/blob/1f08970a/integration/java/src/main/java/org/apache/beam/integration/nexmark/Query2.java
----------------------------------------------------------------------
diff --git a/integration/java/src/main/java/org/apache/beam/integration/nexmark/Query2.java b/integration/java/src/main/java/org/apache/beam/integration/nexmark/Query2.java
new file mode 100644
index 0000000..cede2f3
--- /dev/null
+++ b/integration/java/src/main/java/org/apache/beam/integration/nexmark/Query2.java
@@ -0,0 +1,75 @@
+/*
+ * 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.beam.integration.nexmark;
+
+import org.apache.beam.sdk.transforms.DoFn;
+import org.apache.beam.sdk.transforms.Filter;
+import org.apache.beam.sdk.transforms.ParDo;
+import org.apache.beam.sdk.transforms.SerializableFunction;
+import org.apache.beam.sdk.values.PCollection;
+
+/**
+ * Query 2, 'Filtering. Find bids with specific auction ids and show their bid price.
+ * In CQL syntax:
+ *
+ * <pre>
+ * SELECT Rstream(auction, price)
+ * FROM Bid [NOW]
+ * WHERE auction = 1007 OR auction = 1020 OR auction = 2001 OR auction = 2019 OR auction = 2087;
+ * </pre>
+ *
+ * <p>As written that query will only yield a few hundred results over event streams of
+ * arbitrary size. To make it more interesting we instead choose bids for every
+ * {@code auctionSkip}'th auction.
+ */
+class Query2 extends NexmarkQuery {
+  public Query2(NexmarkConfiguration configuration) {
+    super(configuration, "Query2");
+  }
+
+  private PCollection<AuctionPrice> applyTyped(PCollection<Event> events) {
+    return events
+        // Only want the bid events.
+        .apply(JUST_BIDS)
+
+        // Select just the bids for the auctions we care about.
+        .apply(Filter.byPredicate(new SerializableFunction<Bid, Boolean>() {
+          @Override
+          public Boolean apply(Bid bid) {
+            return bid.auction % configuration.auctionSkip == 0;
+          }
+        }))
+
+        // Project just auction id and price.
+        .apply(
+            ParDo.named(name + ".Project")
+                .of(new DoFn<Bid, AuctionPrice>() {
+                  @Override
+                  public void processElement(ProcessContext c) {
+                    Bid bid = c.element();
+                    c.output(new AuctionPrice(bid.auction, bid.price));
+                  }
+                }));
+  }
+
+  @Override
+  protected PCollection<KnownSize> applyPrim(PCollection<Event> events) {
+    return NexmarkUtils.castToKnownSize(name, applyTyped(events));
+  }
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/1f08970a/integration/java/src/main/java/org/apache/beam/integration/nexmark/Query2Model.java
----------------------------------------------------------------------
diff --git a/integration/java/src/main/java/org/apache/beam/integration/nexmark/Query2Model.java b/integration/java/src/main/java/org/apache/beam/integration/nexmark/Query2Model.java
new file mode 100644
index 0000000..6ccfeeb
--- /dev/null
+++ b/integration/java/src/main/java/org/apache/beam/integration/nexmark/Query2Model.java
@@ -0,0 +1,76 @@
+/*
+ * 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.beam.integration.nexmark;
+
+import org.apache.beam.sdk.values.TimestampedValue;
+
+import java.io.Serializable;
+import java.util.Collection;
+import java.util.Iterator;
+
+/**
+ * A direct implementation of {@link Query2}.
+ */
+public class Query2Model extends NexmarkQueryModel implements Serializable {
+  /**
+   * Simulator for query 2.
+   */
+  private class Simulator extends AbstractSimulator<Event, AuctionPrice> {
+    public Simulator(NexmarkConfiguration configuration) {
+      super(NexmarkUtils.standardEventIterator(configuration));
+    }
+
+    @Override
+    protected void run() {
+      TimestampedValue<Event> timestampedEvent = nextInput();
+      if (timestampedEvent == null) {
+        allDone();
+        return;
+      }
+      Event event = timestampedEvent.getValue();
+      if (event.bid == null) {
+        // Ignore non bid events.
+        return;
+      }
+      Bid bid = event.bid;
+      if (bid.auction % configuration.auctionSkip != 0) {
+        // Ignore bids for auctions we don't care about.
+        return;
+      }
+      AuctionPrice auctionPrice = new AuctionPrice(bid.auction, bid.price);
+      TimestampedValue<AuctionPrice> result =
+          TimestampedValue.of(auctionPrice, timestampedEvent.getTimestamp());
+      addResult(result);
+    }
+  }
+
+  public Query2Model(NexmarkConfiguration configuration) {
+    super(configuration);
+  }
+
+  @Override
+  public AbstractSimulator<?, ?> simulator() {
+    return new Simulator(configuration);
+  }
+
+  @Override
+  protected <T> Collection<String> toCollection(Iterator<TimestampedValue<T>> itr) {
+    return toValueTimestampOrder(itr);
+  }
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/1f08970a/integration/java/src/main/java/org/apache/beam/integration/nexmark/Query3.java
----------------------------------------------------------------------
diff --git a/integration/java/src/main/java/org/apache/beam/integration/nexmark/Query3.java b/integration/java/src/main/java/org/apache/beam/integration/nexmark/Query3.java
new file mode 100644
index 0000000..5b9b17b
--- /dev/null
+++ b/integration/java/src/main/java/org/apache/beam/integration/nexmark/Query3.java
@@ -0,0 +1,248 @@
+/*
+ * 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.beam.integration.nexmark;
+
+import org.apache.beam.sdk.coders.ListCoder;
+import org.apache.beam.sdk.transforms.Aggregator;
+import org.apache.beam.sdk.transforms.DoFn;
+import org.apache.beam.sdk.transforms.Filter;
+import org.apache.beam.sdk.transforms.ParDo;
+import org.apache.beam.sdk.transforms.SerializableFunction;
+import org.apache.beam.sdk.transforms.Sum.SumLongFn;
+import org.apache.beam.sdk.transforms.join.CoGbkResult;
+import org.apache.beam.sdk.transforms.join.CoGroupByKey;
+import org.apache.beam.sdk.transforms.join.KeyedPCollectionTuple;
+import org.apache.beam.sdk.transforms.windowing.FixedWindows;
+import org.apache.beam.sdk.transforms.windowing.Window;
+import org.apache.beam.sdk.util.state.StateInternals;
+import org.apache.beam.sdk.util.state.StateNamespace;
+import org.apache.beam.sdk.util.state.StateNamespaces;
+import org.apache.beam.sdk.util.state.StateTag;
+import org.apache.beam.sdk.util.state.StateTags;
+import org.apache.beam.sdk.util.state.ValueState;
+import org.apache.beam.sdk.values.KV;
+import org.apache.beam.sdk.values.PCollection;
+
+import org.joda.time.Duration;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+
+import javax.annotation.Nullable;
+
+/**
+ * Query 3, 'Local Item Suggestion'. Who is selling in OR, ID or CA in category 10, and for what
+ * auction ids? In CQL syntax:
+ *
+ * <pre>
+ * SELECT Istream(P.name, P.city, P.state, A.id)
+ * FROM Auction A [ROWS UNBOUNDED], Person P [ROWS UNBOUNDED]
+ * WHERE A.seller = P.id AND (P.state = `OR' OR P.state = `ID' OR P.state = `CA') AND A.category
+ * = 10;
+ * </pre>
+ *
+ * <p>We'll implement this query to allow 'new auction' events to come before the 'new person'
+ * events for the auction seller. Those auctions will be stored until the matching person is
+ * seen. Then all subsequent auctions for a person will use the stored person record.
+ *
+ * <p>A real system would use an external system to maintain the id-to-person association.
+ */
+class Query3 extends NexmarkQuery {
+  private static final StateNamespace GLOBAL_NAMESPACE = StateNamespaces.global();
+  private static final StateTag<Object, ValueState<List<Auction>>> AUCTION_LIST_CODED_TAG =
+      StateTags.value("left", ListCoder.of(Auction.CODER));
+  private static final StateTag<Object, ValueState<Person>> PERSON_CODED_TAG =
+      StateTags.value("right", Person.CODER);
+
+  /**
+   * Join {@code auctions} and {@code people} by person id and emit their cross-product one pair
+   * at a time.
+   *
+   * <p>We know a person may submit any number of auctions. Thus new person event must have the
+   * person record stored in persistent state in order to match future auctions by that person.
+   *
+   * <p>However we know that each auction is associated with at most one person, so only need
+   * to store auction records in persistent state until we have seen the corresponding person
+   * record. And of course may have already seen that record.
+   */
+  private static class JoinDoFn extends DoFn<KV<Long, CoGbkResult>, KV<Auction, Person>> {
+    private final Aggregator<Long, Long> newAuctionCounter =
+        createAggregator("newAuction", new SumLongFn());
+    private final Aggregator<Long, Long> newPersonCounter =
+        createAggregator("newPerson", new SumLongFn());
+    private final Aggregator<Long, Long> newNewOutputCounter =
+        createAggregator("newNewOutput", new SumLongFn());
+    private final Aggregator<Long, Long> newOldOutputCounter =
+        createAggregator("newOldOutput", new SumLongFn());
+    private final Aggregator<Long, Long> oldNewOutputCounter =
+        createAggregator("oldNewOutput", new SumLongFn());
+    public final Aggregator<Long, Long> fatalCounter = createAggregator("fatal", new SumLongFn());
+
+    @Override
+    public void processElement(ProcessContext c) throws IOException {
+      // TODO: This is using the internal state API. Rework to use the
+      // We would *almost* implement this by  rewindowing into the global window and
+      // running a combiner over the result. The combiner's accumulator would be the
+      // state we use below. However, combiners cannot emit intermediate results, thus
+      // we need to wait for the pending ReduceFn API.
+      StateInternals<?> stateInternals = c.windowingInternals().stateInternals();
+      ValueState<Person> personState = stateInternals.state(GLOBAL_NAMESPACE, PERSON_CODED_TAG);
+      Person existingPerson = personState.read();
+      if (existingPerson != null) {
+        // We've already seen the new person event for this person id.
+        // We can join with any new auctions on-the-fly without needing any
+        // additional persistent state.
+        for (Auction newAuction : c.element().getValue().getAll(AUCTION_TAG)) {
+          newAuctionCounter.addValue(1L);
+          newOldOutputCounter.addValue(1L);
+          c.output(KV.of(newAuction, existingPerson));
+        }
+        return;
+      }
+
+      ValueState<List<Auction>> auctionsState =
+          stateInternals.state(GLOBAL_NAMESPACE, AUCTION_LIST_CODED_TAG);
+      Person theNewPerson = null;
+      for (Person newPerson : c.element().getValue().getAll(PERSON_TAG)) {
+        if (theNewPerson == null) {
+          theNewPerson = newPerson;
+        } else {
+          if (theNewPerson.equals(newPerson)) {
+            NexmarkUtils.error("**** duplicate person %s ****", theNewPerson);
+          } else {
+            NexmarkUtils.error("**** conflicting persons %s and %s ****", theNewPerson, newPerson);
+          }
+          fatalCounter.addValue(1L);
+          continue;
+        }
+        newPersonCounter.addValue(1L);
+        // We've now seen the person for this person id so can flush any
+        // pending auctions for the same seller id.
+        List<Auction> pendingAuctions = auctionsState.read();
+        if (pendingAuctions != null) {
+          for (Auction pendingAuction : pendingAuctions) {
+            oldNewOutputCounter.addValue(1L);
+            c.output(KV.of(pendingAuction, newPerson));
+          }
+          auctionsState.clear();
+        }
+        // Also deal with any new auctions.
+        for (Auction newAuction : c.element().getValue().getAll(AUCTION_TAG)) {
+          newAuctionCounter.addValue(1L);
+          newNewOutputCounter.addValue(1L);
+          c.output(KV.of(newAuction, newPerson));
+        }
+        // Remember this person for any future auctions.
+        personState.write(newPerson);
+      }
+      if (theNewPerson != null) {
+        return;
+      }
+
+      // We'll need to remember the auctions until we see the corresponding
+      // new person event.
+      List<Auction> pendingAuctions = auctionsState.read();
+      if (pendingAuctions == null) {
+        pendingAuctions = new ArrayList<>();
+      }
+      for (Auction newAuction : c.element().getValue().getAll(AUCTION_TAG)) {
+        newAuctionCounter.addValue(1L);
+        pendingAuctions.add(newAuction);
+      }
+      auctionsState.write(pendingAuctions);
+    }
+  }
+
+  private final JoinDoFn joinDoFn = new JoinDoFn();
+
+  public Query3(NexmarkConfiguration configuration) {
+    super(configuration, "Query3");
+  }
+
+  @Override
+  @Nullable
+  public Aggregator<Long, Long> getFatalCount() {
+    return joinDoFn.fatalCounter;
+  }
+
+  private PCollection<NameCityStateId> applyTyped(PCollection<Event> events) {
+    // Batch into incremental results windows.
+    events = events.apply(
+        Window.<Event>into(FixedWindows.of(Duration.standardSeconds(configuration.windowSizeSec))));
+
+    PCollection<KV<Long, Auction>> auctionsBySellerId =
+        events
+            // Only want the new auction events.
+            .apply(JUST_NEW_AUCTIONS)
+
+            // We only want auctions in category 10.
+            .apply(Filter.byPredicate(new SerializableFunction<Auction, Boolean>() {
+              @Override
+              public Boolean apply(Auction auction) {
+                return auction.category == 10;
+              }
+            }).named(name + ".InCategory"))
+
+            // Key auctions by their seller id.
+            .apply(AUCTION_BY_SELLER);
+
+    PCollection<KV<Long, Person>> personsById =
+        events
+            // Only want the new people events.
+            .apply(JUST_NEW_PERSONS)
+
+            // We only want people in OR, ID, CA.
+            .apply(Filter.byPredicate(new SerializableFunction<Person, Boolean>() {
+              @Override
+              public Boolean apply(Person person) {
+                return person.state.equals("OR") || person.state.equals("ID")
+                    || person.state.equals("CA");
+              }
+            }).named(name + ".InState"))
+
+            // Key people by their id.
+            .apply(PERSON_BY_ID);
+
+    return
+        // Join auctions and people.
+        KeyedPCollectionTuple.of(AUCTION_TAG, auctionsBySellerId)
+            .and(PERSON_TAG, personsById)
+            .apply(CoGroupByKey.<Long>create())
+            .apply(ParDo.named(name + ".Join").of(joinDoFn))
+
+            // Project what we want.
+            .apply(
+                ParDo.named(name + ".Project")
+                    .of(new DoFn<KV<Auction, Person>, NameCityStateId>() {
+                      @Override
+                      public void processElement(ProcessContext c) {
+                        Auction auction = c.element().getKey();
+                        Person person = c.element().getValue();
+                        c.output(new NameCityStateId(
+                            person.name, person.city, person.state, auction.id));
+                      }
+                    }));
+  }
+
+  @Override
+  protected PCollection<KnownSize> applyPrim(PCollection<Event> events) {
+    return NexmarkUtils.castToKnownSize(name, applyTyped(events));
+  }
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/1f08970a/integration/java/src/main/java/org/apache/beam/integration/nexmark/Query3Model.java
----------------------------------------------------------------------
diff --git a/integration/java/src/main/java/org/apache/beam/integration/nexmark/Query3Model.java b/integration/java/src/main/java/org/apache/beam/integration/nexmark/Query3Model.java
new file mode 100644
index 0000000..b865eda
--- /dev/null
+++ b/integration/java/src/main/java/org/apache/beam/integration/nexmark/Query3Model.java
@@ -0,0 +1,119 @@
+/*
+ * 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.beam.integration.nexmark;
+
+import org.apache.beam.sdk.values.TimestampedValue;
+import com.google.common.collect.ArrayListMultimap;
+import com.google.common.collect.Multimap;
+
+import org.joda.time.Instant;
+
+import java.io.Serializable;
+import java.util.Collection;
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.Map;
+
+/**
+ * A direct implementation of {@link Query3}.
+ */
+public class Query3Model extends NexmarkQueryModel implements Serializable {
+  /**
+   * Simulator for query 3.
+   */
+  private class Simulator extends AbstractSimulator<Event, NameCityStateId> {
+    /** Auctions, indexed by seller id. */
+    private final Multimap<Long, Auction> newAuctions;
+
+    /** Persons, indexed by id. */
+    private final Map<Long, Person> newPersons;
+
+    public Simulator(NexmarkConfiguration configuration) {
+      super(NexmarkUtils.standardEventIterator(configuration));
+      newPersons = new HashMap<>();
+      newAuctions = ArrayListMultimap.create();
+    }
+
+    /**
+     * Capture new result.
+     */
+    private void addResult(Auction auction, Person person, Instant timestamp) {
+      TimestampedValue<NameCityStateId> result = TimestampedValue.of(
+          new NameCityStateId(person.name, person.city, person.state, auction.id), timestamp);
+      addResult(result);
+    }
+
+    @Override
+    protected void run() {
+      TimestampedValue<Event> timestampedEvent = nextInput();
+      if (timestampedEvent == null) {
+        allDone();
+        return;
+      }
+      Event event = timestampedEvent.getValue();
+      if (event.bid != null) {
+        // Ignore bid events.
+        return;
+      }
+
+      Instant timestamp = timestampedEvent.getTimestamp();
+
+      if (event.newAuction != null) {
+        // Only want auctions in category 10.
+        if (event.newAuction.category == 10) {
+          // Join new auction with existing person, if any.
+          Person person = newPersons.get(event.newAuction.seller);
+          if (person != null) {
+            addResult(event.newAuction, person, timestamp);
+          } else {
+            // Remember auction for future new person event.
+            newAuctions.put(event.newAuction.seller, event.newAuction);
+          }
+        }
+      } else {
+        // Only want people in OR, ID or CA.
+        if (event.newPerson.state.equals("OR") || event.newPerson.state.equals("ID")
+            || event.newPerson.state.equals("CA")) {
+          // Join new person with existing auctions.
+          for (Auction auction : newAuctions.get(event.newPerson.id)) {
+            addResult(auction, event.newPerson, timestamp);
+          }
+          // We'll never need these auctions again.
+          newAuctions.removeAll(event.newPerson.id);
+          // Remember person for future auctions.
+          newPersons.put(event.newPerson.id, event.newPerson);
+        }
+      }
+    }
+  }
+
+  public Query3Model(NexmarkConfiguration configuration) {
+    super(configuration);
+  }
+
+  @Override
+  public AbstractSimulator<?, ?> simulator() {
+    return new Simulator(configuration);
+  }
+
+  @Override
+  protected <T> Collection<String> toCollection(Iterator<TimestampedValue<T>> itr) {
+    return toValue(itr);
+  }
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/1f08970a/integration/java/src/main/java/org/apache/beam/integration/nexmark/Query4.java
----------------------------------------------------------------------
diff --git a/integration/java/src/main/java/org/apache/beam/integration/nexmark/Query4.java b/integration/java/src/main/java/org/apache/beam/integration/nexmark/Query4.java
new file mode 100644
index 0000000..bc695b7
--- /dev/null
+++ b/integration/java/src/main/java/org/apache/beam/integration/nexmark/Query4.java
@@ -0,0 +1,110 @@
+/*
+ * 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.beam.integration.nexmark;
+
+import org.apache.beam.sdk.transforms.DoFn;
+import org.apache.beam.sdk.transforms.Mean;
+import org.apache.beam.sdk.transforms.ParDo;
+import org.apache.beam.sdk.transforms.windowing.SlidingWindows;
+import org.apache.beam.sdk.transforms.windowing.Window;
+import org.apache.beam.sdk.values.KV;
+import org.apache.beam.sdk.values.PCollection;
+
+import org.joda.time.Duration;
+
+/**
+ * Query 4, 'Average Price for a Category'. Select the average of the wining bid prices for all
+ * closed auctions in each category. In CQL syntax:
+ *
+ * <pre>
+ * SELECT Istream(AVG(Q.final))
+ * FROM Category C, (SELECT Rstream(MAX(B.price) AS final, A.category)
+ *                   FROM Auction A [ROWS UNBOUNDED], Bid B [ROWS UNBOUNDED]
+ *                   WHERE A.id=B.auction AND B.datetime < A.expires AND A.expires < CURRENT_TIME
+ *                   GROUP BY A.id, A.category) Q
+ * WHERE Q.category = C.id
+ * GROUP BY C.id;
+ * </pre>
+ *
+ * <p>For extra spiciness our implementation differs slightly from the above:
+ * <ul>
+ * <li>We select both the average winning price and the category.
+ * <li>We don't bother joining with a static category table, since it's contents are never used.
+ * <li>We only consider bids which are above the auction's reserve price.
+ * <li>We accept the highest-price, earliest valid bid as the winner.
+ * <li>We calculate the averages oven a sliding window of size {@code windowSizeSec} and
+ * period {@code windowPeriodSec}.
+ * </ul>
+ */
+class Query4 extends NexmarkQuery {
+  private final Monitor<AuctionBid> winningBidsMonitor;
+
+  public Query4(NexmarkConfiguration configuration) {
+    super(configuration, "Query4");
+    winningBidsMonitor = new Monitor<>(name + ".WinningBids", "winning");
+  }
+
+  private PCollection<CategoryPrice> applyTyped(PCollection<Event> events) {
+    PCollection<AuctionBid> winningBids =
+        events
+            // Find the winning bid for each closed auction.
+            .apply(new WinningBids(name + ".WinningBids", configuration));
+
+    // Monitor winning bids
+    winningBids = winningBids.apply(winningBidsMonitor.getTransform());
+
+    return winningBids
+        // Key the winning bid price by the auction category.
+        .apply(
+            ParDo.named(name + ".Rekey")
+                .of(new DoFn<AuctionBid, KV<Long, Long>>() {
+                  @Override
+                  public void processElement(ProcessContext c) {
+                    Auction auction = c.element().auction;
+                    Bid bid = c.element().bid;
+                    c.output(KV.of(auction.category, bid.price));
+                  }
+                }))
+
+        // Re-window so we can calculate a sliding average
+        .apply(Window.<KV<Long, Long>>into(
+            SlidingWindows.of(Duration.standardSeconds(configuration.windowSizeSec))
+                .every(Duration.standardSeconds(configuration.windowPeriodSec))))
+
+        // Find the average of the winning bids for each category.
+        // Make sure we share the work for each category between workers.
+        .apply(Mean.<Long, Long>perKey().withHotKeyFanout(configuration.fanout))
+
+        // For testing against Query4Model, capture which results are 'final'.
+        .apply(
+            ParDo.named(name + ".Project")
+                .of(new DoFn<KV<Long, Double>, CategoryPrice>() {
+                  @Override
+                  public void processElement(ProcessContext c) {
+                    c.output(new CategoryPrice(c.element().getKey(),
+                        Math.round(c.element().getValue()), c.pane().isLast()));
+                  }
+                }));
+  }
+
+  @Override
+  protected PCollection<KnownSize> applyPrim(PCollection<Event> events) {
+    return NexmarkUtils.castToKnownSize(name, applyTyped(events));
+  }
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/1f08970a/integration/java/src/main/java/org/apache/beam/integration/nexmark/Query4Model.java
----------------------------------------------------------------------
diff --git a/integration/java/src/main/java/org/apache/beam/integration/nexmark/Query4Model.java b/integration/java/src/main/java/org/apache/beam/integration/nexmark/Query4Model.java
new file mode 100644
index 0000000..2410306
--- /dev/null
+++ b/integration/java/src/main/java/org/apache/beam/integration/nexmark/Query4Model.java
@@ -0,0 +1,181 @@
+/*
+ * 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.beam.integration.nexmark;
+
+import org.apache.beam.sdk.transforms.windowing.BoundedWindow;
+import org.apache.beam.sdk.values.TimestampedValue;
+
+import org.joda.time.Duration;
+import org.joda.time.Instant;
+import org.junit.Assert;
+
+import java.io.Serializable;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.TreeMap;
+
+/**
+ * A direct implementation of {@link Query4}.
+ */
+public class Query4Model extends NexmarkQueryModel implements Serializable {
+  /**
+   * Simulator for query 4.
+   */
+  private class Simulator extends AbstractSimulator<AuctionBid, CategoryPrice> {
+    /** The prices and categories for all winning bids in the last window size. */
+    private final List<TimestampedValue<CategoryPrice>> winningPricesByCategory;
+
+    /** Timestamp of last result (ms since epoch). */
+    private Instant lastTimestamp;
+
+    /** When oldest active window starts. */
+    private Instant windowStart;
+
+    /** The last seen result for each category. */
+    private final Map<Long, TimestampedValue<CategoryPrice>> lastSeenResults;
+
+    public Simulator(NexmarkConfiguration configuration) {
+      super(new WinningBidsSimulator(configuration).results());
+      winningPricesByCategory = new ArrayList<>();
+      lastTimestamp = BoundedWindow.TIMESTAMP_MIN_VALUE;
+      windowStart = NexmarkUtils.BEGINNING_OF_TIME;
+      lastSeenResults = new TreeMap<>();
+    }
+
+    /**
+     * Calculate the average bid price for each category for all winning bids
+     * which are strictly before {@code end}.
+     */
+    private void averages(Instant end) {
+      Map<Long, Long> counts = new TreeMap<>();
+      Map<Long, Long> totals = new TreeMap<>();
+      for (TimestampedValue<CategoryPrice> value : winningPricesByCategory) {
+        if (!value.getTimestamp().isBefore(end)) {
+          continue;
+        }
+        long category = value.getValue().category;
+        long price = value.getValue().price;
+        Long count = counts.get(category);
+        if (count == null) {
+          count = 1L;
+        } else {
+          count += 1;
+        }
+        counts.put(category, count);
+        Long total = totals.get(category);
+        if (total == null) {
+          total = price;
+        } else {
+          total += price;
+        }
+        totals.put(category, total);
+      }
+      for (long category : counts.keySet()) {
+        long count = counts.get(category);
+        long total = totals.get(category);
+        TimestampedValue<CategoryPrice> result = TimestampedValue.of(
+            new CategoryPrice(category, Math.round((double) total / count), true), lastTimestamp);
+        addIntermediateResult(result);
+        lastSeenResults.put(category, result);
+      }
+    }
+
+    /**
+     * Calculate averages for any windows which can now be retired. Also prune entries
+     * which can no longer contribute to any future window.
+     */
+    private void prune(Instant newWindowStart) {
+      while (!newWindowStart.equals(windowStart)) {
+        averages(windowStart.plus(Duration.standardSeconds(configuration.windowSizeSec)));
+        windowStart = windowStart.plus(Duration.standardSeconds(configuration.windowPeriodSec));
+        Iterator<TimestampedValue<CategoryPrice>> itr = winningPricesByCategory.iterator();
+        while (itr.hasNext()) {
+          if (itr.next().getTimestamp().isBefore(windowStart)) {
+            itr.remove();
+          }
+        }
+        if (winningPricesByCategory.isEmpty()) {
+          windowStart = newWindowStart;
+        }
+      }
+    }
+
+    /**
+     * Capture the winning bid.
+     */
+    private void captureWinningBid(Auction auction, Bid bid, Instant timestamp) {
+      winningPricesByCategory.add(
+          TimestampedValue.of(new CategoryPrice(auction.category, bid.price, false), timestamp));
+    }
+
+    @Override
+    protected void run() {
+      TimestampedValue<AuctionBid> timestampedWinningBid = nextInput();
+      if (timestampedWinningBid == null) {
+        prune(NexmarkUtils.END_OF_TIME);
+        for (TimestampedValue<CategoryPrice> result : lastSeenResults.values()) {
+          addResult(result);
+        }
+        allDone();
+        return;
+      }
+      lastTimestamp = timestampedWinningBid.getTimestamp();
+      Instant newWindowStart = windowStart(Duration.standardSeconds(configuration.windowSizeSec),
+          Duration.standardSeconds(configuration.windowPeriodSec), lastTimestamp);
+      prune(newWindowStart);
+      captureWinningBid(timestampedWinningBid.getValue().auction,
+          timestampedWinningBid.getValue().bid, lastTimestamp);
+    }
+  }
+
+  public Query4Model(NexmarkConfiguration configuration) {
+    super(configuration);
+  }
+
+  @Override
+  public AbstractSimulator<?, ?> simulator() {
+    return new Simulator(configuration);
+  }
+
+  @Override
+  protected Iterable<TimestampedValue<KnownSize>> relevantResults(
+      Iterable<TimestampedValue<KnownSize>> results) {
+    // Find the last (in processing time) reported average price for each category.
+    Map<Long, TimestampedValue<KnownSize>> finalAverages = new TreeMap<>();
+    for (TimestampedValue<KnownSize> obj : results) {
+      Assert.assertTrue("have CategoryPrice", obj.getValue() instanceof CategoryPrice);
+      CategoryPrice categoryPrice = (CategoryPrice) obj.getValue();
+      if (categoryPrice.isLast) {
+        finalAverages.put(
+            categoryPrice.category,
+            TimestampedValue.of((KnownSize) categoryPrice, obj.getTimestamp()));
+      }
+    }
+
+    return finalAverages.values();
+  }
+
+  @Override
+  protected <T> Collection<String> toCollection(Iterator<TimestampedValue<T>> itr) {
+    return toValue(itr);
+  }
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/1f08970a/integration/java/src/main/java/org/apache/beam/integration/nexmark/Query5.java
----------------------------------------------------------------------
diff --git a/integration/java/src/main/java/org/apache/beam/integration/nexmark/Query5.java b/integration/java/src/main/java/org/apache/beam/integration/nexmark/Query5.java
new file mode 100644
index 0000000..91a4a28
--- /dev/null
+++ b/integration/java/src/main/java/org/apache/beam/integration/nexmark/Query5.java
@@ -0,0 +1,127 @@
+/*
+ * 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.beam.integration.nexmark;
+
+import org.apache.beam.sdk.transforms.Combine;
+import org.apache.beam.sdk.transforms.Count;
+import org.apache.beam.sdk.transforms.DoFn;
+import org.apache.beam.sdk.transforms.ParDo;
+import org.apache.beam.sdk.transforms.windowing.SlidingWindows;
+import org.apache.beam.sdk.transforms.windowing.Window;
+import org.apache.beam.sdk.values.KV;
+import org.apache.beam.sdk.values.PCollection;
+
+import org.joda.time.Duration;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.List;
+
+/**
+ * Query 5, 'Hot Items'. Which auctions have seen the most bids in the last hour (updated every
+ * minute). In CQL syntax:
+ *
+ * <pre>
+ * SELECT Rstream(auction)
+ * FROM (SELECT B1.auction, count(*) AS num
+ *       FROM Bid [RANGE 60 MINUTE SLIDE 1 MINUTE] B1
+ *       GROUP BY B1.auction)
+ * WHERE num >= ALL (SELECT count(*)
+ *                   FROM Bid [RANGE 60 MINUTE SLIDE 1 MINUTE] B2
+ *                   GROUP BY B2.auction);
+ * </pre>
+ *
+ * <p>To make things a bit more dynamic and easier to test we use much shorter windows, and
+ * we'll also preserve the bid counts.
+ */
+class Query5 extends NexmarkQuery {
+  public Query5(NexmarkConfiguration configuration) {
+    super(configuration, "Query5");
+  }
+
+  private PCollection<AuctionCount> applyTyped(PCollection<Event> events) {
+    return events
+        // Only want the bid events.
+        .apply(JUST_BIDS)
+        // Window the bids into sliding windows.
+        .apply(Window.<Bid>into(
+            SlidingWindows.of(Duration.standardSeconds(configuration.windowSizeSec))
+                .every(Duration.standardSeconds(configuration.windowPeriodSec))))
+        // Project just the auction id.
+        .apply(BID_TO_AUCTION)
+
+        // Count the number of bids per auction id.
+        .apply(Count.<Long>perElement())
+
+        // We'll want to keep all auctions with the maximal number of bids.
+        // Start by lifting each into a singleton list.
+        .apply(
+            ParDo.named(name + ".ToSingletons")
+                .of(new DoFn<KV<Long, Long>, KV<List<Long>, Long>>() {
+                  @Override
+                  public void processElement(ProcessContext c) {
+                    c.output(KV.of(Arrays.asList(c.element().getKey()), c.element().getValue()));
+                  }
+                }))
+
+        // Keep only the auction ids with the most bids.
+        .apply(
+            Combine
+                .globally(new Combine.BinaryCombineFn<KV<List<Long>, Long>>() {
+                  @Override
+                  public KV<List<Long>, Long> apply(
+                      KV<List<Long>, Long> left, KV<List<Long>, Long> right) {
+                    List<Long> leftBestAuctions = left.getKey();
+                    long leftCount = left.getValue();
+                    List<Long> rightBestAuctions = right.getKey();
+                    long rightCount = right.getValue();
+                    if (leftCount > rightCount) {
+                      return left;
+                    } else if (leftCount < rightCount) {
+                      return right;
+                    } else {
+                      List<Long> newBestAuctions = new ArrayList<>();
+                      newBestAuctions.addAll(leftBestAuctions);
+                      newBestAuctions.addAll(rightBestAuctions);
+                      return KV.of(newBestAuctions, leftCount);
+                    }
+                  }
+                })
+                .withoutDefaults()
+                .withFanout(configuration.fanout))
+
+        // Project into result.
+        .apply(
+            ParDo.named(name + ".Select")
+                .of(new DoFn<KV<List<Long>, Long>, AuctionCount>() {
+                  @Override
+                  public void processElement(ProcessContext c) {
+                    long count = c.element().getValue();
+                    for (long auction : c.element().getKey()) {
+                      c.output(new AuctionCount(auction, count));
+                    }
+                  }
+                }));
+  }
+
+  @Override
+  protected PCollection<KnownSize> applyPrim(PCollection<Event> events) {
+    return NexmarkUtils.castToKnownSize(name, applyTyped(events));
+  }
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/1f08970a/integration/java/src/main/java/org/apache/beam/integration/nexmark/Query5Model.java
----------------------------------------------------------------------
diff --git a/integration/java/src/main/java/org/apache/beam/integration/nexmark/Query5Model.java b/integration/java/src/main/java/org/apache/beam/integration/nexmark/Query5Model.java
new file mode 100644
index 0000000..a7dd8f0
--- /dev/null
+++ b/integration/java/src/main/java/org/apache/beam/integration/nexmark/Query5Model.java
@@ -0,0 +1,174 @@
+/*
+ * 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.beam.integration.nexmark;
+
+import org.apache.beam.sdk.values.TimestampedValue;
+
+import org.joda.time.Duration;
+import org.joda.time.Instant;
+
+import java.io.Serializable;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.TreeMap;
+
+/**
+ * A direct implementation of {@link Query5}.
+ */
+public class Query5Model extends NexmarkQueryModel implements Serializable {
+  /**
+   * Simulator for query 5.
+   */
+  private class Simulator extends AbstractSimulator<Event, AuctionCount> {
+    /** Time of bids still contributing to open windows, indexed by their auction id. */
+    private final Map<Long, List<Instant>> bids;
+
+    /** When oldest active window starts. */
+    private Instant windowStart;
+
+    public Simulator(NexmarkConfiguration configuration) {
+      super(NexmarkUtils.standardEventIterator(configuration));
+      bids = new TreeMap<>();
+      windowStart = NexmarkUtils.BEGINNING_OF_TIME;
+    }
+
+    /**
+     * Count bids per auction id for bids strictly before {@code end}. Add the auction ids with
+     * the maximum number of bids to results.
+     */
+    private void countBids(Instant end) {
+      Map<Long, Long> counts = new TreeMap<>();
+      long maxCount = 0L;
+      for (Map.Entry<Long, List<Instant>> entry : bids.entrySet()) {
+        long count = 0L;
+        long auction = entry.getKey();
+        for (Instant bid : entry.getValue()) {
+          if (bid.isBefore(end)) {
+            count++;
+          }
+        }
+        if (count > 0) {
+          counts.put(auction, count);
+          maxCount = Math.max(maxCount, count);
+        }
+      }
+      for (Map.Entry<Long, Long> entry : counts.entrySet()) {
+        long auction = entry.getKey();
+        long count = entry.getValue();
+        if (count == maxCount) {
+          AuctionCount result = new AuctionCount(auction, count);
+          addResult(TimestampedValue.of(result, end));
+        }
+      }
+    }
+
+    /**
+     * Retire bids which are strictly before {@code cutoff}. Return true if there are any bids
+     * remaining.
+     */
+    private boolean retireBids(Instant cutoff) {
+      boolean anyRemain = false;
+      for (Map.Entry<Long, List<Instant>> entry : bids.entrySet()) {
+        long auction = entry.getKey();
+        Iterator<Instant> itr = entry.getValue().iterator();
+        while (itr.hasNext()) {
+          Instant bid = itr.next();
+          if (bid.isBefore(cutoff)) {
+            NexmarkUtils.info("retire: %s for %s", bid, auction);
+            itr.remove();
+          } else {
+            anyRemain = true;
+          }
+        }
+      }
+      return anyRemain;
+    }
+
+    /**
+     * Retire active windows until we've reached {@code newWindowStart}.
+     */
+    private void retireWindows(Instant newWindowStart) {
+      while (!newWindowStart.equals(windowStart)) {
+        NexmarkUtils.info("retiring window %s, aiming for %s", windowStart, newWindowStart);
+        // Count bids in the window (windowStart, windowStart + size].
+        countBids(windowStart.plus(Duration.standardSeconds(configuration.windowSizeSec)));
+        // Advance the window.
+        windowStart = windowStart.plus(Duration.standardSeconds(configuration.windowPeriodSec));
+        // Retire bids which will never contribute to a future window.
+        if (!retireBids(windowStart)) {
+          // Can fast forward to latest window since no more outstanding bids.
+          windowStart = newWindowStart;
+        }
+      }
+    }
+
+    /**
+     * Add bid to state.
+     */
+    private void captureBid(Bid bid, Instant timestamp) {
+      List<Instant> existing = bids.get(bid.auction);
+      if (existing == null) {
+        existing = new ArrayList<>();
+        bids.put(bid.auction, existing);
+      }
+      existing.add(timestamp);
+    }
+
+    @Override
+    public void run() {
+      TimestampedValue<Event> timestampedEvent = nextInput();
+      if (timestampedEvent == null) {
+        // Drain the remaining windows.
+        retireWindows(NexmarkUtils.END_OF_TIME);
+        allDone();
+        return;
+      }
+
+      Event event = timestampedEvent.getValue();
+      if (event.bid == null) {
+        // Ignore non-bid events.
+        return;
+      }
+      Instant timestamp = timestampedEvent.getTimestamp();
+      Instant newWindowStart = windowStart(Duration.standardSeconds(configuration.windowSizeSec),
+          Duration.standardSeconds(configuration.windowPeriodSec), timestamp);
+      // Capture results from any windows we can now retire.
+      retireWindows(newWindowStart);
+      // Capture current bid.
+      captureBid(event.bid, timestamp);
+    }
+  }
+
+  public Query5Model(NexmarkConfiguration configuration) {
+    super(configuration);
+  }
+
+  @Override
+  public AbstractSimulator<?, ?> simulator() {
+    return new Simulator(configuration);
+  }
+
+  @Override
+  protected <T> Collection<String> toCollection(Iterator<TimestampedValue<T>> itr) {
+    return toValue(itr);
+  }
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/1f08970a/integration/java/src/main/java/org/apache/beam/integration/nexmark/Query6.java
----------------------------------------------------------------------
diff --git a/integration/java/src/main/java/org/apache/beam/integration/nexmark/Query6.java b/integration/java/src/main/java/org/apache/beam/integration/nexmark/Query6.java
new file mode 100644
index 0000000..49c0d68
--- /dev/null
+++ b/integration/java/src/main/java/org/apache/beam/integration/nexmark/Query6.java
@@ -0,0 +1,154 @@
+/*
+ * 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.beam.integration.nexmark;
+
+import org.apache.beam.sdk.transforms.Combine;
+import org.apache.beam.sdk.transforms.DoFn;
+import org.apache.beam.sdk.transforms.ParDo;
+import org.apache.beam.sdk.transforms.windowing.AfterPane;
+import org.apache.beam.sdk.transforms.windowing.GlobalWindows;
+import org.apache.beam.sdk.transforms.windowing.Repeatedly;
+import org.apache.beam.sdk.transforms.windowing.Window;
+import org.apache.beam.sdk.values.KV;
+import org.apache.beam.sdk.values.PCollection;
+import com.google.common.collect.Lists;
+
+import org.joda.time.Duration;
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+
+/**
+ * Query 6, 'Average Selling Price by Seller'. Select the average selling price over the
+ * last 10 closed auctions by the same seller. In CQL syntax:
+ *
+ * <pre>
+ * SELECT Istream(AVG(Q.final), Q.seller)
+ * FROM (SELECT Rstream(MAX(B.price) AS final, A.seller)
+ *       FROM Auction A [ROWS UNBOUNDED], Bid B [ROWS UNBOUNDED]
+ *       WHERE A.id=B.auction AND B.datetime < A.expires AND A.expires < CURRENT_TIME
+ *       GROUP BY A.id, A.seller) [PARTITION BY A.seller ROWS 10] Q
+ * GROUP BY Q.seller;
+ * </pre>
+ *
+ * <p>We are a little more exact with selecting winning bids: see {@link WinningBids}.
+ */
+class Query6 extends NexmarkQuery {
+  /**
+   * Combiner to keep track of up to {@code maxNumBids} of the most recent wining bids and calculate
+   * their average selling price.
+   */
+  private static class MovingMeanSellingPrice extends Combine.CombineFn<Bid, List<Bid>, Long> {
+    private final int maxNumBids;
+
+    public MovingMeanSellingPrice(int maxNumBids) {
+      this.maxNumBids = maxNumBids;
+    }
+
+    @Override
+    public List<Bid> createAccumulator() {
+      return new ArrayList<>();
+    }
+
+    @Override
+    public List<Bid> addInput(List<Bid> accumulator, Bid input) {
+      accumulator.add(input);
+      Collections.sort(accumulator, Bid.ASCENDING_TIME_THEN_PRICE);
+      if (accumulator.size() > maxNumBids) {
+        accumulator.remove(0);
+      }
+      return accumulator;
+    }
+
+    @Override
+    public List<Bid> mergeAccumulators(Iterable<List<Bid>> accumulators) {
+      List<Bid> result = new ArrayList<>();
+      for (List<Bid> accumulator : accumulators) {
+        for (Bid bid : accumulator) {
+          result.add(bid);
+        }
+      }
+      Collections.sort(result, Bid.ASCENDING_TIME_THEN_PRICE);
+      if (result.size() > maxNumBids) {
+        result = Lists.newArrayList(result.listIterator(result.size() - maxNumBids));
+      }
+      return result;
+    }
+
+    @Override
+    public Long extractOutput(List<Bid> accumulator) {
+      if (accumulator.isEmpty()) {
+        return 0L;
+      }
+      long sumOfPrice = 0;
+      for (Bid bid : accumulator) {
+        sumOfPrice += bid.price;
+      }
+      return Math.round((double) sumOfPrice / accumulator.size());
+    }
+  }
+
+  public Query6(NexmarkConfiguration configuration) {
+    super(configuration, "Query6");
+  }
+
+  private PCollection<SellerPrice> applyTyped(PCollection<Event> events) {
+    return events
+        // Find the winning bid for each closed auction.
+        .apply(new WinningBids(name + ".WinningBids", configuration))
+
+        // Key the winning bid by the seller id.
+        .apply(
+            ParDo.named(name + ".Rekey")
+                .of(new DoFn<AuctionBid, KV<Long, Bid>>() {
+                  @Override
+                  public void processElement(ProcessContext c) {
+                    Auction auction = c.element().auction;
+                    Bid bid = c.element().bid;
+                    c.output(KV.of(auction.seller, bid));
+                  }
+                }))
+
+        // Re-window to update on every wining bid.
+        .apply(
+            Window.<KV<Long, Bid>>into(new GlobalWindows())
+                .triggering(Repeatedly.forever(AfterPane.elementCountAtLeast(1)))
+                .accumulatingFiredPanes()
+                .withAllowedLateness(Duration.ZERO))
+
+        // Find the average of last 10 winning bids for each seller.
+        .apply(Combine.<Long, Bid, Long>perKey(new MovingMeanSellingPrice(10)))
+
+        // Project into our datatype.
+        .apply(
+            ParDo.named(name + ".Select")
+                .of(new DoFn<KV<Long, Long>, SellerPrice>() {
+                  @Override
+                  public void processElement(ProcessContext c) {
+                    c.output(new SellerPrice(c.element().getKey(), c.element().getValue()));
+                  }
+                }));
+  }
+
+  @Override
+  protected PCollection<KnownSize> applyPrim(PCollection<Event> events) {
+    return NexmarkUtils.castToKnownSize(name, applyTyped(events));
+  }
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/1f08970a/integration/java/src/main/java/org/apache/beam/integration/nexmark/Query6Model.java
----------------------------------------------------------------------
diff --git a/integration/java/src/main/java/org/apache/beam/integration/nexmark/Query6Model.java b/integration/java/src/main/java/org/apache/beam/integration/nexmark/Query6Model.java
new file mode 100644
index 0000000..639ec9f
--- /dev/null
+++ b/integration/java/src/main/java/org/apache/beam/integration/nexmark/Query6Model.java
@@ -0,0 +1,128 @@
+/*
+ * 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.beam.integration.nexmark;
+
+import org.apache.beam.sdk.transforms.windowing.BoundedWindow;
+import org.apache.beam.sdk.values.TimestampedValue;
+
+import org.joda.time.Instant;
+import org.junit.Assert;
+
+import java.io.Serializable;
+import java.util.Collection;
+import java.util.Iterator;
+import java.util.Map;
+import java.util.TreeMap;
+
+/**
+ * A direct implementation of {@link Query6}.
+ */
+public class Query6Model extends NexmarkQueryModel implements Serializable {
+  /**
+   * Simulator for query 6.
+   */
+  private static class Simulator extends AbstractSimulator<AuctionBid, SellerPrice> {
+    /** The cumulative count of winning bids, indexed by seller id. */
+    private final Map<Long, Long> numWinningBidsPerSeller;
+
+    /** The cumulative total of winning bid prices, indexed by seller id. */
+    private final Map<Long, Long> totalWinningBidPricesPerSeller;
+
+    private Instant lastTimestamp;
+
+    public Simulator(NexmarkConfiguration configuration) {
+      super(new WinningBidsSimulator(configuration).results());
+      numWinningBidsPerSeller = new TreeMap<>();
+      totalWinningBidPricesPerSeller = new TreeMap<>();
+      lastTimestamp = BoundedWindow.TIMESTAMP_MIN_VALUE;
+    }
+
+    /**
+     * Update the per-seller running counts/sums.
+     */
+    private void captureWinningBid(Auction auction, Bid bid, Instant timestamp) {
+      NexmarkUtils.info("winning auction, bid: %s, %s", auction, bid);
+      Long count = numWinningBidsPerSeller.get(auction.seller);
+      if (count == null) {
+        count = 1L;
+      } else {
+        count += 1;
+      }
+      numWinningBidsPerSeller.put(auction.seller, count);
+      Long total = totalWinningBidPricesPerSeller.get(auction.seller);
+      if (total == null) {
+        total = bid.price;
+      } else {
+        total += bid.price;
+      }
+      totalWinningBidPricesPerSeller.put(auction.seller, total);
+      TimestampedValue<SellerPrice> intermediateResult = TimestampedValue.of(
+          new SellerPrice(auction.seller, Math.round((double) total / count)), timestamp);
+      addIntermediateResult(intermediateResult);
+    }
+
+
+    @Override
+    protected void run() {
+      TimestampedValue<AuctionBid> timestampedWinningBid = nextInput();
+      if (timestampedWinningBid == null) {
+        for (long seller : numWinningBidsPerSeller.keySet()) {
+          long count = numWinningBidsPerSeller.get(seller);
+          long total = totalWinningBidPricesPerSeller.get(seller);
+          addResult(TimestampedValue.of(
+              new SellerPrice(seller, Math.round((double) total / count)), lastTimestamp));
+        }
+        allDone();
+        return;
+      }
+
+      lastTimestamp = timestampedWinningBid.getTimestamp();
+      captureWinningBid(timestampedWinningBid.getValue().auction,
+          timestampedWinningBid.getValue().bid, lastTimestamp);
+    }
+  }
+
+  public Query6Model(NexmarkConfiguration configuration) {
+    super(configuration);
+  }
+
+  @Override
+  public AbstractSimulator<?, ?> simulator() {
+    return new Simulator(configuration);
+  }
+
+  @Override
+  protected Iterable<TimestampedValue<KnownSize>> relevantResults(
+      Iterable<TimestampedValue<KnownSize>> results) {
+    // Find the last (in processing time) reported average price for each seller.
+    Map<Long, TimestampedValue<KnownSize>> finalAverages = new TreeMap<>();
+    for (TimestampedValue<KnownSize> obj : results) {
+      Assert.assertTrue("have SellerPrice", obj.getValue() instanceof SellerPrice);
+      SellerPrice sellerPrice = (SellerPrice) obj.getValue();
+      finalAverages.put(
+          sellerPrice.seller, TimestampedValue.of((KnownSize) sellerPrice, obj.getTimestamp()));
+    }
+    return finalAverages.values();
+  }
+
+  @Override
+  protected <T> Collection<String> toCollection(Iterator<TimestampedValue<T>> itr) {
+    return toValue(itr);
+  }
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/1f08970a/integration/java/src/main/java/org/apache/beam/integration/nexmark/Query7.java
----------------------------------------------------------------------
diff --git a/integration/java/src/main/java/org/apache/beam/integration/nexmark/Query7.java b/integration/java/src/main/java/org/apache/beam/integration/nexmark/Query7.java
new file mode 100644
index 0000000..1f63b35
--- /dev/null
+++ b/integration/java/src/main/java/org/apache/beam/integration/nexmark/Query7.java
@@ -0,0 +1,87 @@
+/*
+ * 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.beam.integration.nexmark;
+
+import org.apache.beam.sdk.transforms.DoFn;
+import org.apache.beam.sdk.transforms.Max;
+import org.apache.beam.sdk.transforms.ParDo;
+import org.apache.beam.sdk.transforms.windowing.FixedWindows;
+import org.apache.beam.sdk.transforms.windowing.Window;
+import org.apache.beam.sdk.values.PCollection;
+import org.apache.beam.sdk.values.PCollectionView;
+
+import org.joda.time.Duration;
+
+/**
+ * Query 7, 'Highest Bid'. Select the bids with the highest bid
+ * price in the last minute. In CQL syntax:
+ *
+ * <pre>
+ * SELECT Rstream(B.auction, B.price, B.bidder)
+ * FROM Bid [RANGE 1 MINUTE SLIDE 1 MINUTE] B
+ * WHERE B.price = (SELECT MAX(B1.price)
+ *                  FROM BID [RANGE 1 MINUTE SLIDE 1 MINUTE] B1);
+ * </pre>
+ *
+ * <p>We will use a shorter window to help make testing easier. We'll also implement this using
+ * a side-input in order to exercise that functionality. (A combiner, as used in Query 5, is
+ * a more efficient approach.).
+ */
+class Query7 extends NexmarkQuery {
+  public Query7(NexmarkConfiguration configuration) {
+    super(configuration, "Query7");
+  }
+
+  private PCollection<Bid> applyTyped(PCollection<Event> events) {
+    // Window the bids.
+    PCollection<Bid> slidingBids = events.apply(JUST_BIDS).apply(
+        Window.<Bid>into(FixedWindows.of(Duration.standardSeconds(configuration.windowSizeSec))));
+
+    // Find the largest price in all bids.
+    // NOTE: It would be more efficient to write this query much as we did for Query5, using
+    // a binary combiner to accumulate the bids with maximal price. As written this query
+    // requires an additional scan per window, with the associated cost of snapshotted state and
+    // its I/O. We'll keep this implementation since it illustrates the use of side inputs.
+    final PCollectionView<Long> maxPriceView =
+        slidingBids //
+            .apply(BID_TO_PRICE)
+            .apply(Max.longsGlobally().withFanout(configuration.fanout).asSingletonView());
+
+    return slidingBids
+        // Select all bids which have that maximum price (there may be more than one).
+        .apply(
+            ParDo.named(name + ".Select")
+                .withSideInputs(maxPriceView)
+                .of(new DoFn<Bid, Bid>() {
+                  @Override
+                  public void processElement(ProcessContext c) {
+                    long maxPrice = c.sideInput(maxPriceView);
+                    Bid bid = c.element();
+                    if (bid.price == maxPrice) {
+                      c.output(bid);
+                    }
+                  }
+                }));
+  }
+
+  @Override
+  protected PCollection<KnownSize> applyPrim(PCollection<Event> events) {
+    return NexmarkUtils.castToKnownSize(name, applyTyped(events));
+  }
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/1f08970a/integration/java/src/main/java/org/apache/beam/integration/nexmark/Query7Model.java
----------------------------------------------------------------------
diff --git a/integration/java/src/main/java/org/apache/beam/integration/nexmark/Query7Model.java b/integration/java/src/main/java/org/apache/beam/integration/nexmark/Query7Model.java
new file mode 100644
index 0000000..e835133
--- /dev/null
+++ b/integration/java/src/main/java/org/apache/beam/integration/nexmark/Query7Model.java
@@ -0,0 +1,128 @@
+/*
+ * 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.beam.integration.nexmark;
+
+import org.apache.beam.sdk.transforms.windowing.BoundedWindow;
+import org.apache.beam.sdk.values.TimestampedValue;
+
+import org.joda.time.Duration;
+import org.joda.time.Instant;
+
+import java.io.Serializable;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Iterator;
+import java.util.List;
+
+/**
+ * A direct implementation of {@link Query7}.
+ */
+public class Query7Model extends NexmarkQueryModel implements Serializable {
+  /**
+   * Simulator for query 7.
+   */
+  private class Simulator extends AbstractSimulator<Event, Bid> {
+    /** Bids with highest bid price seen in the current window. */
+    private final List<Bid> highestBids;
+
+    /** When current window started. */
+    private Instant windowStart;
+
+    private Instant lastTimestamp;
+
+    public Simulator(NexmarkConfiguration configuration) {
+      super(NexmarkUtils.standardEventIterator(configuration));
+      highestBids = new ArrayList<>();
+      windowStart = NexmarkUtils.BEGINNING_OF_TIME;
+      lastTimestamp = BoundedWindow.TIMESTAMP_MIN_VALUE;
+    }
+
+    /**
+     * Transfer the currently winning bids into results and retire them.
+     */
+    private void retireWindow(Instant timestamp) {
+      for (Bid bid : highestBids) {
+        addResult(TimestampedValue.of(bid, timestamp));
+      }
+      highestBids.clear();
+    }
+
+    /**
+     * Keep just the highest price bid.
+     */
+    private void captureBid(Bid bid) {
+      Iterator<Bid> itr = highestBids.iterator();
+      boolean isWinning = true;
+      while (itr.hasNext()) {
+        Bid existingBid = itr.next();
+        if (existingBid.price > bid.price) {
+          isWinning = false;
+          break;
+        }
+        NexmarkUtils.info("smaller price: %s", existingBid);
+        itr.remove();
+      }
+      if (isWinning) {
+        NexmarkUtils.info("larger price: %s", bid);
+        highestBids.add(bid);
+      }
+    }
+
+    @Override
+    protected void run() {
+      TimestampedValue<Event> timestampedEvent = nextInput();
+      if (timestampedEvent == null) {
+        // Capture all remaining bids in results.
+        retireWindow(lastTimestamp);
+        allDone();
+        return;
+      }
+
+      Event event = timestampedEvent.getValue();
+      if (event.bid == null) {
+        // Ignore non-bid events.
+        return;
+      }
+      lastTimestamp = timestampedEvent.getTimestamp();
+      Instant newWindowStart = windowStart(Duration.standardSeconds(configuration.windowSizeSec),
+          Duration.standardSeconds(configuration.windowSizeSec), lastTimestamp);
+      if (!newWindowStart.equals(windowStart)) {
+        // Capture highest priced bids in current window and retire it.
+        retireWindow(lastTimestamp);
+        windowStart = newWindowStart;
+      }
+      // Keep only the highest bids.
+      captureBid(event.bid);
+    }
+  }
+
+  public Query7Model(NexmarkConfiguration configuration) {
+    super(configuration);
+  }
+
+  @Override
+  public AbstractSimulator<?, ?> simulator() {
+    return new Simulator(configuration);
+  }
+
+  @Override
+  protected <T> Collection<String> toCollection(Iterator<TimestampedValue<T>> itr) {
+    return toValueOrder(itr);
+  }
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/1f08970a/integration/java/src/main/java/org/apache/beam/integration/nexmark/Query8.java
----------------------------------------------------------------------
diff --git a/integration/java/src/main/java/org/apache/beam/integration/nexmark/Query8.java b/integration/java/src/main/java/org/apache/beam/integration/nexmark/Query8.java
new file mode 100644
index 0000000..e58453b
--- /dev/null
+++ b/integration/java/src/main/java/org/apache/beam/integration/nexmark/Query8.java
@@ -0,0 +1,92 @@
+/*
+ * 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.beam.integration.nexmark;
+
+import org.apache.beam.sdk.transforms.DoFn;
+import org.apache.beam.sdk.transforms.ParDo;
+import org.apache.beam.sdk.transforms.join.CoGbkResult;
+import org.apache.beam.sdk.transforms.join.CoGroupByKey;
+import org.apache.beam.sdk.transforms.join.KeyedPCollectionTuple;
+import org.apache.beam.sdk.transforms.windowing.FixedWindows;
+import org.apache.beam.sdk.transforms.windowing.Window;
+import org.apache.beam.sdk.values.KV;
+import org.apache.beam.sdk.values.PCollection;
+
+import org.joda.time.Duration;
+
+/**
+ * Query 8, 'Monitor New Users'. Select people who have entered the system and created auctions
+ * in the last 12 hours, updated every 12 hours. In CQL syntax:
+ *
+ * <pre>
+ * SELECT Rstream(P.id, P.name, A.reserve)
+ * FROM Person [RANGE 12 HOUR] P, Auction [RANGE 12 HOUR] A
+ * WHERE P.id = A.seller;
+ * </pre>
+ *
+ * <p>To make things a bit more dynamic and easier to test we'll use a much shorter window.
+ */
+class Query8 extends NexmarkQuery {
+  public Query8(NexmarkConfiguration configuration) {
+    super(configuration, "Query8");
+  }
+
+  private PCollection<IdNameReserve> applyTyped(PCollection<Event> events) {
+    // Window and key new people by their id.
+    PCollection<KV<Long, Person>> personsById =
+        events.apply(JUST_NEW_PERSONS)
+            .apply(Window.<Person>into(
+                             FixedWindows.of(Duration.standardSeconds(configuration.windowSizeSec)))
+                    .named("Query8.WindowPersons"))
+            .apply(PERSON_BY_ID);
+
+    // Window and key new auctions by their id.
+    PCollection<KV<Long, Auction>> auctionsBySeller =
+        events.apply(JUST_NEW_AUCTIONS)
+            .apply(Window.<Auction>into(
+                             FixedWindows.of(Duration.standardSeconds(configuration.windowSizeSec)))
+                    .named("Query8.WindowAuctions"))
+            .apply(AUCTION_BY_SELLER);
+
+    // Join people and auctions and project the person id, name and auction reserve price.
+    return KeyedPCollectionTuple.of(PERSON_TAG, personsById)
+        .and(AUCTION_TAG, auctionsBySeller)
+        .apply(CoGroupByKey.<Long>create())
+        .apply(
+            ParDo.named(name + ".Select")
+                .of(new DoFn<KV<Long, CoGbkResult>, IdNameReserve>() {
+                  @Override
+                  public void processElement(ProcessContext c) {
+                    Person person = c.element().getValue().getOnly(PERSON_TAG, null);
+                    if (person == null) {
+                      // Person was not created in last window period.
+                      return;
+                    }
+                    for (Auction auction : c.element().getValue().getAll(AUCTION_TAG)) {
+                      c.output(new IdNameReserve(person.id, person.name, auction.reserve));
+                    }
+                  }
+                }));
+  }
+
+  @Override
+  protected PCollection<KnownSize> applyPrim(PCollection<Event> events) {
+    return NexmarkUtils.castToKnownSize(name, applyTyped(events));
+  }
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/1f08970a/integration/java/src/main/java/org/apache/beam/integration/nexmark/Query8Model.java
----------------------------------------------------------------------
diff --git a/integration/java/src/main/java/org/apache/beam/integration/nexmark/Query8Model.java b/integration/java/src/main/java/org/apache/beam/integration/nexmark/Query8Model.java
new file mode 100644
index 0000000..00f7355
--- /dev/null
+++ b/integration/java/src/main/java/org/apache/beam/integration/nexmark/Query8Model.java
@@ -0,0 +1,145 @@
+/*
+ * 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.beam.integration.nexmark;
+
+import org.apache.beam.sdk.values.TimestampedValue;
+import com.google.common.collect.ArrayListMultimap;
+import com.google.common.collect.Multimap;
+
+import org.joda.time.Duration;
+import org.joda.time.Instant;
+
+import java.io.Serializable;
+import java.util.Collection;
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.Map;
+
+/**
+ * A direct implementation of {@link Query8}.
+ */
+public class Query8Model extends NexmarkQueryModel implements Serializable {
+  /**
+   * Simulator for query 8.
+   */
+  private class Simulator extends AbstractSimulator<Event, IdNameReserve> {
+    /** New persons seen in the current window, indexed by id. */
+    private final Map<Long, Person> newPersons;
+
+    /** New auctions seen in the current window, indexed by seller id. */
+    private final Multimap<Long, Auction> newAuctions;
+
+    /** When did the current window start. */
+    private Instant windowStart;
+
+    public Simulator(NexmarkConfiguration configuration) {
+      super(NexmarkUtils.standardEventIterator(configuration));
+      newPersons = new HashMap<>();
+      newAuctions = ArrayListMultimap.create();
+      windowStart = NexmarkUtils.BEGINNING_OF_TIME;
+    }
+
+    /**
+     * Retire all persons added in last window.
+     */
+    private void retirePersons() {
+      for (Map.Entry<Long, Person> entry : newPersons.entrySet()) {
+        NexmarkUtils.info("retire: %s", entry.getValue());
+      }
+      newPersons.clear();
+    }
+
+    /**
+     * Retire all auctions added in last window.
+     */
+    private void retireAuctions() {
+      for (Map.Entry<Long, Auction> entry : newAuctions.entries()) {
+        NexmarkUtils.info("retire: %s", entry.getValue());
+      }
+      newAuctions.clear();
+    }
+
+    /**
+     * Capture new result.
+     */
+    private void addResult(Auction auction, Person person, Instant timestamp) {
+      addResult(TimestampedValue.of(
+          new IdNameReserve(person.id, person.name, auction.reserve), timestamp));
+    }
+
+    @Override
+    public void run() {
+      TimestampedValue<Event> timestampedEvent = nextInput();
+      if (timestampedEvent == null) {
+        allDone();
+        return;
+      }
+
+      Event event = timestampedEvent.getValue();
+      if (event.bid != null) {
+        // Ignore bid events.
+        // Keep looking for next events.
+        return;
+      }
+      Instant timestamp = timestampedEvent.getTimestamp();
+      Instant newWindowStart = windowStart(Duration.standardSeconds(configuration.windowSizeSec),
+          Duration.standardSeconds(configuration.windowSizeSec), timestamp);
+      if (!newWindowStart.equals(windowStart)) {
+        // Retire this window.
+        retirePersons();
+        retireAuctions();
+        windowStart = newWindowStart;
+      }
+
+      if (event.newAuction != null) {
+        // Join new auction with existing person, if any.
+        Person person = newPersons.get(event.newAuction.seller);
+        if (person != null) {
+          addResult(event.newAuction, person, timestamp);
+        } else {
+          // Remember auction for future new people.
+          newAuctions.put(event.newAuction.seller, event.newAuction);
+        }
+      } else {
+        // Join new person with existing auctions.
+        for (Auction auction : newAuctions.get(event.newPerson.id)) {
+          addResult(auction, event.newPerson, timestamp);
+        }
+        // We'll never need these auctions again.
+        newAuctions.removeAll(event.newPerson.id);
+        // Remember person for future auctions.
+        newPersons.put(event.newPerson.id, event.newPerson);
+      }
+    }
+  }
+
+  public Query8Model(NexmarkConfiguration configuration) {
+    super(configuration);
+  }
+
+  @Override
+  public AbstractSimulator<?, ?> simulator() {
+    return new Simulator(configuration);
+  }
+
+  @Override
+  protected <T> Collection<String> toCollection(Iterator<TimestampedValue<T>> itr) {
+    return toValue(itr);
+  }
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/1f08970a/integration/java/src/main/java/org/apache/beam/integration/nexmark/Query9.java
----------------------------------------------------------------------
diff --git a/integration/java/src/main/java/org/apache/beam/integration/nexmark/Query9.java b/integration/java/src/main/java/org/apache/beam/integration/nexmark/Query9.java
new file mode 100644
index 0000000..2c0a526
--- /dev/null
+++ b/integration/java/src/main/java/org/apache/beam/integration/nexmark/Query9.java
@@ -0,0 +1,40 @@
+/*
+ * 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.beam.integration.nexmark;
+
+import org.apache.beam.sdk.values.PCollection;
+
+/**
+ * Query "9", 'Winning bids'. Select just the winning bids. Not in original NEXMark suite, but
+ * handy for testing. See {@link WinningBids} for the details.
+ */
+class Query9 extends NexmarkQuery {
+  public Query9(NexmarkConfiguration configuration) {
+    super(configuration, "Query9");
+  }
+
+  private PCollection<AuctionBid> applyTyped(PCollection<Event> events) {
+    return events.apply(new WinningBids(name, configuration));
+  }
+
+  @Override
+  protected PCollection<KnownSize> applyPrim(PCollection<Event> events) {
+    return NexmarkUtils.castToKnownSize(name, applyTyped(events));
+  }
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/1f08970a/integration/java/src/main/java/org/apache/beam/integration/nexmark/Query9Model.java
----------------------------------------------------------------------
diff --git a/integration/java/src/main/java/org/apache/beam/integration/nexmark/Query9Model.java b/integration/java/src/main/java/org/apache/beam/integration/nexmark/Query9Model.java
new file mode 100644
index 0000000..1fad648
--- /dev/null
+++ b/integration/java/src/main/java/org/apache/beam/integration/nexmark/Query9Model.java
@@ -0,0 +1,44 @@
+/*
+ * 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.beam.integration.nexmark;
+
+import org.apache.beam.sdk.values.TimestampedValue;
+
+import java.io.Serializable;
+import java.util.Collection;
+import java.util.Iterator;
+
+/**
+ * A direct implementation of {@link Query9}.
+ */
+public class Query9Model extends NexmarkQueryModel implements Serializable {
+  public Query9Model(NexmarkConfiguration configuration) {
+    super(configuration);
+  }
+
+  @Override
+  public AbstractSimulator<?, ?> simulator() {
+    return new WinningBidsSimulator(configuration);
+  }
+
+  @Override
+  protected <T> Collection<String> toCollection(Iterator<TimestampedValue<T>> itr) {
+    return toValue(itr);
+  }
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/1f08970a/integration/java/src/main/java/org/apache/beam/integration/nexmark/README.md
----------------------------------------------------------------------
diff --git a/integration/java/src/main/java/org/apache/beam/integration/nexmark/README.md b/integration/java/src/main/java/org/apache/beam/integration/nexmark/README.md
new file mode 100644
index 0000000..5e33327
--- /dev/null
+++ b/integration/java/src/main/java/org/apache/beam/integration/nexmark/README.md
@@ -0,0 +1,166 @@
+<!--
+    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.
+-->
+
+# NEXMark integration suite
+
+This is a suite of pipelines inspired by the 'continuous data stream'
+queries in [http://datalab.cs.pdx.edu/niagaraST/NEXMark/]
+(http://datalab.cs.pdx.edu/niagaraST/NEXMark/).
+
+The queries are over a simple online auction system with tables of
+**Person**, **Auction** and **Bid** records.
+
+The queries are:
+
+* **Query1**: What are the bid values in Euro's?
+  Illustrates a simple map.
+* **Query2**: What are the auctions with particular auction numbers?
+  Illustrates a simple filter.
+* **Query3**: Who is selling in particular US states?
+  Illustrates an incremental join (using per-key state) and filter.
+* **Query4**: What is the average selling price for each auction
+  category?
+  Illustrates complex join (using custom window functions) and
+  aggregation.
+* **Query5**: Which auctions have seen the most bids in the last period?
+  Illustrates sliding windows and combiners.
+* **Query6**: What is the average selling price per seller for their
+  last 10 closed auctions.
+  Shares the same 'winning bids' core as for **Query4**, and
+  illustrates a specialized combiner.
+* **Query7**: What are the highest bids per period?
+  Deliberately implemented using a side input to illustrate fanout.
+* **Query8**: Who has entered the system and created an auction in
+  the last period?
+  Illustrates a simple join.
+
+We have augmented the original queries with five more:
+
+* **Query0**: Pass-through.
+  Allows us to measure the monitoring overhead.
+* **Query9**: Winning-bids.
+  A common sub-query shared by **Query4** and **Query6**.
+* **Query10**: Log all events to GCS files.
+  Illustrates windows with large side effects on firing.
+* **Query11**: How many bids did a user make in each session they
+  were active?
+  Illustrates session windows.
+* **Query12**: How many bids does a user make within a fixed
+  processing time limit?
+  Illustrates working in processing time in the Global window, as
+  compared with event time in non-Global windows for all the other
+  queries.
+
+The queries can be executed using a 'Driver' for a given backend.
+Currently the supported drivers are:
+
+* **NexmarkInProcessDriver** for running locally on a single machine.
+* **NexmarkGoogleDriver** for running on the Google Cloud Dataflow
+  service. Requires a Google Cloud account.
+* **NexmarkFlinkDriver** for running on a Flink cluster. Requires the
+  cluster to be established and the Nexmark jar to be distributed to
+  each worker.
+
+Other drivers are straightforward.
+
+Test data is deterministically synthesized on demand. The test
+data may be synthesized in the same pipeline as the query itself,
+or may be published to Pubsub.
+
+The query results may be:
+
+* Published to Pubsub.
+* Written to text files as plain text.
+* Written to text files using an Avro encoding.
+* Send to BigQuery.
+* Discarded.
+
+Options are provided for measuring progress, measuring overall
+pipeline performance, and comparing that performance against a known
+baseline. However that machinery has only been implemented against
+the Google Cloud Dataflow driver.
+
+## Running on Google Cloud Dataflow
+
+An example invocation for **Query10** on the Google Cloud Dataflow
+service.
+
+```
+java -cp integration/java/target/java-integration-all-bundled-0.2.0-incubating-SNAPSHOT.jar \
+  org.apache.beam.integration.nexmark.NexmarkGoogleDriver \
+  --project=<your project> \
+  --zone=<your zone> \
+  --workerMachineType=n1-highmem-8 \
+  --stagingLocation=<a gs path for staging> \
+  --streaming=true \
+  --sourceType=PUBSUB \
+  --pubSubMode=PUBLISH_ONLY \
+  --pubsubTopic=<an existing Pubsub topic> \
+  --resourceNameMode=VERBATIM \
+  --manageResources=false \
+  --monitorJobs=false \
+  --numEventGenerators=64 \
+  --numWorkers=16 \
+  --maxNumWorkers=16 \
+  --query=10 \
+  --firstEventRate=100000 \
+  --nextEventRate=100000 \
+  --ratePeriodSec=3600 \
+  --isRateLimited=true \
+  --avgPersonByteSize=500 \
+  --avgAuctionByteSize=500 \
+  --avgBidByteSize=500 \
+  --probDelayedEvent=0.000001 \
+  --occasionalDelaySec=3600 \
+  --numEvents=0 \
+  --useWallclockEventTime=true \
+  --usePubsubPublishTime=true \
+  --experiments=enable_custom_pubsub_sink
+```
+
+```
+java -cp integration/java/target/java-integration-all-bundled-0.2.0-incubating-SNAPSHOT.jar \
+  org.apache.beam.integration.nexmark.NexmarkGoogleDriver \
+  --project=<your project> \
+  --zone=<your zone> \
+  --workerMachineType=n1-highmem-8 \
+  --stagingLocation=<a gs path for staging> \
+  --streaming=true \
+  --sourceType=PUBSUB \
+  --pubSubMode=SUBSCRIBE_ONLY \
+  --pubsubSubscription=<an existing Pubsub subscription to above topic> \
+  --resourceNameMode=VERBATIM \
+  --manageResources=false \
+  --monitorJobs=false \
+  --numWorkers=64 \
+  --maxNumWorkers=64 \
+  --query=10 \
+  --usePubsubPublishTime=true \
+  --outputPath=<a gs path under which log files will be written> \
+  --windowSizeSec=600 \
+  --occasionalDelaySec=3600 \
+  --maxLogEvents=10000 \
+  --experiments=enable_custom_pubsub_source
+```
+
+## Running on Flink
+
+See [BEAM_ON_FLINK_ON_GCP](./BEAM_ON_FLINK_ON_GCP.md) for instructions
+on running a NexMark pipeline using Flink hosted on a Google Compute
+Platform cluster.


[13/55] [abbrv] beam git commit: Refactor classes into packages

Posted by ie...@apache.org.
http://git-wip-us.apache.org/repos/asf/beam/blob/a7f9f7d0/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/Query3.java
----------------------------------------------------------------------
diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/Query3.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/Query3.java
deleted file mode 100644
index 71969c4..0000000
--- a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/Query3.java
+++ /dev/null
@@ -1,249 +0,0 @@
-/*
- * 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.beam.integration.nexmark;
-
-import java.io.IOException;
-import java.util.ArrayList;
-import java.util.List;
-import javax.annotation.Nullable;
-
-import org.apache.beam.sdk.coders.ListCoder;
-import org.apache.beam.sdk.transforms.Aggregator;
-import org.apache.beam.sdk.transforms.DoFn;
-import org.apache.beam.sdk.transforms.Filter;
-import org.apache.beam.sdk.transforms.ParDo;
-import org.apache.beam.sdk.transforms.SerializableFunction;
-import org.apache.beam.sdk.transforms.Sum;
-import org.apache.beam.sdk.transforms.join.CoGbkResult;
-import org.apache.beam.sdk.transforms.join.CoGroupByKey;
-import org.apache.beam.sdk.transforms.join.KeyedPCollectionTuple;
-import org.apache.beam.sdk.transforms.windowing.FixedWindows;
-import org.apache.beam.sdk.transforms.windowing.Window;
-import org.apache.beam.sdk.util.state.StateSpec;
-import org.apache.beam.sdk.util.state.StateSpecs;
-import org.apache.beam.sdk.util.state.ValueState;
-import org.apache.beam.sdk.values.KV;
-import org.apache.beam.sdk.values.PCollection;
-import org.joda.time.Duration;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-/**
- * Query 3, 'Local Item Suggestion'. Who is selling in OR, ID or CA in category 10, and for what
- * auction ids? In CQL syntax:
- *
- * <pre>
- * SELECT Istream(P.name, P.city, P.state, A.id)
- * FROM Auction A [ROWS UNBOUNDED], Person P [ROWS UNBOUNDED]
- * WHERE A.seller = P.id AND (P.state = `OR' OR P.state = `ID' OR P.state = `CA') AND A.category
- * = 10;
- * </pre>
- *
- * <p>We'll implement this query to allow 'new auction' events to come before the 'new person'
- * events for the auction seller. Those auctions will be stored until the matching person is
- * seen. Then all subsequent auctions for a person will use the stored person record.
- *
- * <p>A real system would use an external system to maintain the id-to-person association.
- */
-class Query3 extends NexmarkQuery {
-  private static final Logger LOG = LoggerFactory.getLogger(Query3.class);
-//  private static final StateContext GLOBAL_NAMESPACE = StateContexts.global();
-  private static final StateSpec<Object, ValueState<List<Auction>>> AUCTION_LIST_CODED_TAG =
-      StateSpecs.value(ListCoder.of(Auction.CODER));
-  private static final StateSpec<Object, ValueState<Person>> PERSON_CODED_TAG =
-      StateSpecs.value(Person.CODER);
-
-  /**
-   * Join {@code auctions} and {@code people} by person id and emit their cross-product one pair
-   * at a time.
-   *
-   * <p>We know a person may submit any number of auctions. Thus new person event must have the
-   * person record stored in persistent state in order to match future auctions by that person.
-   *
-   * <p>However we know that each auction is associated with at most one person, so only need
-   * to store auction records in persistent state until we have seen the corresponding person
-   * record. And of course may have already seen that record.
-   */
-  private static class JoinDoFn extends DoFn<KV<Long, CoGbkResult>, KV<Auction, Person>> {
-    private final Aggregator<Long, Long> newAuctionCounter =
-        createAggregator("newAuction", Sum.ofLongs());
-    private final Aggregator<Long, Long> newPersonCounter =
-        createAggregator("newPerson", Sum.ofLongs());
-    private final Aggregator<Long, Long> newNewOutputCounter =
-        createAggregator("newNewOutput", Sum.ofLongs());
-    private final Aggregator<Long, Long> newOldOutputCounter =
-        createAggregator("newOldOutput", Sum.ofLongs());
-    private final Aggregator<Long, Long> oldNewOutputCounter =
-        createAggregator("oldNewOutput", Sum.ofLongs());
-    public final Aggregator<Long, Long> fatalCounter = createAggregator("fatal", Sum.ofLongs());
-
-    @ProcessElement
-    public void processElement(ProcessContext c) throws IOException {
-      //TODO: This is using the internal state API. Rework to use the
-      //TODO Ismael this is broken for not access to state
-      // We would *almost* implement this by  rewindowing into the global window and
-      // running a combiner over the result. The combiner's accumulator would be the
-      // state we use below. However, combiners cannot emit intermediate results, thus
-      // we need to wait for the pending ReduceFn API.
-//      StateInternals<?> stateInternals = c.windowingInternals().stateInternals();
-//      ValueState<Person> personState = stateInternals.state(GLOBAL_NAMESPACE, PERSON_CODED_TAG);
-//      Person existingPerson = personState.read();
-      Person existingPerson = null;
-      if (existingPerson != null) {
-        // We've already seen the new person event for this person id.
-        // We can join with any new auctions on-the-fly without needing any
-        // additional persistent state.
-        for (Auction newAuction : c.element().getValue().getAll(AUCTION_TAG)) {
-          newAuctionCounter.addValue(1L);
-          newOldOutputCounter.addValue(1L);
-          c.output(KV.of(newAuction, existingPerson));
-        }
-        return;
-      }
-
-//      ValueState<List<Auction>> auctionsState =
-//          stateInternals.state(GLOBAL_NAMESPACE, AUCTION_LIST_CODED_TAG);
-      Person theNewPerson = null;
-      for (Person newPerson : c.element().getValue().getAll(PERSON_TAG)) {
-        if (theNewPerson == null) {
-          theNewPerson = newPerson;
-        } else {
-          if (theNewPerson.equals(newPerson)) {
-            LOG.error("**** duplicate person {} ****", theNewPerson);
-          } else {
-            LOG.error("**** conflicting persons {} and {} ****", theNewPerson, newPerson);
-          }
-          fatalCounter.addValue(1L);
-          continue;
-        }
-        newPersonCounter.addValue(1L);
-        // We've now seen the person for this person id so can flush any
-        // pending auctions for the same seller id.
-        List<Auction> pendingAuctions = null; //auctionsState.read();
-        if (pendingAuctions != null) {
-          for (Auction pendingAuction : pendingAuctions) {
-            oldNewOutputCounter.addValue(1L);
-            c.output(KV.of(pendingAuction, newPerson));
-          }
-//          auctionsState.clear();
-        }
-        // Also deal with any new auctions.
-        for (Auction newAuction : c.element().getValue().getAll(AUCTION_TAG)) {
-          newAuctionCounter.addValue(1L);
-          newNewOutputCounter.addValue(1L);
-          c.output(KV.of(newAuction, newPerson));
-        }
-        // Remember this person for any future auctions.
-
-//        personState.write(newPerson);
-      }
-      if (theNewPerson != null) {
-        return;
-      }
-
-      // We'll need to remember the auctions until we see the corresponding
-      // new person event.
-      List<Auction> pendingAuctions = null; //auctionsState.read();
-      if (pendingAuctions == null) {
-        pendingAuctions = new ArrayList<>();
-      }
-      for (Auction newAuction : c.element().getValue().getAll(AUCTION_TAG)) {
-        newAuctionCounter.addValue(1L);
-        pendingAuctions.add(newAuction);
-      }
-//      auctionsState.write(pendingAuctions);
-    }
-  }
-
-  private final JoinDoFn joinDoFn = new JoinDoFn();
-
-  public Query3(NexmarkConfiguration configuration) {
-    super(configuration, "Query3");
-  }
-
-  @Override
-  @Nullable
-  public Aggregator<Long, Long> getFatalCount() {
-    return joinDoFn.fatalCounter;
-  }
-
-  private PCollection<NameCityStateId> applyTyped(PCollection<Event> events) {
-    // Batch into incremental results windows.
-    events = events.apply(
-        Window.<Event>into(FixedWindows.of(Duration.standardSeconds(configuration.windowSizeSec))));
-
-    PCollection<KV<Long, Auction>> auctionsBySellerId =
-        events
-            // Only want the new auction events.
-            .apply(JUST_NEW_AUCTIONS)
-
-            // We only want auctions in category 10.
-            .apply(name + ".InCategory", Filter.by(new SerializableFunction<Auction, Boolean>() {
-              @Override
-              public Boolean apply(Auction auction) {
-                return auction.category == 10;
-              }
-            }))
-
-            // Key auctions by their seller id.
-            .apply("AuctionBySeller", AUCTION_BY_SELLER);
-
-    PCollection<KV<Long, Person>> personsById =
-        events
-            // Only want the new people events.
-            .apply(JUST_NEW_PERSONS)
-
-            // We only want people in OR, ID, CA.
-            .apply(name + ".InState", Filter.by(new SerializableFunction<Person, Boolean>() {
-              @Override
-              public Boolean apply(Person person) {
-                return person.state.equals("OR") || person.state.equals("ID")
-                    || person.state.equals("CA");
-              }
-            }))
-
-            // Key people by their id.
-            .apply("PersonById", PERSON_BY_ID);
-
-    return
-      // Join auctions and people.
-        // concatenate KeyedPCollections
-      KeyedPCollectionTuple.of(AUCTION_TAG, auctionsBySellerId)
-            .and(PERSON_TAG, personsById)
-        // group auctions and persons by personId
-        .apply(CoGroupByKey.<Long>create())
-            .apply(name + ".Join", ParDo.of(joinDoFn))
-
-            // Project what we want.
-            .apply(name + ".Project",
-                ParDo.of(new DoFn<KV<Auction, Person>, NameCityStateId>() {
-                      @ProcessElement
-                      public void processElement(ProcessContext c) {
-                        Auction auction = c.element().getKey();
-                        Person person = c.element().getValue();
-                        c.output(new NameCityStateId(
-                            person.name, person.city, person.state, auction.id));
-                      }
-                    }));
-  }
-
-  @Override
-  protected PCollection<KnownSize> applyPrim(PCollection<Event> events) {
-    return NexmarkUtils.castToKnownSize(name, applyTyped(events));
-  }
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/a7f9f7d0/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/Query3Model.java
----------------------------------------------------------------------
diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/Query3Model.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/Query3Model.java
deleted file mode 100644
index 85796ee..0000000
--- a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/Query3Model.java
+++ /dev/null
@@ -1,118 +0,0 @@
-/*
- * 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.beam.integration.nexmark;
-
-import com.google.common.collect.ArrayListMultimap;
-import com.google.common.collect.Multimap;
-
-import java.io.Serializable;
-import java.util.Collection;
-import java.util.HashMap;
-import java.util.Iterator;
-import java.util.Map;
-
-import org.apache.beam.sdk.values.TimestampedValue;
-import org.joda.time.Instant;
-
-/**
- * A direct implementation of {@link Query3}.
- */
-public class Query3Model extends NexmarkQueryModel implements Serializable {
-  /**
-   * Simulator for query 3.
-   */
-  private class Simulator extends AbstractSimulator<Event, NameCityStateId> {
-    /** Auctions, indexed by seller id. */
-    private final Multimap<Long, Auction> newAuctions;
-
-    /** Persons, indexed by id. */
-    private final Map<Long, Person> newPersons;
-
-    public Simulator(NexmarkConfiguration configuration) {
-      super(NexmarkUtils.standardEventIterator(configuration));
-      newPersons = new HashMap<>();
-      newAuctions = ArrayListMultimap.create();
-    }
-
-    /**
-     * Capture new result.
-     */
-    private void addResult(Auction auction, Person person, Instant timestamp) {
-      TimestampedValue<NameCityStateId> result = TimestampedValue.of(
-          new NameCityStateId(person.name, person.city, person.state, auction.id), timestamp);
-      addResult(result);
-    }
-
-    @Override
-    protected void run() {
-      TimestampedValue<Event> timestampedEvent = nextInput();
-      if (timestampedEvent == null) {
-        allDone();
-        return;
-      }
-      Event event = timestampedEvent.getValue();
-      if (event.bid != null) {
-        // Ignore bid events.
-        return;
-      }
-
-      Instant timestamp = timestampedEvent.getTimestamp();
-
-      if (event.newAuction != null) {
-        // Only want auctions in category 10.
-        if (event.newAuction.category == 10) {
-          // Join new auction with existing person, if any.
-          Person person = newPersons.get(event.newAuction.seller);
-          if (person != null) {
-            addResult(event.newAuction, person, timestamp);
-          } else {
-            // Remember auction for future new person event.
-            newAuctions.put(event.newAuction.seller, event.newAuction);
-          }
-        }
-      } else {
-        // Only want people in OR, ID or CA.
-        if (event.newPerson.state.equals("OR") || event.newPerson.state.equals("ID")
-            || event.newPerson.state.equals("CA")) {
-          // Join new person with existing auctions.
-          for (Auction auction : newAuctions.get(event.newPerson.id)) {
-            addResult(auction, event.newPerson, timestamp);
-          }
-          // We'll never need these auctions again.
-          newAuctions.removeAll(event.newPerson.id);
-          // Remember person for future auctions.
-          newPersons.put(event.newPerson.id, event.newPerson);
-        }
-      }
-    }
-  }
-
-  public Query3Model(NexmarkConfiguration configuration) {
-    super(configuration);
-  }
-
-  @Override
-  public AbstractSimulator<?, ?> simulator() {
-    return new Simulator(configuration);
-  }
-
-  @Override
-  protected <T> Collection<String> toCollection(Iterator<TimestampedValue<T>> itr) {
-    return toValue(itr);
-  }
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/a7f9f7d0/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/Query4.java
----------------------------------------------------------------------
diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/Query4.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/Query4.java
deleted file mode 100644
index b24410d..0000000
--- a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/Query4.java
+++ /dev/null
@@ -1,107 +0,0 @@
-/*
- * 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.beam.integration.nexmark;
-
-import org.apache.beam.sdk.transforms.DoFn;
-import org.apache.beam.sdk.transforms.Mean;
-import org.apache.beam.sdk.transforms.ParDo;
-import org.apache.beam.sdk.transforms.windowing.SlidingWindows;
-import org.apache.beam.sdk.transforms.windowing.Window;
-import org.apache.beam.sdk.values.KV;
-import org.apache.beam.sdk.values.PCollection;
-import org.joda.time.Duration;
-
-/**
- * Query 4, 'Average Price for a Category'. Select the average of the wining bid prices for all
- * closed auctions in each category. In CQL syntax:
- *
- * <pre>{@code
- * SELECT Istream(AVG(Q.final))
- * FROM Category C, (SELECT Rstream(MAX(B.price) AS final, A.category)
- *                   FROM Auction A [ROWS UNBOUNDED], Bid B [ROWS UNBOUNDED]
- *                   WHERE A.id=B.auction AND B.datetime < A.expires AND A.expires < CURRENT_TIME
- *                   GROUP BY A.id, A.category) Q
- * WHERE Q.category = C.id
- * GROUP BY C.id;
- * }</pre>
- *
- * <p>For extra spiciness our implementation differs slightly from the above:
- * <ul>
- * <li>We select both the average winning price and the category.
- * <li>We don't bother joining with a static category table, since it's contents are never used.
- * <li>We only consider bids which are above the auction's reserve price.
- * <li>We accept the highest-price, earliest valid bid as the winner.
- * <li>We calculate the averages oven a sliding window of size {@code windowSizeSec} and
- * period {@code windowPeriodSec}.
- * </ul>
- */
-class Query4 extends NexmarkQuery {
-  private final Monitor<AuctionBid> winningBidsMonitor;
-
-  public Query4(NexmarkConfiguration configuration) {
-    super(configuration, "Query4");
-    winningBidsMonitor = new Monitor<>(name + ".WinningBids", "winning");
-  }
-
-  private PCollection<CategoryPrice> applyTyped(PCollection<Event> events) {
-    PCollection<AuctionBid> winningBids =
-        events
-            // Find the winning bid for each closed auction.
-            .apply(new WinningBids(name + ".WinningBids", configuration));
-
-    // Monitor winning bids
-    winningBids = winningBids.apply(name + ".WinningBidsMonitor",
-            winningBidsMonitor.getTransform());
-
-    return winningBids
-        // Key the winning bid price by the auction category.
-        .apply(name + ".Rekey",
-            ParDo.of(new DoFn<AuctionBid, KV<Long, Long>>() {
-                  @ProcessElement
-                  public void processElement(ProcessContext c) {
-                    Auction auction = c.element().auction;
-                    Bid bid = c.element().bid;
-                    c.output(KV.of(auction.category, bid.price));
-                  }
-                }))
-
-        // Re-window so we can calculate a sliding average
-        .apply(Window.<KV<Long, Long>>into(
-            SlidingWindows.of(Duration.standardSeconds(configuration.windowSizeSec))
-                .every(Duration.standardSeconds(configuration.windowPeriodSec))))
-
-        // Find the average of the winning bids for each category.
-        // Make sure we share the work for each category between workers.
-        .apply(Mean.<Long, Long>perKey().withHotKeyFanout(configuration.fanout))
-
-        // For testing against Query4Model, capture which results are 'final'.
-        .apply(name + ".Project",
-            ParDo.of(new DoFn<KV<Long, Double>, CategoryPrice>() {
-                  @ProcessElement
-                  public void processElement(ProcessContext c) {
-                    c.output(new CategoryPrice(c.element().getKey(),
-                        Math.round(c.element().getValue()), c.pane().isLast()));
-                  }
-                }));
-  }
-
-  @Override
-  protected PCollection<KnownSize> applyPrim(PCollection<Event> events) {
-    return NexmarkUtils.castToKnownSize(name, applyTyped(events));
-  }
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/a7f9f7d0/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/Query4Model.java
----------------------------------------------------------------------
diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/Query4Model.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/Query4Model.java
deleted file mode 100644
index afab7e8..0000000
--- a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/Query4Model.java
+++ /dev/null
@@ -1,179 +0,0 @@
-/*
- * 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.beam.integration.nexmark;
-
-import java.io.Serializable;
-import java.util.ArrayList;
-import java.util.Collection;
-import java.util.Iterator;
-import java.util.List;
-import java.util.Map;
-import java.util.TreeMap;
-
-import org.apache.beam.sdk.transforms.windowing.BoundedWindow;
-import org.apache.beam.sdk.values.TimestampedValue;
-import org.joda.time.Duration;
-import org.joda.time.Instant;
-import org.junit.Assert;
-
-/**
- * A direct implementation of {@link Query4}.
- */
-public class Query4Model extends NexmarkQueryModel implements Serializable {
-  /**
-   * Simulator for query 4.
-   */
-  private class Simulator extends AbstractSimulator<AuctionBid, CategoryPrice> {
-    /** The prices and categories for all winning bids in the last window size. */
-    private final List<TimestampedValue<CategoryPrice>> winningPricesByCategory;
-
-    /** Timestamp of last result (ms since epoch). */
-    private Instant lastTimestamp;
-
-    /** When oldest active window starts. */
-    private Instant windowStart;
-
-    /** The last seen result for each category. */
-    private final Map<Long, TimestampedValue<CategoryPrice>> lastSeenResults;
-
-    public Simulator(NexmarkConfiguration configuration) {
-      super(new WinningBidsSimulator(configuration).results());
-      winningPricesByCategory = new ArrayList<>();
-      lastTimestamp = BoundedWindow.TIMESTAMP_MIN_VALUE;
-      windowStart = NexmarkUtils.BEGINNING_OF_TIME;
-      lastSeenResults = new TreeMap<>();
-    }
-
-    /**
-     * Calculate the average bid price for each category for all winning bids
-     * which are strictly before {@code end}.
-     */
-    private void averages(Instant end) {
-      Map<Long, Long> counts = new TreeMap<>();
-      Map<Long, Long> totals = new TreeMap<>();
-      for (TimestampedValue<CategoryPrice> value : winningPricesByCategory) {
-        if (!value.getTimestamp().isBefore(end)) {
-          continue;
-        }
-        long category = value.getValue().category;
-        long price = value.getValue().price;
-        Long count = counts.get(category);
-        if (count == null) {
-          count = 1L;
-        } else {
-          count += 1;
-        }
-        counts.put(category, count);
-        Long total = totals.get(category);
-        if (total == null) {
-          total = price;
-        } else {
-          total += price;
-        }
-        totals.put(category, total);
-      }
-      for (long category : counts.keySet()) {
-        long count = counts.get(category);
-        long total = totals.get(category);
-        TimestampedValue<CategoryPrice> result = TimestampedValue.of(
-            new CategoryPrice(category, Math.round((double) total / count), true), lastTimestamp);
-        addIntermediateResult(result);
-        lastSeenResults.put(category, result);
-      }
-    }
-
-    /**
-     * Calculate averages for any windows which can now be retired. Also prune entries
-     * which can no longer contribute to any future window.
-     */
-    private void prune(Instant newWindowStart) {
-      while (!newWindowStart.equals(windowStart)) {
-        averages(windowStart.plus(Duration.standardSeconds(configuration.windowSizeSec)));
-        windowStart = windowStart.plus(Duration.standardSeconds(configuration.windowPeriodSec));
-        Iterator<TimestampedValue<CategoryPrice>> itr = winningPricesByCategory.iterator();
-        while (itr.hasNext()) {
-          if (itr.next().getTimestamp().isBefore(windowStart)) {
-            itr.remove();
-          }
-        }
-        if (winningPricesByCategory.isEmpty()) {
-          windowStart = newWindowStart;
-        }
-      }
-    }
-
-    /**
-     * Capture the winning bid.
-     */
-    private void captureWinningBid(Auction auction, Bid bid, Instant timestamp) {
-      winningPricesByCategory.add(
-          TimestampedValue.of(new CategoryPrice(auction.category, bid.price, false), timestamp));
-    }
-
-    @Override
-    protected void run() {
-      TimestampedValue<AuctionBid> timestampedWinningBid = nextInput();
-      if (timestampedWinningBid == null) {
-        prune(NexmarkUtils.END_OF_TIME);
-        for (TimestampedValue<CategoryPrice> result : lastSeenResults.values()) {
-          addResult(result);
-        }
-        allDone();
-        return;
-      }
-      lastTimestamp = timestampedWinningBid.getTimestamp();
-      Instant newWindowStart = windowStart(Duration.standardSeconds(configuration.windowSizeSec),
-          Duration.standardSeconds(configuration.windowPeriodSec), lastTimestamp);
-      prune(newWindowStart);
-      captureWinningBid(timestampedWinningBid.getValue().auction,
-          timestampedWinningBid.getValue().bid, lastTimestamp);
-    }
-  }
-
-  public Query4Model(NexmarkConfiguration configuration) {
-    super(configuration);
-  }
-
-  @Override
-  public AbstractSimulator<?, ?> simulator() {
-    return new Simulator(configuration);
-  }
-
-  @Override
-  protected Iterable<TimestampedValue<KnownSize>> relevantResults(
-      Iterable<TimestampedValue<KnownSize>> results) {
-    // Find the last (in processing time) reported average price for each category.
-    Map<Long, TimestampedValue<KnownSize>> finalAverages = new TreeMap<>();
-    for (TimestampedValue<KnownSize> obj : results) {
-      Assert.assertTrue("have CategoryPrice", obj.getValue() instanceof CategoryPrice);
-      CategoryPrice categoryPrice = (CategoryPrice) obj.getValue();
-      if (categoryPrice.isLast) {
-        finalAverages.put(
-            categoryPrice.category,
-            TimestampedValue.of((KnownSize) categoryPrice, obj.getTimestamp()));
-      }
-    }
-
-    return finalAverages.values();
-  }
-
-  @Override
-  protected <T> Collection<String> toCollection(Iterator<TimestampedValue<T>> itr) {
-    return toValue(itr);
-  }
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/a7f9f7d0/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/Query5.java
----------------------------------------------------------------------
diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/Query5.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/Query5.java
deleted file mode 100644
index 2c9fb9b..0000000
--- a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/Query5.java
+++ /dev/null
@@ -1,123 +0,0 @@
-/*
- * 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.beam.integration.nexmark;
-
-import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.List;
-
-import org.apache.beam.sdk.transforms.Combine;
-import org.apache.beam.sdk.transforms.Count;
-import org.apache.beam.sdk.transforms.DoFn;
-import org.apache.beam.sdk.transforms.ParDo;
-import org.apache.beam.sdk.transforms.windowing.SlidingWindows;
-import org.apache.beam.sdk.transforms.windowing.Window;
-import org.apache.beam.sdk.values.KV;
-import org.apache.beam.sdk.values.PCollection;
-import org.joda.time.Duration;
-
-/**
- * Query 5, 'Hot Items'. Which auctions have seen the most bids in the last hour (updated every
- * minute). In CQL syntax:
- *
- * <pre>{@code
- * SELECT Rstream(auction)
- * FROM (SELECT B1.auction, count(*) AS num
- *       FROM Bid [RANGE 60 MINUTE SLIDE 1 MINUTE] B1
- *       GROUP BY B1.auction)
- * WHERE num >= ALL (SELECT count(*)
- *                   FROM Bid [RANGE 60 MINUTE SLIDE 1 MINUTE] B2
- *                   GROUP BY B2.auction);
- * }</pre>
- *
- * <p>To make things a bit more dynamic and easier to test we use much shorter windows, and
- * we'll also preserve the bid counts.
- */
-class Query5 extends NexmarkQuery {
-  public Query5(NexmarkConfiguration configuration) {
-    super(configuration, "Query5");
-  }
-
-  private PCollection<AuctionCount> applyTyped(PCollection<Event> events) {
-    return events
-        // Only want the bid events.
-        .apply(JUST_BIDS)
-        // Window the bids into sliding windows.
-        .apply(Window.<Bid>into(
-            SlidingWindows.of(Duration.standardSeconds(configuration.windowSizeSec))
-                .every(Duration.standardSeconds(configuration.windowPeriodSec))))
-        // Project just the auction id.
-        .apply("BidToAuction", BID_TO_AUCTION)
-
-        // Count the number of bids per auction id.
-        .apply(Count.<Long>perElement())
-
-      // We'll want to keep all auctions with the maximal number of bids.
-        // Start by lifting each into a singleton list.
-        .apply(name + ".ToSingletons",
-            ParDo.of(new DoFn<KV<Long, Long>, KV<List<Long>, Long>>() {
-                  @ProcessElement
-                  public void processElement(ProcessContext c) {
-                    c.output(KV.of(Arrays.asList(c.element().getKey()), c.element().getValue()));
-                  }
-                }))
-
-        // Keep only the auction ids with the most bids.
-        .apply(
-            Combine
-                .globally(new Combine.BinaryCombineFn<KV<List<Long>, Long>>() {
-                  @Override
-                  public KV<List<Long>, Long> apply(
-                      KV<List<Long>, Long> left, KV<List<Long>, Long> right) {
-                    List<Long> leftBestAuctions = left.getKey();
-                    long leftCount = left.getValue();
-                    List<Long> rightBestAuctions = right.getKey();
-                    long rightCount = right.getValue();
-                    if (leftCount > rightCount) {
-                      return left;
-                    } else if (leftCount < rightCount) {
-                      return right;
-                    } else {
-                      List<Long> newBestAuctions = new ArrayList<>();
-                      newBestAuctions.addAll(leftBestAuctions);
-                      newBestAuctions.addAll(rightBestAuctions);
-                      return KV.of(newBestAuctions, leftCount);
-                    }
-                  }
-                })
-                .withoutDefaults()
-                .withFanout(configuration.fanout))
-
-        // Project into result.
-        .apply(name + ".Select",
-            ParDo.of(new DoFn<KV<List<Long>, Long>, AuctionCount>() {
-                  @ProcessElement
-                  public void processElement(ProcessContext c) {
-                    long count = c.element().getValue();
-                    for (long auction : c.element().getKey()) {
-                      c.output(new AuctionCount(auction, count));
-                    }
-                  }
-                }));
-  }
-
-  @Override
-  protected PCollection<KnownSize> applyPrim(PCollection<Event> events) {
-    return NexmarkUtils.castToKnownSize(name, applyTyped(events));
-  }
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/a7f9f7d0/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/Query5Model.java
----------------------------------------------------------------------
diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/Query5Model.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/Query5Model.java
deleted file mode 100644
index f8e466e..0000000
--- a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/Query5Model.java
+++ /dev/null
@@ -1,172 +0,0 @@
-/*
- * 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.beam.integration.nexmark;
-
-import java.io.Serializable;
-import java.util.ArrayList;
-import java.util.Collection;
-import java.util.Iterator;
-import java.util.List;
-import java.util.Map;
-import java.util.TreeMap;
-
-import org.apache.beam.sdk.values.TimestampedValue;
-import org.joda.time.Duration;
-import org.joda.time.Instant;
-
-/**
- * A direct implementation of {@link Query5}.
- */
-public class Query5Model extends NexmarkQueryModel implements Serializable {
-  /**
-   * Simulator for query 5.
-   */
-  private class Simulator extends AbstractSimulator<Event, AuctionCount> {
-    /** Time of bids still contributing to open windows, indexed by their auction id. */
-    private final Map<Long, List<Instant>> bids;
-
-    /** When oldest active window starts. */
-    private Instant windowStart;
-
-    public Simulator(NexmarkConfiguration configuration) {
-      super(NexmarkUtils.standardEventIterator(configuration));
-      bids = new TreeMap<>();
-      windowStart = NexmarkUtils.BEGINNING_OF_TIME;
-    }
-
-    /**
-     * Count bids per auction id for bids strictly before {@code end}. Add the auction ids with
-     * the maximum number of bids to results.
-     */
-    private void countBids(Instant end) {
-      Map<Long, Long> counts = new TreeMap<>();
-      long maxCount = 0L;
-      for (Map.Entry<Long, List<Instant>> entry : bids.entrySet()) {
-        long count = 0L;
-        long auction = entry.getKey();
-        for (Instant bid : entry.getValue()) {
-          if (bid.isBefore(end)) {
-            count++;
-          }
-        }
-        if (count > 0) {
-          counts.put(auction, count);
-          maxCount = Math.max(maxCount, count);
-        }
-      }
-      for (Map.Entry<Long, Long> entry : counts.entrySet()) {
-        long auction = entry.getKey();
-        long count = entry.getValue();
-        if (count == maxCount) {
-          AuctionCount result = new AuctionCount(auction, count);
-          addResult(TimestampedValue.of(result, end));
-        }
-      }
-    }
-
-    /**
-     * Retire bids which are strictly before {@code cutoff}. Return true if there are any bids
-     * remaining.
-     */
-    private boolean retireBids(Instant cutoff) {
-      boolean anyRemain = false;
-      for (Map.Entry<Long, List<Instant>> entry : bids.entrySet()) {
-        long auction = entry.getKey();
-        Iterator<Instant> itr = entry.getValue().iterator();
-        while (itr.hasNext()) {
-          Instant bid = itr.next();
-          if (bid.isBefore(cutoff)) {
-            NexmarkUtils.info("retire: %s for %s", bid, auction);
-            itr.remove();
-          } else {
-            anyRemain = true;
-          }
-        }
-      }
-      return anyRemain;
-    }
-
-    /**
-     * Retire active windows until we've reached {@code newWindowStart}.
-     */
-    private void retireWindows(Instant newWindowStart) {
-      while (!newWindowStart.equals(windowStart)) {
-        NexmarkUtils.info("retiring window %s, aiming for %s", windowStart, newWindowStart);
-        // Count bids in the window (windowStart, windowStart + size].
-        countBids(windowStart.plus(Duration.standardSeconds(configuration.windowSizeSec)));
-        // Advance the window.
-        windowStart = windowStart.plus(Duration.standardSeconds(configuration.windowPeriodSec));
-        // Retire bids which will never contribute to a future window.
-        if (!retireBids(windowStart)) {
-          // Can fast forward to latest window since no more outstanding bids.
-          windowStart = newWindowStart;
-        }
-      }
-    }
-
-    /**
-     * Add bid to state.
-     */
-    private void captureBid(Bid bid, Instant timestamp) {
-      List<Instant> existing = bids.get(bid.auction);
-      if (existing == null) {
-        existing = new ArrayList<>();
-        bids.put(bid.auction, existing);
-      }
-      existing.add(timestamp);
-    }
-
-    @Override
-    public void run() {
-      TimestampedValue<Event> timestampedEvent = nextInput();
-      if (timestampedEvent == null) {
-        // Drain the remaining windows.
-        retireWindows(NexmarkUtils.END_OF_TIME);
-        allDone();
-        return;
-      }
-
-      Event event = timestampedEvent.getValue();
-      if (event.bid == null) {
-        // Ignore non-bid events.
-        return;
-      }
-      Instant timestamp = timestampedEvent.getTimestamp();
-      Instant newWindowStart = windowStart(Duration.standardSeconds(configuration.windowSizeSec),
-          Duration.standardSeconds(configuration.windowPeriodSec), timestamp);
-      // Capture results from any windows we can now retire.
-      retireWindows(newWindowStart);
-      // Capture current bid.
-      captureBid(event.bid, timestamp);
-    }
-  }
-
-  public Query5Model(NexmarkConfiguration configuration) {
-    super(configuration);
-  }
-
-  @Override
-  public AbstractSimulator<?, ?> simulator() {
-    return new Simulator(configuration);
-  }
-
-  @Override
-  protected <T> Collection<String> toCollection(Iterator<TimestampedValue<T>> itr) {
-    return toValue(itr);
-  }
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/a7f9f7d0/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/Query6.java
----------------------------------------------------------------------
diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/Query6.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/Query6.java
deleted file mode 100644
index d5bcc30..0000000
--- a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/Query6.java
+++ /dev/null
@@ -1,151 +0,0 @@
-/*
- * 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.beam.integration.nexmark;
-
-import com.google.common.collect.Lists;
-
-import java.util.ArrayList;
-import java.util.Collections;
-import java.util.List;
-
-import org.apache.beam.sdk.transforms.Combine;
-import org.apache.beam.sdk.transforms.DoFn;
-import org.apache.beam.sdk.transforms.ParDo;
-import org.apache.beam.sdk.transforms.windowing.AfterPane;
-import org.apache.beam.sdk.transforms.windowing.GlobalWindows;
-import org.apache.beam.sdk.transforms.windowing.Repeatedly;
-import org.apache.beam.sdk.transforms.windowing.Window;
-import org.apache.beam.sdk.values.KV;
-import org.apache.beam.sdk.values.PCollection;
-import org.joda.time.Duration;
-
-/**
- * Query 6, 'Average Selling Price by Seller'. Select the average selling price over the
- * last 10 closed auctions by the same seller. In CQL syntax:
- *
- * <pre>{@code
- * SELECT Istream(AVG(Q.final), Q.seller)
- * FROM (SELECT Rstream(MAX(B.price) AS final, A.seller)
- *       FROM Auction A [ROWS UNBOUNDED], Bid B [ROWS UNBOUNDED]
- *       WHERE A.id=B.auction AND B.datetime < A.expires AND A.expires < CURRENT_TIME
- *       GROUP BY A.id, A.seller) [PARTITION BY A.seller ROWS 10] Q
- * GROUP BY Q.seller;
- * }</pre>
- *
- * <p>We are a little more exact with selecting winning bids: see {@link WinningBids}.
- */
-class Query6 extends NexmarkQuery {
-  /**
-   * Combiner to keep track of up to {@code maxNumBids} of the most recent wining bids and calculate
-   * their average selling price.
-   */
-  private static class MovingMeanSellingPrice extends Combine.CombineFn<Bid, List<Bid>, Long> {
-    private final int maxNumBids;
-
-    public MovingMeanSellingPrice(int maxNumBids) {
-      this.maxNumBids = maxNumBids;
-    }
-
-    @Override
-    public List<Bid> createAccumulator() {
-      return new ArrayList<>();
-    }
-
-    @Override
-    public List<Bid> addInput(List<Bid> accumulator, Bid input) {
-      accumulator.add(input);
-      Collections.sort(accumulator, Bid.ASCENDING_TIME_THEN_PRICE);
-      if (accumulator.size() > maxNumBids) {
-        accumulator.remove(0);
-      }
-      return accumulator;
-    }
-
-    @Override
-    public List<Bid> mergeAccumulators(Iterable<List<Bid>> accumulators) {
-      List<Bid> result = new ArrayList<>();
-      for (List<Bid> accumulator : accumulators) {
-        for (Bid bid : accumulator) {
-          result.add(bid);
-        }
-      }
-      Collections.sort(result, Bid.ASCENDING_TIME_THEN_PRICE);
-      if (result.size() > maxNumBids) {
-        result = Lists.newArrayList(result.listIterator(result.size() - maxNumBids));
-      }
-      return result;
-    }
-
-    @Override
-    public Long extractOutput(List<Bid> accumulator) {
-      if (accumulator.isEmpty()) {
-        return 0L;
-      }
-      long sumOfPrice = 0;
-      for (Bid bid : accumulator) {
-        sumOfPrice += bid.price;
-      }
-      return Math.round((double) sumOfPrice / accumulator.size());
-    }
-  }
-
-  public Query6(NexmarkConfiguration configuration) {
-    super(configuration, "Query6");
-  }
-
-  private PCollection<SellerPrice> applyTyped(PCollection<Event> events) {
-    return events
-        // Find the winning bid for each closed auction.
-        .apply(new WinningBids(name + ".WinningBids", configuration))
-
-        // Key the winning bid by the seller id.
-        .apply(name + ".Rekey",
-            ParDo.of(new DoFn<AuctionBid, KV<Long, Bid>>() {
-                  @ProcessElement
-                  public void processElement(ProcessContext c) {
-                    Auction auction = c.element().auction;
-                    Bid bid = c.element().bid;
-                    c.output(KV.of(auction.seller, bid));
-                  }
-                }))
-
-        // Re-window to update on every wining bid.
-        .apply(
-            Window.<KV<Long, Bid>>into(new GlobalWindows())
-                .triggering(Repeatedly.forever(AfterPane.elementCountAtLeast(1)))
-                .accumulatingFiredPanes()
-                .withAllowedLateness(Duration.ZERO))
-
-        // Find the average of last 10 winning bids for each seller.
-        .apply(Combine.<Long, Bid, Long>perKey(new MovingMeanSellingPrice(10)))
-
-        // Project into our datatype.
-        .apply(name + ".Select",
-            ParDo.of(new DoFn<KV<Long, Long>, SellerPrice>() {
-                  @ProcessElement
-                  public void processElement(ProcessContext c) {
-                    c.output(new SellerPrice(c.element().getKey(), c.element().getValue()));
-                  }
-                }));
-  }
-
-  @Override
-  protected PCollection<KnownSize> applyPrim(PCollection<Event> events) {
-    return NexmarkUtils.castToKnownSize(name, applyTyped(events));
-  }
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/a7f9f7d0/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/Query6Model.java
----------------------------------------------------------------------
diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/Query6Model.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/Query6Model.java
deleted file mode 100644
index d03f0fe..0000000
--- a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/Query6Model.java
+++ /dev/null
@@ -1,126 +0,0 @@
-/*
- * 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.beam.integration.nexmark;
-
-import java.io.Serializable;
-import java.util.Collection;
-import java.util.Iterator;
-import java.util.Map;
-import java.util.TreeMap;
-
-import org.apache.beam.sdk.transforms.windowing.BoundedWindow;
-import org.apache.beam.sdk.values.TimestampedValue;
-import org.joda.time.Instant;
-import org.junit.Assert;
-
-/**
- * A direct implementation of {@link Query6}.
- */
-public class Query6Model extends NexmarkQueryModel implements Serializable {
-  /**
-   * Simulator for query 6.
-   */
-  private static class Simulator extends AbstractSimulator<AuctionBid, SellerPrice> {
-    /** The cumulative count of winning bids, indexed by seller id. */
-    private final Map<Long, Long> numWinningBidsPerSeller;
-
-    /** The cumulative total of winning bid prices, indexed by seller id. */
-    private final Map<Long, Long> totalWinningBidPricesPerSeller;
-
-    private Instant lastTimestamp;
-
-    public Simulator(NexmarkConfiguration configuration) {
-      super(new WinningBidsSimulator(configuration).results());
-      numWinningBidsPerSeller = new TreeMap<>();
-      totalWinningBidPricesPerSeller = new TreeMap<>();
-      lastTimestamp = BoundedWindow.TIMESTAMP_MIN_VALUE;
-    }
-
-    /**
-     * Update the per-seller running counts/sums.
-     */
-    private void captureWinningBid(Auction auction, Bid bid, Instant timestamp) {
-      NexmarkUtils.info("winning auction, bid: %s, %s", auction, bid);
-      Long count = numWinningBidsPerSeller.get(auction.seller);
-      if (count == null) {
-        count = 1L;
-      } else {
-        count += 1;
-      }
-      numWinningBidsPerSeller.put(auction.seller, count);
-      Long total = totalWinningBidPricesPerSeller.get(auction.seller);
-      if (total == null) {
-        total = bid.price;
-      } else {
-        total += bid.price;
-      }
-      totalWinningBidPricesPerSeller.put(auction.seller, total);
-      TimestampedValue<SellerPrice> intermediateResult = TimestampedValue.of(
-          new SellerPrice(auction.seller, Math.round((double) total / count)), timestamp);
-      addIntermediateResult(intermediateResult);
-    }
-
-
-    @Override
-    protected void run() {
-      TimestampedValue<AuctionBid> timestampedWinningBid = nextInput();
-      if (timestampedWinningBid == null) {
-        for (long seller : numWinningBidsPerSeller.keySet()) {
-          long count = numWinningBidsPerSeller.get(seller);
-          long total = totalWinningBidPricesPerSeller.get(seller);
-          addResult(TimestampedValue.of(
-              new SellerPrice(seller, Math.round((double) total / count)), lastTimestamp));
-        }
-        allDone();
-        return;
-      }
-
-      lastTimestamp = timestampedWinningBid.getTimestamp();
-      captureWinningBid(timestampedWinningBid.getValue().auction,
-          timestampedWinningBid.getValue().bid, lastTimestamp);
-    }
-  }
-
-  public Query6Model(NexmarkConfiguration configuration) {
-    super(configuration);
-  }
-
-  @Override
-  public AbstractSimulator<?, ?> simulator() {
-    return new Simulator(configuration);
-  }
-
-  @Override
-  protected Iterable<TimestampedValue<KnownSize>> relevantResults(
-      Iterable<TimestampedValue<KnownSize>> results) {
-    // Find the last (in processing time) reported average price for each seller.
-    Map<Long, TimestampedValue<KnownSize>> finalAverages = new TreeMap<>();
-    for (TimestampedValue<KnownSize> obj : results) {
-      Assert.assertTrue("have SellerPrice", obj.getValue() instanceof SellerPrice);
-      SellerPrice sellerPrice = (SellerPrice) obj.getValue();
-      finalAverages.put(
-          sellerPrice.seller, TimestampedValue.of((KnownSize) sellerPrice, obj.getTimestamp()));
-    }
-    return finalAverages.values();
-  }
-
-  @Override
-  protected <T> Collection<String> toCollection(Iterator<TimestampedValue<T>> itr) {
-    return toValue(itr);
-  }
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/a7f9f7d0/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/Query7.java
----------------------------------------------------------------------
diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/Query7.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/Query7.java
deleted file mode 100644
index 7c51c18..0000000
--- a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/Query7.java
+++ /dev/null
@@ -1,85 +0,0 @@
-/*
- * 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.beam.integration.nexmark;
-
-import org.apache.beam.sdk.transforms.DoFn;
-import org.apache.beam.sdk.transforms.Max;
-import org.apache.beam.sdk.transforms.ParDo;
-import org.apache.beam.sdk.transforms.windowing.FixedWindows;
-import org.apache.beam.sdk.transforms.windowing.Window;
-import org.apache.beam.sdk.values.PCollection;
-import org.apache.beam.sdk.values.PCollectionView;
-
-import org.joda.time.Duration;
-
-/**
- * Query 7, 'Highest Bid'. Select the bids with the highest bid
- * price in the last minute. In CQL syntax:
- *
- * <pre>
- * SELECT Rstream(B.auction, B.price, B.bidder)
- * FROM Bid [RANGE 1 MINUTE SLIDE 1 MINUTE] B
- * WHERE B.price = (SELECT MAX(B1.price)
- *                  FROM BID [RANGE 1 MINUTE SLIDE 1 MINUTE] B1);
- * </pre>
- *
- * <p>We will use a shorter window to help make testing easier. We'll also implement this using
- * a side-input in order to exercise that functionality. (A combiner, as used in Query 5, is
- * a more efficient approach.).
- */
-class Query7 extends NexmarkQuery {
-  public Query7(NexmarkConfiguration configuration) {
-    super(configuration, "Query7");
-  }
-
-  private PCollection<Bid> applyTyped(PCollection<Event> events) {
-    // Window the bids.
-    PCollection<Bid> slidingBids = events.apply(JUST_BIDS).apply(
-        Window.<Bid>into(FixedWindows.of(Duration.standardSeconds(configuration.windowSizeSec))));
-
-    // Find the largest price in all bids.
-    // NOTE: It would be more efficient to write this query much as we did for Query5, using
-    // a binary combiner to accumulate the bids with maximal price. As written this query
-    // requires an additional scan per window, with the associated cost of snapshotted state and
-    // its I/O. We'll keep this implementation since it illustrates the use of side inputs.
-    final PCollectionView<Long> maxPriceView =
-        slidingBids //
-            .apply("BidToPrice", BID_TO_PRICE)
-            .apply(Max.longsGlobally().withFanout(configuration.fanout).asSingletonView());
-
-    return slidingBids
-        // Select all bids which have that maximum price (there may be more than one).
-        .apply(name + ".Select",
-            ParDo.withSideInputs(maxPriceView)
-                .of(new DoFn<Bid, Bid>() {
-                  @ProcessElement
-                  public void processElement(ProcessContext c) {
-                    long maxPrice = c.sideInput(maxPriceView);
-                    Bid bid = c.element();
-                    if (bid.price == maxPrice) {
-                      c.output(bid);
-                    }
-                  }
-                }));
-  }
-
-  @Override
-  protected PCollection<KnownSize> applyPrim(PCollection<Event> events) {
-    return NexmarkUtils.castToKnownSize(name, applyTyped(events));
-  }
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/a7f9f7d0/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/Query7Model.java
----------------------------------------------------------------------
diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/Query7Model.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/Query7Model.java
deleted file mode 100644
index 0033c68..0000000
--- a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/Query7Model.java
+++ /dev/null
@@ -1,127 +0,0 @@
-/*
- * 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.beam.integration.nexmark;
-
-import java.io.Serializable;
-import java.util.ArrayList;
-import java.util.Collection;
-import java.util.Iterator;
-import java.util.List;
-
-import org.apache.beam.sdk.transforms.windowing.BoundedWindow;
-import org.apache.beam.sdk.values.TimestampedValue;
-import org.joda.time.Duration;
-import org.joda.time.Instant;
-
-/**
- * A direct implementation of {@link Query7}.
- */
-public class Query7Model extends NexmarkQueryModel implements Serializable {
-  /**
-   * Simulator for query 7.
-   */
-  private class Simulator extends AbstractSimulator<Event, Bid> {
-    /** Bids with highest bid price seen in the current window. */
-    private final List<Bid> highestBids;
-
-    /** When current window started. */
-    private Instant windowStart;
-
-    private Instant lastTimestamp;
-
-    public Simulator(NexmarkConfiguration configuration) {
-      super(NexmarkUtils.standardEventIterator(configuration));
-      highestBids = new ArrayList<>();
-      windowStart = NexmarkUtils.BEGINNING_OF_TIME;
-      lastTimestamp = BoundedWindow.TIMESTAMP_MIN_VALUE;
-    }
-
-    /**
-     * Transfer the currently winning bids into results and retire them.
-     */
-    private void retireWindow(Instant timestamp) {
-      for (Bid bid : highestBids) {
-        addResult(TimestampedValue.of(bid, timestamp));
-      }
-      highestBids.clear();
-    }
-
-    /**
-     * Keep just the highest price bid.
-     */
-    private void captureBid(Bid bid) {
-      Iterator<Bid> itr = highestBids.iterator();
-      boolean isWinning = true;
-      while (itr.hasNext()) {
-        Bid existingBid = itr.next();
-        if (existingBid.price > bid.price) {
-          isWinning = false;
-          break;
-        }
-        NexmarkUtils.info("smaller price: %s", existingBid);
-        itr.remove();
-      }
-      if (isWinning) {
-        NexmarkUtils.info("larger price: %s", bid);
-        highestBids.add(bid);
-      }
-    }
-
-    @Override
-    protected void run() {
-      TimestampedValue<Event> timestampedEvent = nextInput();
-      if (timestampedEvent == null) {
-        // Capture all remaining bids in results.
-        retireWindow(lastTimestamp);
-        allDone();
-        return;
-      }
-
-      Event event = timestampedEvent.getValue();
-      if (event.bid == null) {
-        // Ignore non-bid events.
-        return;
-      }
-      lastTimestamp = timestampedEvent.getTimestamp();
-      Instant newWindowStart = windowStart(Duration.standardSeconds(configuration.windowSizeSec),
-          Duration.standardSeconds(configuration.windowSizeSec), lastTimestamp);
-      if (!newWindowStart.equals(windowStart)) {
-        // Capture highest priced bids in current window and retire it.
-        retireWindow(lastTimestamp);
-        windowStart = newWindowStart;
-      }
-      // Keep only the highest bids.
-      captureBid(event.bid);
-      //TODO test fails because offset of some hundreds of ms between expect and actual
-    }
-  }
-
-  public Query7Model(NexmarkConfiguration configuration) {
-    super(configuration);
-  }
-
-  @Override
-  public AbstractSimulator<?, ?> simulator() {
-    return new Simulator(configuration);
-  }
-
-  @Override
-  protected <T> Collection<String> toCollection(Iterator<TimestampedValue<T>> itr) {
-    return toValueOrder(itr);
-  }
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/a7f9f7d0/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/Query8.java
----------------------------------------------------------------------
diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/Query8.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/Query8.java
deleted file mode 100644
index ee5c26c..0000000
--- a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/Query8.java
+++ /dev/null
@@ -1,91 +0,0 @@
-/*
- * 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.beam.integration.nexmark;
-
-import org.apache.beam.sdk.transforms.DoFn;
-import org.apache.beam.sdk.transforms.ParDo;
-import org.apache.beam.sdk.transforms.join.CoGbkResult;
-import org.apache.beam.sdk.transforms.join.CoGroupByKey;
-import org.apache.beam.sdk.transforms.join.KeyedPCollectionTuple;
-import org.apache.beam.sdk.transforms.windowing.FixedWindows;
-import org.apache.beam.sdk.transforms.windowing.Window;
-import org.apache.beam.sdk.values.KV;
-import org.apache.beam.sdk.values.PCollection;
-
-import org.joda.time.Duration;
-
-/**
- * Query 8, 'Monitor New Users'. Select people who have entered the system and created auctions
- * in the last 12 hours, updated every 12 hours. In CQL syntax:
- *
- * <pre>
- * SELECT Rstream(P.id, P.name, A.reserve)
- * FROM Person [RANGE 12 HOUR] P, Auction [RANGE 12 HOUR] A
- * WHERE P.id = A.seller;
- * </pre>
- *
- * <p>To make things a bit more dynamic and easier to test we'll use a much shorter window.
- */
-class Query8 extends NexmarkQuery {
-  public Query8(NexmarkConfiguration configuration) {
-    super(configuration, "Query8");
-  }
-
-  private PCollection<IdNameReserve> applyTyped(PCollection<Event> events) {
-    // Window and key new people by their id.
-    PCollection<KV<Long, Person>> personsById =
-        events
-          .apply(JUST_NEW_PERSONS)
-          .apply("Query8.WindowPersons",
-            Window.<Person>into(
-              FixedWindows.of(Duration.standardSeconds(configuration.windowSizeSec))))
-            .apply("PersonById", PERSON_BY_ID);
-
-    // Window and key new auctions by their id.
-    PCollection<KV<Long, Auction>> auctionsBySeller =
-        events.apply(JUST_NEW_AUCTIONS)
-          .apply("Query8.WindowAuctions",
-            Window.<Auction>into(
-              FixedWindows.of(Duration.standardSeconds(configuration.windowSizeSec))))
-            .apply("AuctionBySeller", AUCTION_BY_SELLER);
-
-    // Join people and auctions and project the person id, name and auction reserve price.
-    return KeyedPCollectionTuple.of(PERSON_TAG, personsById)
-        .and(AUCTION_TAG, auctionsBySeller)
-        .apply(CoGroupByKey.<Long>create())
-        .apply(name + ".Select",
-            ParDo.of(new DoFn<KV<Long, CoGbkResult>, IdNameReserve>() {
-                  @ProcessElement
-                  public void processElement(ProcessContext c) {
-                    Person person = c.element().getValue().getOnly(PERSON_TAG, null);
-                    if (person == null) {
-                      // Person was not created in last window period.
-                      return;
-                    }
-                    for (Auction auction : c.element().getValue().getAll(AUCTION_TAG)) {
-                      c.output(new IdNameReserve(person.id, person.name, auction.reserve));
-                    }
-                  }
-                }));
-  }
-
-  @Override
-  protected PCollection<KnownSize> applyPrim(PCollection<Event> events) {
-    return NexmarkUtils.castToKnownSize(name, applyTyped(events));
-  }
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/a7f9f7d0/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/Query8Model.java
----------------------------------------------------------------------
diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/Query8Model.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/Query8Model.java
deleted file mode 100644
index 261e383..0000000
--- a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/Query8Model.java
+++ /dev/null
@@ -1,144 +0,0 @@
-/*
- * 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.beam.integration.nexmark;
-
-import com.google.common.collect.ArrayListMultimap;
-import com.google.common.collect.Multimap;
-
-import java.io.Serializable;
-import java.util.Collection;
-import java.util.HashMap;
-import java.util.Iterator;
-import java.util.Map;
-
-import org.apache.beam.sdk.values.TimestampedValue;
-import org.joda.time.Duration;
-import org.joda.time.Instant;
-
-/**
- * A direct implementation of {@link Query8}.
- */
-public class Query8Model extends NexmarkQueryModel implements Serializable {
-  /**
-   * Simulator for query 8.
-   */
-  private class Simulator extends AbstractSimulator<Event, IdNameReserve> {
-    /** New persons seen in the current window, indexed by id. */
-    private final Map<Long, Person> newPersons;
-
-    /** New auctions seen in the current window, indexed by seller id. */
-    private final Multimap<Long, Auction> newAuctions;
-
-    /** When did the current window start. */
-    private Instant windowStart;
-
-    public Simulator(NexmarkConfiguration configuration) {
-      super(NexmarkUtils.standardEventIterator(configuration));
-      newPersons = new HashMap<>();
-      newAuctions = ArrayListMultimap.create();
-      windowStart = NexmarkUtils.BEGINNING_OF_TIME;
-    }
-
-    /**
-     * Retire all persons added in last window.
-     */
-    private void retirePersons() {
-      for (Map.Entry<Long, Person> entry : newPersons.entrySet()) {
-        NexmarkUtils.info("retire: %s", entry.getValue());
-      }
-      newPersons.clear();
-    }
-
-    /**
-     * Retire all auctions added in last window.
-     */
-    private void retireAuctions() {
-      for (Map.Entry<Long, Auction> entry : newAuctions.entries()) {
-        NexmarkUtils.info("retire: %s", entry.getValue());
-      }
-      newAuctions.clear();
-    }
-
-    /**
-     * Capture new result.
-     */
-    private void addResult(Auction auction, Person person, Instant timestamp) {
-      addResult(TimestampedValue.of(
-          new IdNameReserve(person.id, person.name, auction.reserve), timestamp));
-    }
-
-    @Override
-    public void run() {
-      TimestampedValue<Event> timestampedEvent = nextInput();
-      if (timestampedEvent == null) {
-        allDone();
-        return;
-      }
-
-      Event event = timestampedEvent.getValue();
-      if (event.bid != null) {
-        // Ignore bid events.
-        // Keep looking for next events.
-        return;
-      }
-      Instant timestamp = timestampedEvent.getTimestamp();
-      Instant newWindowStart = windowStart(Duration.standardSeconds(configuration.windowSizeSec),
-          Duration.standardSeconds(configuration.windowSizeSec), timestamp);
-      if (!newWindowStart.equals(windowStart)) {
-        // Retire this window.
-        retirePersons();
-        retireAuctions();
-        windowStart = newWindowStart;
-      }
-
-      if (event.newAuction != null) {
-        // Join new auction with existing person, if any.
-        Person person = newPersons.get(event.newAuction.seller);
-        if (person != null) {
-          addResult(event.newAuction, person, timestamp);
-        } else {
-          // Remember auction for future new people.
-          newAuctions.put(event.newAuction.seller, event.newAuction);
-        }
-      } else { // event is not an auction, nor a bid, so it is a person
-        // Join new person with existing auctions.
-        for (Auction auction : newAuctions.get(event.newPerson.id)) {
-          addResult(auction, event.newPerson, timestamp);
-        }
-        // We'll never need these auctions again.
-        newAuctions.removeAll(event.newPerson.id);
-        // Remember person for future auctions.
-        newPersons.put(event.newPerson.id, event.newPerson);
-      }
-    }
-  }
-
-  public Query8Model(NexmarkConfiguration configuration) {
-    super(configuration);
-  }
-
-  @Override
-  public AbstractSimulator<?, ?> simulator() {
-    return new Simulator(configuration);
-  }
-
-  @Override
-  protected <T> Collection<String> toCollection(Iterator<TimestampedValue<T>> itr) {
-    return toValue(itr);
-  }
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/a7f9f7d0/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/Query9.java
----------------------------------------------------------------------
diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/Query9.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/Query9.java
deleted file mode 100644
index 64bf653..0000000
--- a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/Query9.java
+++ /dev/null
@@ -1,39 +0,0 @@
-/*
- * 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.beam.integration.nexmark;
-
-import org.apache.beam.sdk.values.PCollection;
-
-/**
- * Query "9", 'Winning bids'. Select just the winning bids. Not in original NEXMark suite, but
- * handy for testing. See {@link WinningBids} for the details.
- */
-class Query9 extends NexmarkQuery {
-  public Query9(NexmarkConfiguration configuration) {
-    super(configuration, "Query9");
-  }
-
-  private PCollection<AuctionBid> applyTyped(PCollection<Event> events) {
-    return events.apply(new WinningBids(name, configuration));
-  }
-
-  @Override
-  protected PCollection<KnownSize> applyPrim(PCollection<Event> events) {
-    return NexmarkUtils.castToKnownSize(name, applyTyped(events));
-  }
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/a7f9f7d0/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/Query9Model.java
----------------------------------------------------------------------
diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/Query9Model.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/Query9Model.java
deleted file mode 100644
index 338f02a..0000000
--- a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/Query9Model.java
+++ /dev/null
@@ -1,43 +0,0 @@
-/*
- * 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.beam.integration.nexmark;
-
-import java.io.Serializable;
-import java.util.Collection;
-import java.util.Iterator;
-
-import org.apache.beam.sdk.values.TimestampedValue;
-
-/**
- * A direct implementation of {@link Query9}.
- */
-public class Query9Model extends NexmarkQueryModel implements Serializable {
-  public Query9Model(NexmarkConfiguration configuration) {
-    super(configuration);
-  }
-
-  @Override
-  public AbstractSimulator<?, ?> simulator() {
-    return new WinningBidsSimulator(configuration);
-  }
-
-  @Override
-  protected <T> Collection<String> toCollection(Iterator<TimestampedValue<T>> itr) {
-    return toValue(itr);
-  }
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/a7f9f7d0/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/SellerPrice.java
----------------------------------------------------------------------
diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/SellerPrice.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/SellerPrice.java
deleted file mode 100644
index 4081287..0000000
--- a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/SellerPrice.java
+++ /dev/null
@@ -1,90 +0,0 @@
-/*
- * 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.beam.integration.nexmark;
-
-import com.fasterxml.jackson.annotation.JsonProperty;
-import com.fasterxml.jackson.core.JsonProcessingException;
-
-import java.io.IOException;
-import java.io.InputStream;
-import java.io.OutputStream;
-import java.io.Serializable;
-
-import org.apache.beam.sdk.coders.AtomicCoder;
-import org.apache.beam.sdk.coders.Coder;
-import org.apache.beam.sdk.coders.CoderException;
-import org.apache.beam.sdk.coders.VarLongCoder;
-
-/**
- * Result of {@link Query6}.
- */
-public class SellerPrice implements KnownSize, Serializable {
-  private static final Coder<Long> LONG_CODER = VarLongCoder.of();
-
-  public static final Coder<SellerPrice> CODER = new AtomicCoder<SellerPrice>() {
-    @Override
-    public void encode(SellerPrice value, OutputStream outStream,
-        Coder.Context context)
-        throws CoderException, IOException {
-      LONG_CODER.encode(value.seller, outStream, Context.NESTED);
-      LONG_CODER.encode(value.price, outStream, Context.NESTED);
-    }
-
-    @Override
-    public SellerPrice decode(
-        InputStream inStream, Coder.Context context)
-        throws CoderException, IOException {
-      long seller = LONG_CODER.decode(inStream, Context.NESTED);
-      long price = LONG_CODER.decode(inStream, Context.NESTED);
-      return new SellerPrice(seller, price);
-    }
-  };
-
-  @JsonProperty
-  public final long seller;
-
-  /** Price in cents. */
-  @JsonProperty
-  public final long price;
-
-  // For Avro only.
-  @SuppressWarnings("unused")
-  private SellerPrice() {
-    seller = 0;
-    price = 0;
-  }
-
-  public SellerPrice(long seller, long price) {
-    this.seller = seller;
-    this.price = price;
-  }
-
-  @Override
-  public long sizeInBytes() {
-    return 8 + 8;
-  }
-
-  @Override
-  public String toString() {
-    try {
-      return NexmarkUtils.MAPPER.writeValueAsString(this);
-    } catch (JsonProcessingException e) {
-      throw new RuntimeException(e);
-    }
-  }
-}


[51/55] [abbrv] beam git commit: Move module beam-integration-java-nexmark to beam-sdks-java-nexmark

Posted by ie...@apache.org.
http://git-wip-us.apache.org/repos/asf/beam/blob/f4333df7/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkLauncher.java
----------------------------------------------------------------------
diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkLauncher.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkLauncher.java
deleted file mode 100644
index a609975..0000000
--- a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkLauncher.java
+++ /dev/null
@@ -1,1158 +0,0 @@
-/*
- * 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.beam.integration.nexmark;
-
-import static com.google.common.base.Preconditions.checkState;
-
-import com.google.api.services.bigquery.model.TableFieldSchema;
-import com.google.api.services.bigquery.model.TableRow;
-import com.google.api.services.bigquery.model.TableSchema;
-import com.google.common.base.Strings;
-import com.google.common.collect.ImmutableList;
-import com.google.common.collect.Lists;
-import java.io.IOException;
-import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.Collections;
-import java.util.HashMap;
-import java.util.List;
-import java.util.NoSuchElementException;
-import java.util.concurrent.ThreadLocalRandom;
-import javax.annotation.Nullable;
-import org.apache.beam.integration.nexmark.model.Auction;
-import org.apache.beam.integration.nexmark.model.Bid;
-import org.apache.beam.integration.nexmark.model.Event;
-import org.apache.beam.integration.nexmark.model.KnownSize;
-import org.apache.beam.integration.nexmark.model.Person;
-import org.apache.beam.integration.nexmark.queries.NexmarkQuery;
-import org.apache.beam.integration.nexmark.queries.NexmarkQueryModel;
-import org.apache.beam.integration.nexmark.queries.Query0;
-import org.apache.beam.integration.nexmark.queries.Query0Model;
-import org.apache.beam.integration.nexmark.queries.Query1;
-import org.apache.beam.integration.nexmark.queries.Query10;
-import org.apache.beam.integration.nexmark.queries.Query11;
-import org.apache.beam.integration.nexmark.queries.Query12;
-import org.apache.beam.integration.nexmark.queries.Query1Model;
-import org.apache.beam.integration.nexmark.queries.Query2;
-import org.apache.beam.integration.nexmark.queries.Query2Model;
-import org.apache.beam.integration.nexmark.queries.Query3;
-import org.apache.beam.integration.nexmark.queries.Query3Model;
-import org.apache.beam.integration.nexmark.queries.Query4;
-import org.apache.beam.integration.nexmark.queries.Query4Model;
-import org.apache.beam.integration.nexmark.queries.Query5;
-import org.apache.beam.integration.nexmark.queries.Query5Model;
-import org.apache.beam.integration.nexmark.queries.Query6;
-import org.apache.beam.integration.nexmark.queries.Query6Model;
-import org.apache.beam.integration.nexmark.queries.Query7;
-import org.apache.beam.integration.nexmark.queries.Query7Model;
-import org.apache.beam.integration.nexmark.queries.Query8;
-import org.apache.beam.integration.nexmark.queries.Query8Model;
-import org.apache.beam.integration.nexmark.queries.Query9;
-import org.apache.beam.integration.nexmark.queries.Query9Model;
-import org.apache.beam.sdk.Pipeline;
-import org.apache.beam.sdk.PipelineResult;
-import org.apache.beam.sdk.coders.CoderException;
-import org.apache.beam.sdk.io.AvroIO;
-import org.apache.beam.sdk.io.TextIO;
-import org.apache.beam.sdk.io.gcp.bigquery.BigQueryIO;
-import org.apache.beam.sdk.io.gcp.pubsub.PubsubIO;
-import org.apache.beam.sdk.io.gcp.pubsub.PubsubMessage;
-import org.apache.beam.sdk.metrics.DistributionResult;
-import org.apache.beam.sdk.metrics.MetricNameFilter;
-import org.apache.beam.sdk.metrics.MetricQueryResults;
-import org.apache.beam.sdk.metrics.MetricResult;
-import org.apache.beam.sdk.metrics.MetricsFilter;
-import org.apache.beam.sdk.testing.PAssert;
-import org.apache.beam.sdk.transforms.DoFn;
-import org.apache.beam.sdk.transforms.ParDo;
-import org.apache.beam.sdk.util.CoderUtils;
-import org.apache.beam.sdk.values.PCollection;
-import org.apache.beam.sdk.values.PCollectionTuple;
-import org.apache.beam.sdk.values.TimestampedValue;
-import org.apache.beam.sdk.values.TupleTag;
-import org.apache.beam.sdk.values.TupleTagList;
-import org.joda.time.Duration;
-import org.slf4j.LoggerFactory;
-
-/**
- * Run a single Nexmark query using a given configuration.
- */
-public class NexmarkLauncher<OptionT extends NexmarkOptions> {
-  private static final org.slf4j.Logger LOG = LoggerFactory.getLogger(NexmarkLauncher.class);
-  /**
-   * Minimum number of samples needed for 'stead-state' rate calculation.
-   */
-  private static final int MIN_SAMPLES = 9;
-  /**
-   * Minimum length of time over which to consider samples for 'steady-state' rate calculation.
-   */
-  private static final Duration MIN_WINDOW = Duration.standardMinutes(2);
-  /**
-   * Delay between perf samples.
-   */
-  private static final Duration PERF_DELAY = Duration.standardSeconds(15);
-  /**
-   * How long to let streaming pipeline run after all events have been generated and we've
-   * seen no activity.
-   */
-  private static final Duration DONE_DELAY = Duration.standardMinutes(1);
-  /**
-   * How long to allow no activity without warning.
-   */
-  private static final Duration STUCK_WARNING_DELAY = Duration.standardMinutes(10);
-  /**
-   * How long to let streaming pipeline run after we've
-   * seen no activity, even if all events have not been generated.
-   */
-  private static final Duration STUCK_TERMINATE_DELAY = Duration.standardDays(3);
-  /**
-   * NexmarkOptions shared by all runs.
-   */
-  private final OptionT options;
-
-  /**
-   * Which configuration we are running.
-   */
-  @Nullable
-  private NexmarkConfiguration configuration;
-
-  /**
-   * If in --pubsubMode=COMBINED, the event monitor for the publisher pipeline. Otherwise null.
-   */
-  @Nullable
-  private Monitor<Event> publisherMonitor;
-
-  /**
-   * If in --pubsubMode=COMBINED, the pipeline result for the publisher pipeline. Otherwise null.
-   */
-  @Nullable
-  private PipelineResult publisherResult;
-
-  /**
-   * Result for the main pipeline.
-   */
-  @Nullable
-  private PipelineResult mainResult;
-
-  /**
-   * Query name we are running.
-   */
-  @Nullable
-  private String queryName;
-
-  public NexmarkLauncher(OptionT options) {
-    this.options = options;
-  }
-
-
-  /**
-   * Is this query running in streaming mode?
-   */
-  private boolean isStreaming() {
-    return options.isStreaming();
-  }
-
-  /**
-   * Return maximum number of workers.
-   */
-  private int maxNumWorkers() {
-    return 5;
-  }
-
-  /**
-   * Return the current value for a long counter, or a default value if can't be retrieved.
-   * Note this uses only attempted metrics because some runners don't support committed metrics.
-   */
-  private long getCounterMetric(PipelineResult result, String namespace, String name,
-    long defaultValue) {
-    MetricQueryResults metrics = result.metrics().queryMetrics(
-        MetricsFilter.builder().addNameFilter(MetricNameFilter.named(namespace, name)).build());
-    Iterable<MetricResult<Long>> counters = metrics.counters();
-    try {
-      MetricResult<Long> metricResult = counters.iterator().next();
-      return metricResult.attempted();
-    } catch (NoSuchElementException e) {
-      LOG.error("Failed to get metric {}, from namespace {}", name, namespace);
-    }
-    return defaultValue;
-  }
-
-  /**
-   * Return the current value for a long counter, or a default value if can't be retrieved.
-   * Note this uses only attempted metrics because some runners don't support committed metrics.
-   */
-  private long getDistributionMetric(PipelineResult result, String namespace, String name,
-      DistributionType distType, long defaultValue) {
-    MetricQueryResults metrics = result.metrics().queryMetrics(
-        MetricsFilter.builder().addNameFilter(MetricNameFilter.named(namespace, name)).build());
-    Iterable<MetricResult<DistributionResult>> distributions = metrics.distributions();
-    try {
-      MetricResult<DistributionResult> distributionResult = distributions.iterator().next();
-      switch (distType)
-      {
-        case MIN:
-          return distributionResult.attempted().min();
-        case MAX:
-          return distributionResult.attempted().max();
-        default:
-          return defaultValue;
-      }
-    } catch (NoSuchElementException e) {
-      LOG.error(
-          "Failed to get distribution metric {} for namespace {}",
-          name,
-          namespace);
-    }
-    return defaultValue;
-  }
-
-  private enum DistributionType {MIN, MAX}
-
-  /**
-   * Return the current value for a time counter, or -1 if can't be retrieved.
-   */
-  private long getTimestampMetric(long now, long value) {
-    // timestamp metrics are used to monitor time of execution of transforms.
-    // If result timestamp metric is too far from now, consider that metric is erroneous
-
-    if (Math.abs(value - now) > Duration.standardDays(10000).getMillis()) {
-      return -1;
-    }
-    return value;
-  }
-
-  /**
-   * Find a 'steady state' events/sec from {@code snapshots} and
-   * store it in {@code perf} if found.
-   */
-  private void captureSteadyState(NexmarkPerf perf, List<NexmarkPerf.ProgressSnapshot> snapshots) {
-    if (!options.isStreaming()) {
-      return;
-    }
-
-    // Find the first sample with actual event and result counts.
-    int dataStart = 0;
-    for (; dataStart < snapshots.size(); dataStart++) {
-      if (snapshots.get(dataStart).numEvents >= 0 && snapshots.get(dataStart).numResults >= 0) {
-        break;
-      }
-    }
-
-    // Find the last sample which demonstrated progress.
-    int dataEnd = snapshots.size() - 1;
-    for (; dataEnd > dataStart; dataEnd--) {
-      if (snapshots.get(dataEnd).anyActivity(snapshots.get(dataEnd - 1))) {
-        break;
-      }
-    }
-
-    int numSamples = dataEnd - dataStart + 1;
-    if (numSamples < MIN_SAMPLES) {
-      // Not enough samples.
-      NexmarkUtils.console("%d samples not enough to calculate steady-state event rate",
-          numSamples);
-      return;
-    }
-
-    // We'll look at only the middle third samples.
-    int sampleStart = dataStart + numSamples / 3;
-    int sampleEnd = dataEnd - numSamples / 3;
-
-    double sampleSec =
-        snapshots.get(sampleEnd).secSinceStart - snapshots.get(sampleStart).secSinceStart;
-    if (sampleSec < MIN_WINDOW.getStandardSeconds()) {
-      // Not sampled over enough time.
-      NexmarkUtils.console(
-          "sample of %.1f sec not long enough to calculate steady-state event rate",
-          sampleSec);
-      return;
-    }
-
-    // Find rate with least squares error.
-    double sumxx = 0.0;
-    double sumxy = 0.0;
-    long prevNumEvents = -1;
-    for (int i = sampleStart; i <= sampleEnd; i++) {
-      if (prevNumEvents == snapshots.get(i).numEvents) {
-        // Skip samples with no change in number of events since they contribute no data.
-        continue;
-      }
-      // Use the effective runtime instead of wallclock time so we can
-      // insulate ourselves from delays and stutters in the query manager.
-      double x = snapshots.get(i).runtimeSec;
-      prevNumEvents = snapshots.get(i).numEvents;
-      double y = prevNumEvents;
-      sumxx += x * x;
-      sumxy += x * y;
-    }
-    double eventsPerSec = sumxy / sumxx;
-    NexmarkUtils.console("revising events/sec from %.1f to %.1f", perf.eventsPerSec, eventsPerSec);
-    perf.eventsPerSec = eventsPerSec;
-  }
-
-  /**
-   * Return the current performance given {@code eventMonitor} and {@code resultMonitor}.
-   */
-  private NexmarkPerf currentPerf(
-      long startMsSinceEpoch, long now, PipelineResult result,
-      List<NexmarkPerf.ProgressSnapshot> snapshots, Monitor<?> eventMonitor,
-      Monitor<?> resultMonitor) {
-    NexmarkPerf perf = new NexmarkPerf();
-
-    long numEvents =
-      getCounterMetric(result, eventMonitor.name, eventMonitor.prefix + ".elements", -1);
-    long numEventBytes =
-      getCounterMetric(result, eventMonitor.name, eventMonitor.prefix + ".bytes", -1);
-    long eventStart =
-      getTimestampMetric(now,
-        getDistributionMetric(result, eventMonitor.name, eventMonitor.prefix + ".startTime",
-          DistributionType.MIN, -1));
-    long eventEnd =
-      getTimestampMetric(now,
-        getDistributionMetric(result, eventMonitor.name, eventMonitor.prefix + ".endTime",
-          DistributionType.MAX, -1));
-
-    long numResults =
-      getCounterMetric(result, resultMonitor.name, resultMonitor.prefix + ".elements", -1);
-    long numResultBytes =
-      getCounterMetric(result, resultMonitor.name, resultMonitor.prefix + ".bytes", -1);
-    long resultStart =
-      getTimestampMetric(now,
-        getDistributionMetric(result, resultMonitor.name, resultMonitor.prefix + ".startTime",
-          DistributionType.MIN, -1));
-    long resultEnd =
-      getTimestampMetric(now,
-        getDistributionMetric(result, resultMonitor.name, resultMonitor.prefix + ".endTime",
-          DistributionType.MAX, -1));
-    long timestampStart =
-      getTimestampMetric(now,
-        getDistributionMetric(result,
-          resultMonitor.name, resultMonitor.prefix + ".startTimestamp",
-          DistributionType.MIN, -1));
-    long timestampEnd =
-      getTimestampMetric(now,
-        getDistributionMetric(result,
-          resultMonitor.name, resultMonitor.prefix + ".endTimestamp",
-          DistributionType.MAX, -1));
-
-    long effectiveEnd = -1;
-    if (eventEnd >= 0 && resultEnd >= 0) {
-      // It is possible for events to be generated after the last result was emitted.
-      // (Eg Query 2, which only yields results for a small prefix of the event stream.)
-      // So use the max of last event and last result times.
-      effectiveEnd = Math.max(eventEnd, resultEnd);
-    } else if (resultEnd >= 0) {
-      effectiveEnd = resultEnd;
-    } else if (eventEnd >= 0) {
-      // During startup we may have no result yet, but we would still like to track how
-      // long the pipeline has been running.
-      effectiveEnd = eventEnd;
-    }
-
-    if (effectiveEnd >= 0 && eventStart >= 0 && effectiveEnd >= eventStart) {
-      perf.runtimeSec = (effectiveEnd - eventStart) / 1000.0;
-    }
-
-    if (numEvents >= 0) {
-      perf.numEvents = numEvents;
-    }
-
-    if (numEvents >= 0 && perf.runtimeSec > 0.0) {
-      // For streaming we may later replace this with a 'steady-state' value calculated
-      // from the progress snapshots.
-      perf.eventsPerSec = numEvents / perf.runtimeSec;
-    }
-
-    if (numEventBytes >= 0 && perf.runtimeSec > 0.0) {
-      perf.eventBytesPerSec = numEventBytes / perf.runtimeSec;
-    }
-
-    if (numResults >= 0) {
-      perf.numResults = numResults;
-    }
-
-    if (numResults >= 0 && perf.runtimeSec > 0.0) {
-      perf.resultsPerSec = numResults / perf.runtimeSec;
-    }
-
-    if (numResultBytes >= 0 && perf.runtimeSec > 0.0) {
-      perf.resultBytesPerSec = numResultBytes / perf.runtimeSec;
-    }
-
-    if (eventStart >= 0) {
-      perf.startupDelaySec = (eventStart - startMsSinceEpoch) / 1000.0;
-    }
-
-    if (resultStart >= 0 && eventStart >= 0 && resultStart >= eventStart) {
-      perf.processingDelaySec = (resultStart - eventStart) / 1000.0;
-    }
-
-    if (timestampStart >= 0 && timestampEnd >= 0 && perf.runtimeSec > 0.0) {
-      double eventRuntimeSec = (timestampEnd - timestampStart) / 1000.0;
-      perf.timeDilation = eventRuntimeSec / perf.runtimeSec;
-    }
-
-    if (resultEnd >= 0) {
-      // Fill in the shutdown delay assuming the job has now finished.
-      perf.shutdownDelaySec = (now - resultEnd) / 1000.0;
-    }
-
-    // As soon as available, try to capture cumulative cost at this point too.
-
-    NexmarkPerf.ProgressSnapshot snapshot = new NexmarkPerf.ProgressSnapshot();
-    snapshot.secSinceStart = (now - startMsSinceEpoch) / 1000.0;
-    snapshot.runtimeSec = perf.runtimeSec;
-    snapshot.numEvents = numEvents;
-    snapshot.numResults = numResults;
-    snapshots.add(snapshot);
-
-    captureSteadyState(perf, snapshots);
-
-    return perf;
-  }
-
-  /**
-   * Build and run a pipeline using specified options.
-   */
-  interface PipelineBuilder<OptionT extends NexmarkOptions> {
-    void build(OptionT publishOnlyOptions);
-  }
-
-  /**
-   * Invoke the builder with options suitable for running a publish-only child pipeline.
-   */
-  private void invokeBuilderForPublishOnlyPipeline(PipelineBuilder<NexmarkOptions> builder) {
-    builder.build(options);
-  }
-
-  /**
-   * Monitor the performance and progress of a running job. Return final performance if
-   * it was measured.
-   */
-  @Nullable
-  private NexmarkPerf monitor(NexmarkQuery query) {
-    if (!options.getMonitorJobs()) {
-      return null;
-    }
-
-    if (configuration.debug) {
-      NexmarkUtils.console("Waiting for main pipeline to 'finish'");
-    } else {
-      NexmarkUtils.console("--debug=false, so job will not self-cancel");
-    }
-
-    PipelineResult job = mainResult;
-    PipelineResult publisherJob = publisherResult;
-    List<NexmarkPerf.ProgressSnapshot> snapshots = new ArrayList<>();
-    long startMsSinceEpoch = System.currentTimeMillis();
-    long endMsSinceEpoch = -1;
-    if (options.getRunningTimeMinutes() != null) {
-      endMsSinceEpoch = startMsSinceEpoch
-                        + Duration.standardMinutes(options.getRunningTimeMinutes()).getMillis()
-                        - Duration.standardSeconds(configuration.preloadSeconds).getMillis();
-    }
-    long lastActivityMsSinceEpoch = -1;
-    NexmarkPerf perf = null;
-    boolean waitingForShutdown = false;
-    boolean publisherCancelled = false;
-    List<String> errors = new ArrayList<>();
-
-    while (true) {
-      long now = System.currentTimeMillis();
-      if (endMsSinceEpoch >= 0 && now > endMsSinceEpoch && !waitingForShutdown) {
-        NexmarkUtils.console("Reached end of test, cancelling job");
-        try {
-          job.cancel();
-        } catch (IOException e) {
-          throw new RuntimeException("Unable to cancel main job: ", e);
-        }
-        if (publisherResult != null) {
-          try {
-            publisherJob.cancel();
-          } catch (IOException e) {
-            throw new RuntimeException("Unable to cancel publisher job: ", e);
-          }
-          publisherCancelled = true;
-        }
-        waitingForShutdown = true;
-      }
-
-      PipelineResult.State state = job.getState();
-      NexmarkUtils.console("%s %s%s", state, queryName,
-          waitingForShutdown ? " (waiting for shutdown)" : "");
-
-      NexmarkPerf currPerf;
-      if (configuration.debug) {
-        currPerf = currentPerf(startMsSinceEpoch, now, job, snapshots,
-                               query.eventMonitor, query.resultMonitor);
-      } else {
-        currPerf = null;
-      }
-
-      if (perf == null || perf.anyActivity(currPerf)) {
-        lastActivityMsSinceEpoch = now;
-      }
-
-      if (options.isStreaming() && !waitingForShutdown) {
-        Duration quietFor = new Duration(lastActivityMsSinceEpoch, now);
-        long fatalCount = getCounterMetric(job, query.getName(), "fatal", 0);
-        if (fatalCount > 0) {
-          NexmarkUtils.console("job has fatal errors, cancelling.");
-          errors.add(String.format("Pipeline reported %s fatal errors", fatalCount));
-          waitingForShutdown = true;
-        } else if (configuration.debug && configuration.numEvents > 0
-                   && currPerf.numEvents == configuration.numEvents
-                   && currPerf.numResults >= 0 && quietFor.isLongerThan(DONE_DELAY)) {
-          NexmarkUtils.console("streaming query appears to have finished, cancelling job.");
-          waitingForShutdown = true;
-        } else if (quietFor.isLongerThan(STUCK_TERMINATE_DELAY)) {
-          NexmarkUtils.console("streaming query appears to have gotten stuck, cancelling job.");
-          errors.add("Streaming job was cancelled since appeared stuck");
-          waitingForShutdown = true;
-        } else if (quietFor.isLongerThan(STUCK_WARNING_DELAY)) {
-          NexmarkUtils.console("WARNING: streaming query appears to have been stuck for %d min.",
-              quietFor.getStandardMinutes());
-          errors.add(
-              String.format("Streaming query was stuck for %d min", quietFor.getStandardMinutes()));
-        }
-
-        if (waitingForShutdown) {
-          try {
-            job.cancel();
-          } catch (IOException e) {
-            throw new RuntimeException("Unable to cancel main job: ", e);
-          }
-        }
-      }
-
-      perf = currPerf;
-
-      boolean running = true;
-      switch (state) {
-        case UNKNOWN:
-        case STOPPED:
-        case RUNNING:
-          // Keep going.
-          break;
-        case DONE:
-          // All done.
-          running = false;
-          break;
-        case CANCELLED:
-          running = false;
-          if (!waitingForShutdown) {
-            errors.add("Job was unexpectedly cancelled");
-          }
-          break;
-        case FAILED:
-        case UPDATED:
-          // Abnormal termination.
-          running = false;
-          errors.add("Job was unexpectedly updated");
-          break;
-      }
-
-      if (!running) {
-        break;
-      }
-
-      if (lastActivityMsSinceEpoch == now) {
-        NexmarkUtils.console("new perf %s", perf);
-      } else {
-        NexmarkUtils.console("no activity");
-      }
-
-      try {
-        Thread.sleep(PERF_DELAY.getMillis());
-      } catch (InterruptedException e) {
-        Thread.interrupted();
-        NexmarkUtils.console("Interrupted: pipeline is still running");
-      }
-    }
-
-    perf.errors = errors;
-    perf.snapshots = snapshots;
-
-    if (publisherResult != null) {
-      NexmarkUtils.console("Shutting down publisher pipeline.");
-      try {
-        if (!publisherCancelled) {
-          publisherJob.cancel();
-        }
-        publisherJob.waitUntilFinish(Duration.standardMinutes(5));
-      } catch (IOException e) {
-        throw new RuntimeException("Unable to cancel publisher job: ", e);
-      }
-    }
-
-    return perf;
-  }
-
-  // ================================================================================
-  // Basic sources and sinks
-  // ================================================================================
-
-  /**
-   * Return a topic name.
-   */
-  private String shortTopic(long now) {
-    String baseTopic = options.getPubsubTopic();
-    if (Strings.isNullOrEmpty(baseTopic)) {
-      throw new RuntimeException("Missing --pubsubTopic");
-    }
-    switch (options.getResourceNameMode()) {
-      case VERBATIM:
-        return baseTopic;
-      case QUERY:
-        return String.format("%s_%s_source", baseTopic, queryName);
-      case QUERY_AND_SALT:
-        return String.format("%s_%s_%d_source", baseTopic, queryName, now);
-    }
-    throw new RuntimeException("Unrecognized enum " + options.getResourceNameMode());
-  }
-
-  /**
-   * Return a subscription name.
-   */
-  private String shortSubscription(long now) {
-    String baseSubscription = options.getPubsubSubscription();
-    if (Strings.isNullOrEmpty(baseSubscription)) {
-      throw new RuntimeException("Missing --pubsubSubscription");
-    }
-    switch (options.getResourceNameMode()) {
-      case VERBATIM:
-        return baseSubscription;
-      case QUERY:
-        return String.format("%s_%s_source", baseSubscription, queryName);
-      case QUERY_AND_SALT:
-        return String.format("%s_%s_%d_source", baseSubscription, queryName, now);
-    }
-    throw new RuntimeException("Unrecognized enum " + options.getResourceNameMode());
-  }
-
-  /**
-   * Return a file name for plain text.
-   */
-  private String textFilename(long now) {
-    String baseFilename = options.getOutputPath();
-    if (Strings.isNullOrEmpty(baseFilename)) {
-      throw new RuntimeException("Missing --outputPath");
-    }
-    switch (options.getResourceNameMode()) {
-      case VERBATIM:
-        return baseFilename;
-      case QUERY:
-        return String.format("%s/nexmark_%s.txt", baseFilename, queryName);
-      case QUERY_AND_SALT:
-        return String.format("%s/nexmark_%s_%d.txt", baseFilename, queryName, now);
-    }
-    throw new RuntimeException("Unrecognized enum " + options.getResourceNameMode());
-  }
-
-  /**
-   * Return a BigQuery table spec.
-   */
-  private String tableSpec(long now, String version) {
-    String baseTableName = options.getBigQueryTable();
-    if (Strings.isNullOrEmpty(baseTableName)) {
-      throw new RuntimeException("Missing --bigQueryTable");
-    }
-    switch (options.getResourceNameMode()) {
-      case VERBATIM:
-        return String.format("%s:nexmark.%s_%s",
-                             options.getProject(), baseTableName, version);
-      case QUERY:
-        return String.format("%s:nexmark.%s_%s_%s",
-                             options.getProject(), baseTableName, queryName, version);
-      case QUERY_AND_SALT:
-        return String.format("%s:nexmark.%s_%s_%s_%d",
-                             options.getProject(), baseTableName, queryName, version, now);
-    }
-    throw new RuntimeException("Unrecognized enum " + options.getResourceNameMode());
-  }
-
-  /**
-   * Return a directory for logs.
-   */
-  private String logsDir(long now) {
-    String baseFilename = options.getOutputPath();
-    if (Strings.isNullOrEmpty(baseFilename)) {
-      throw new RuntimeException("Missing --outputPath");
-    }
-    switch (options.getResourceNameMode()) {
-      case VERBATIM:
-        return baseFilename;
-      case QUERY:
-        return String.format("%s/logs_%s", baseFilename, queryName);
-      case QUERY_AND_SALT:
-        return String.format("%s/logs_%s_%d", baseFilename, queryName, now);
-    }
-    throw new RuntimeException("Unrecognized enum " + options.getResourceNameMode());
-  }
-
-  /**
-   * Return a source of synthetic events.
-   */
-  private PCollection<Event> sourceEventsFromSynthetic(Pipeline p) {
-    if (isStreaming()) {
-      NexmarkUtils.console("Generating %d events in streaming mode", configuration.numEvents);
-      return p.apply(queryName + ".ReadUnbounded", NexmarkUtils.streamEventsSource(configuration));
-    } else {
-      NexmarkUtils.console("Generating %d events in batch mode", configuration.numEvents);
-      return p.apply(queryName + ".ReadBounded", NexmarkUtils.batchEventsSource(configuration));
-    }
-  }
-
-  /**
-   * Return source of events from Pubsub.
-   */
-  private PCollection<Event> sourceEventsFromPubsub(Pipeline p, long now) {
-    String shortSubscription = shortSubscription(now);
-    NexmarkUtils.console("Reading events from Pubsub %s", shortSubscription);
-
-    PubsubIO.Read<PubsubMessage> io =
-        PubsubIO.readMessagesWithAttributes().fromSubscription(shortSubscription)
-            .withIdAttribute(NexmarkUtils.PUBSUB_ID);
-    if (!configuration.usePubsubPublishTime) {
-      io = io.withTimestampAttribute(NexmarkUtils.PUBSUB_TIMESTAMP);
-    }
-
-    return p
-      .apply(queryName + ".ReadPubsubEvents", io)
-      .apply(queryName + ".PubsubMessageToEvent", ParDo.of(new DoFn<PubsubMessage, Event>() {
-        @ProcessElement
-        public void processElement(ProcessContext c) {
-          byte[] payload = c.element().getPayload();
-          try {
-            Event event = CoderUtils.decodeFromByteArray(Event.CODER, payload);
-            c.output(event);
-          } catch (CoderException e) {
-            LOG.error("Error while decoding Event from pusbSub message: serialization error");
-          }
-        }
-      }));
-  }
-
-  /**
-   * Return Avro source of events from {@code options.getInputFilePrefix}.
-   */
-  private PCollection<Event> sourceEventsFromAvro(Pipeline p) {
-    String filename = options.getInputPath();
-    if (Strings.isNullOrEmpty(filename)) {
-      throw new RuntimeException("Missing --inputPath");
-    }
-    NexmarkUtils.console("Reading events from Avro files at %s", filename);
-    return p
-        .apply(queryName + ".ReadAvroEvents", AvroIO.read(Event.class)
-                          .from(filename + "*.avro"))
-        .apply("OutputWithTimestamp", NexmarkQuery.EVENT_TIMESTAMP_FROM_DATA);
-  }
-
-  /**
-   * Send {@code events} to Pubsub.
-   */
-  private void sinkEventsToPubsub(PCollection<Event> events, long now) {
-    String shortTopic = shortTopic(now);
-    NexmarkUtils.console("Writing events to Pubsub %s", shortTopic);
-
-    PubsubIO.Write<PubsubMessage> io =
-        PubsubIO.writeMessages().to(shortTopic)
-            .withIdAttribute(NexmarkUtils.PUBSUB_ID);
-    if (!configuration.usePubsubPublishTime) {
-      io = io.withTimestampAttribute(NexmarkUtils.PUBSUB_TIMESTAMP);
-    }
-
-    events.apply(queryName + ".EventToPubsubMessage",
-            ParDo.of(new DoFn<Event, PubsubMessage>() {
-              @ProcessElement
-              public void processElement(ProcessContext c) {
-                try {
-                  byte[] payload = CoderUtils.encodeToByteArray(Event.CODER, c.element());
-                  c.output(new PubsubMessage(payload, new HashMap<String, String>()));
-                } catch (CoderException e1) {
-                  LOG.error("Error while sending Event {} to pusbSub: serialization error",
-                      c.element().toString());
-                }
-              }
-            })
-        )
-        .apply(queryName + ".WritePubsubEvents", io);
-  }
-
-  /**
-   * Send {@code formattedResults} to Pubsub.
-   */
-  private void sinkResultsToPubsub(PCollection<String> formattedResults, long now) {
-    String shortTopic = shortTopic(now);
-    NexmarkUtils.console("Writing results to Pubsub %s", shortTopic);
-    PubsubIO.Write<String> io =
-        PubsubIO.writeStrings().to(shortTopic)
-            .withIdAttribute(NexmarkUtils.PUBSUB_ID);
-    if (!configuration.usePubsubPublishTime) {
-      io = io.withTimestampAttribute(NexmarkUtils.PUBSUB_TIMESTAMP);
-    }
-    formattedResults.apply(queryName + ".WritePubsubResults", io);
-  }
-
-  /**
-   * Sink all raw Events in {@code source} to {@code options.getOutputPath}.
-   * This will configure the job to write the following files:
-   * <ul>
-   * <li>{@code $outputPath/event*.avro} All Event entities.
-   * <li>{@code $outputPath/auction*.avro} Auction entities.
-   * <li>{@code $outputPath/bid*.avro} Bid entities.
-   * <li>{@code $outputPath/person*.avro} Person entities.
-   * </ul>
-   *
-   * @param source A PCollection of events.
-   */
-  private void sinkEventsToAvro(PCollection<Event> source) {
-    String filename = options.getOutputPath();
-    if (Strings.isNullOrEmpty(filename)) {
-      throw new RuntimeException("Missing --outputPath");
-    }
-    NexmarkUtils.console("Writing events to Avro files at %s", filename);
-    source.apply(queryName + ".WriteAvroEvents",
-            AvroIO.write(Event.class).to(filename + "/event").withSuffix(".avro"));
-    source.apply(NexmarkQuery.JUST_BIDS)
-          .apply(queryName + ".WriteAvroBids",
-            AvroIO.write(Bid.class).to(filename + "/bid").withSuffix(".avro"));
-    source.apply(NexmarkQuery.JUST_NEW_AUCTIONS)
-          .apply(queryName + ".WriteAvroAuctions",
-            AvroIO.write(Auction.class).to(filename + "/auction").withSuffix(".avro"));
-    source.apply(NexmarkQuery.JUST_NEW_PERSONS)
-          .apply(queryName + ".WriteAvroPeople",
-            AvroIO.write(Person.class).to(filename + "/person").withSuffix(".avro"));
-  }
-
-  /**
-   * Send {@code formattedResults} to text files.
-   */
-  private void sinkResultsToText(PCollection<String> formattedResults, long now) {
-    String filename = textFilename(now);
-    NexmarkUtils.console("Writing results to text files at %s", filename);
-    formattedResults.apply(queryName + ".WriteTextResults",
-        TextIO.write().to(filename));
-  }
-
-  private static class StringToTableRow extends DoFn<String, TableRow> {
-    @ProcessElement
-    public void processElement(ProcessContext c) {
-      int n = ThreadLocalRandom.current().nextInt(10);
-      List<TableRow> records = new ArrayList<>(n);
-      for (int i = 0; i < n; i++) {
-        records.add(new TableRow().set("index", i).set("value", Integer.toString(i)));
-      }
-      c.output(new TableRow().set("result", c.element()).set("records", records));
-    }
-  }
-
-  /**
-   * Send {@code formattedResults} to BigQuery.
-   */
-  private void sinkResultsToBigQuery(
-      PCollection<String> formattedResults, long now,
-      String version) {
-    String tableSpec = tableSpec(now, version);
-    TableSchema tableSchema =
-        new TableSchema().setFields(ImmutableList.of(
-            new TableFieldSchema().setName("result").setType("STRING"),
-            new TableFieldSchema().setName("records").setMode("REPEATED").setType("RECORD")
-                                  .setFields(ImmutableList.of(
-                                      new TableFieldSchema().setName("index").setType("INTEGER"),
-                                      new TableFieldSchema().setName("value").setType("STRING")))));
-    NexmarkUtils.console("Writing results to BigQuery table %s", tableSpec);
-    BigQueryIO.Write io =
-        BigQueryIO.write().to(tableSpec)
-                        .withSchema(tableSchema)
-                        .withCreateDisposition(BigQueryIO.Write.CreateDisposition.CREATE_IF_NEEDED)
-                        .withWriteDisposition(BigQueryIO.Write.WriteDisposition.WRITE_APPEND);
-    formattedResults
-        .apply(queryName + ".StringToTableRow", ParDo.of(new StringToTableRow()))
-        .apply(queryName + ".WriteBigQueryResults", io);
-  }
-
-  // ================================================================================
-  // Construct overall pipeline
-  // ================================================================================
-
-  /**
-   * Return source of events for this run, or null if we are simply publishing events
-   * to Pubsub.
-   */
-  private PCollection<Event> createSource(Pipeline p, final long now) {
-    PCollection<Event> source = null;
-    switch (configuration.sourceType) {
-      case DIRECT:
-        source = sourceEventsFromSynthetic(p);
-        break;
-      case AVRO:
-        source = sourceEventsFromAvro(p);
-        break;
-      case PUBSUB:
-        // Setup the sink for the publisher.
-        switch (configuration.pubSubMode) {
-          case SUBSCRIBE_ONLY:
-            // Nothing to publish.
-            break;
-          case PUBLISH_ONLY:
-            // Send synthesized events to Pubsub in this job.
-            sinkEventsToPubsub(sourceEventsFromSynthetic(p).apply(queryName + ".Snoop",
-                    NexmarkUtils.snoop(queryName)), now);
-            break;
-          case COMBINED:
-            // Send synthesized events to Pubsub in separate publisher job.
-            // We won't start the main pipeline until the publisher has sent the pre-load events.
-            // We'll shutdown the publisher job when we notice the main job has finished.
-            invokeBuilderForPublishOnlyPipeline(new PipelineBuilder<NexmarkOptions>() {
-              @Override
-              public void build(NexmarkOptions publishOnlyOptions) {
-                Pipeline sp = Pipeline.create(options);
-                NexmarkUtils.setupPipeline(configuration.coderStrategy, sp);
-                publisherMonitor = new Monitor<>(queryName, "publisher");
-                sinkEventsToPubsub(
-                    sourceEventsFromSynthetic(sp)
-                            .apply(queryName + ".Monitor", publisherMonitor.getTransform()),
-                    now);
-                publisherResult = sp.run();
-              }
-            });
-            break;
-        }
-
-        // Setup the source for the consumer.
-        switch (configuration.pubSubMode) {
-          case PUBLISH_ONLY:
-            // Nothing to consume. Leave source null.
-            break;
-          case SUBSCRIBE_ONLY:
-          case COMBINED:
-            // Read events from pubsub.
-            source = sourceEventsFromPubsub(p, now);
-            break;
-        }
-        break;
-    }
-    return source;
-  }
-
-  private static final TupleTag<String> MAIN = new TupleTag<String>(){};
-  private static final TupleTag<String> SIDE = new TupleTag<String>(){};
-
-  private static class PartitionDoFn extends DoFn<String, String> {
-    @ProcessElement
-    public void processElement(ProcessContext c) {
-      if (c.element().hashCode() % 2 == 0) {
-        c.output(c.element());
-      } else {
-        c.output(SIDE, c.element());
-      }
-    }
-  }
-
-  /**
-   * Consume {@code results}.
-   */
-  private void sink(PCollection<TimestampedValue<KnownSize>> results, long now) {
-    if (configuration.sinkType == NexmarkUtils.SinkType.COUNT_ONLY) {
-      // Avoid the cost of formatting the results.
-      results.apply(queryName + ".DevNull", NexmarkUtils.devNull(queryName));
-      return;
-    }
-
-    PCollection<String> formattedResults =
-      results.apply(queryName + ".Format", NexmarkUtils.format(queryName));
-    if (options.getLogResults()) {
-      formattedResults = formattedResults.apply(queryName + ".Results.Log",
-              NexmarkUtils.<String>log(queryName + ".Results"));
-    }
-
-    switch (configuration.sinkType) {
-      case DEVNULL:
-        // Discard all results
-        formattedResults.apply(queryName + ".DevNull", NexmarkUtils.devNull(queryName));
-        break;
-      case PUBSUB:
-        sinkResultsToPubsub(formattedResults, now);
-        break;
-      case TEXT:
-        sinkResultsToText(formattedResults, now);
-        break;
-      case AVRO:
-        NexmarkUtils.console(
-            "WARNING: with --sinkType=AVRO, actual query results will be discarded.");
-        break;
-      case BIGQUERY:
-        // Multiple BigQuery backends to mimic what most customers do.
-        PCollectionTuple res = formattedResults.apply(queryName + ".Partition",
-            ParDo.of(new PartitionDoFn()).withOutputTags(MAIN, TupleTagList.of(SIDE)));
-        sinkResultsToBigQuery(res.get(MAIN), now, "main");
-        sinkResultsToBigQuery(res.get(SIDE), now, "side");
-        sinkResultsToBigQuery(formattedResults, now, "copy");
-        break;
-      case COUNT_ONLY:
-        // Short-circuited above.
-        throw new RuntimeException();
-    }
-  }
-
-  // ================================================================================
-  // Entry point
-  // ================================================================================
-
-  /**
-   * Calculate the distribution of the expected rate of results per minute (in event time, not
-   * wallclock time).
-   */
-  private void modelResultRates(NexmarkQueryModel model) {
-    List<Long> counts = Lists.newArrayList(model.simulator().resultsPerWindow());
-    Collections.sort(counts);
-    int n = counts.size();
-    if (n < 5) {
-      NexmarkUtils.console("Query%d: only %d samples", model.configuration.query, n);
-    } else {
-      NexmarkUtils.console("Query%d: N:%d; min:%d; 1st%%:%d; mean:%d; 3rd%%:%d; max:%d",
-                           model.configuration.query, n, counts.get(0), counts.get(n / 4),
-                           counts.get(n / 2),
-                           counts.get(n - 1 - n / 4), counts.get(n - 1));
-    }
-  }
-
-  /**
-   * Run {@code configuration} and return its performance if possible.
-   */
-  @Nullable
-  public NexmarkPerf run(NexmarkConfiguration runConfiguration) {
-    if (options.getManageResources() && !options.getMonitorJobs()) {
-      throw new RuntimeException("If using --manageResources then must also use --monitorJobs.");
-    }
-
-    //
-    // Setup per-run state.
-    //
-    checkState(configuration == null);
-    checkState(queryName == null);
-    configuration = runConfiguration;
-
-    try {
-      NexmarkUtils.console("Running %s", configuration.toShortString());
-
-      if (configuration.numEvents < 0) {
-        NexmarkUtils.console("skipping since configuration is disabled");
-        return null;
-      }
-
-      List<NexmarkQuery> queries = Arrays.asList(new Query0(configuration),
-                                                 new Query1(configuration),
-                                                 new Query2(configuration),
-                                                 new Query3(configuration),
-                                                 new Query4(configuration),
-                                                 new Query5(configuration),
-                                                 new Query6(configuration),
-                                                 new Query7(configuration),
-                                                 new Query8(configuration),
-                                                 new Query9(configuration),
-                                                 new Query10(configuration),
-                                                 new Query11(configuration),
-                                                 new Query12(configuration));
-      NexmarkQuery query = queries.get(configuration.query);
-      queryName = query.getName();
-
-      List<NexmarkQueryModel> models = Arrays.asList(
-          new Query0Model(configuration),
-          new Query1Model(configuration),
-          new Query2Model(configuration),
-          new Query3Model(configuration),
-          new Query4Model(configuration),
-          new Query5Model(configuration),
-          new Query6Model(configuration),
-          new Query7Model(configuration),
-          new Query8Model(configuration),
-          new Query9Model(configuration),
-          null,
-          null,
-          null);
-      NexmarkQueryModel model = models.get(configuration.query);
-
-      if (options.getJustModelResultRate()) {
-        if (model == null) {
-          throw new RuntimeException(String.format("No model for %s", queryName));
-        }
-        modelResultRates(model);
-        return null;
-      }
-
-      long now = System.currentTimeMillis();
-      Pipeline p = Pipeline.create(options);
-      NexmarkUtils.setupPipeline(configuration.coderStrategy, p);
-
-      // Generate events.
-      PCollection<Event> source = createSource(p, now);
-
-      if (options.getLogEvents()) {
-        source = source.apply(queryName + ".Events.Log",
-                NexmarkUtils.<Event>log(queryName + ".Events"));
-      }
-
-      // Source will be null if source type is PUBSUB and mode is PUBLISH_ONLY.
-      // In that case there's nothing more to add to pipeline.
-      if (source != null) {
-        // Optionally sink events in Avro format.
-        // (Query results are ignored).
-        if (configuration.sinkType == NexmarkUtils.SinkType.AVRO) {
-          sinkEventsToAvro(source);
-        }
-
-        // Query 10 logs all events to Google Cloud storage files. It could generate a lot of logs,
-        // so, set parallelism. Also set the output path where to write log files.
-        if (configuration.query == 10) {
-          String path = null;
-          if (options.getOutputPath() != null && !options.getOutputPath().isEmpty()) {
-            path = logsDir(now);
-          }
-          ((Query10) query).setOutputPath(path);
-          ((Query10) query).setMaxNumWorkers(maxNumWorkers());
-        }
-
-        // Apply query.
-        PCollection<TimestampedValue<KnownSize>> results = source.apply(query);
-
-        if (options.getAssertCorrectness()) {
-          if (model == null) {
-            throw new RuntimeException(String.format("No model for %s", queryName));
-          }
-          // We know all our streams have a finite number of elements.
-          results.setIsBoundedInternal(PCollection.IsBounded.BOUNDED);
-          // If we have a finite number of events then assert our pipeline's
-          // results match those of a model using the same sequence of events.
-          PAssert.that(results).satisfies(model.assertionFor());
-        }
-
-        // Output results.
-        sink(results, now);
-      }
-
-      mainResult = p.run();
-      mainResult.waitUntilFinish(Duration.standardSeconds(configuration.streamTimeout));
-      return monitor(query);
-    } finally {
-      configuration = null;
-      queryName = null;
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/f4333df7/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkOptions.java
----------------------------------------------------------------------
diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkOptions.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkOptions.java
deleted file mode 100644
index fbd3e74..0000000
--- a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkOptions.java
+++ /dev/null
@@ -1,403 +0,0 @@
-/*
- * 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.beam.integration.nexmark;
-
-import javax.annotation.Nullable;
-import org.apache.beam.sdk.extensions.gcp.options.GcpOptions;
-import org.apache.beam.sdk.options.ApplicationNameOptions;
-import org.apache.beam.sdk.options.Default;
-import org.apache.beam.sdk.options.Description;
-import org.apache.beam.sdk.options.PipelineOptions;
-import org.apache.beam.sdk.options.StreamingOptions;
-
-/**
- * Command line flags.
- */
-public interface NexmarkOptions
-    extends ApplicationNameOptions, GcpOptions, PipelineOptions, StreamingOptions {
-  @Description("Which suite to run. Default is to use command line arguments for one job.")
-  @Default.Enum("DEFAULT")
-  NexmarkSuite getSuite();
-
-  void setSuite(NexmarkSuite suite);
-
-  @Description("If true, monitor the jobs as they run.")
-  @Default.Boolean(false)
-  boolean getMonitorJobs();
-
-  void setMonitorJobs(boolean monitorJobs);
-
-  @Description("Where the events come from.")
-  @Nullable
-  NexmarkUtils.SourceType getSourceType();
-
-  void setSourceType(NexmarkUtils.SourceType sourceType);
-
-  @Description("Prefix for input files if using avro input")
-  @Nullable
-  String getInputPath();
-
-  void setInputPath(String inputPath);
-
-  @Description("Where results go.")
-  @Nullable
-  NexmarkUtils.SinkType getSinkType();
-
-  void setSinkType(NexmarkUtils.SinkType sinkType);
-
-  @Description("Which mode to run in when source is PUBSUB.")
-  @Nullable
-  NexmarkUtils.PubSubMode getPubSubMode();
-
-  void setPubSubMode(NexmarkUtils.PubSubMode pubSubMode);
-
-  @Description("Which query to run.")
-  @Nullable
-  Integer getQuery();
-
-  void setQuery(Integer query);
-
-  @Description("Prefix for output files if using text output for results or running Query 10.")
-  @Nullable
-  String getOutputPath();
-
-  void setOutputPath(String outputPath);
-
-  @Description("Base name of pubsub topic to publish to in streaming mode.")
-  @Nullable
-  @Default.String("nexmark")
-  String getPubsubTopic();
-
-  void setPubsubTopic(String pubsubTopic);
-
-  @Description("Base name of pubsub subscription to read from in streaming mode.")
-  @Nullable
-  @Default.String("nexmark")
-  String getPubsubSubscription();
-
-  void setPubsubSubscription(String pubsubSubscription);
-
-  @Description("Base name of BigQuery table name if using BigQuery output.")
-  @Nullable
-  @Default.String("nexmark")
-  String getBigQueryTable();
-
-  void setBigQueryTable(String bigQueryTable);
-
-  @Description("Approximate number of events to generate. "
-               + "Zero for effectively unlimited in streaming mode.")
-  @Nullable
-  Long getNumEvents();
-
-  void setNumEvents(Long numEvents);
-
-  @Description("Time in seconds to preload the subscription with data, at the initial input rate "
-               + "of the pipeline.")
-  @Nullable
-  Integer getPreloadSeconds();
-
-  void setPreloadSeconds(Integer preloadSeconds);
-
-  @Description(
-      "Time in seconds to wait in pipelineResult.waitUntilFinish(), useful in streaming mode")
-  @Nullable
-  Integer getStreamTimeout();
-
-  void setStreamTimeout(Integer streamTimeout);
-
-  @Description("Number of unbounded sources to create events.")
-  @Nullable
-  Integer getNumEventGenerators();
-
-  void setNumEventGenerators(Integer numEventGenerators);
-
-  @Description("Shape of event rate curve.")
-  @Nullable
-  NexmarkUtils.RateShape getRateShape();
-
-  void setRateShape(NexmarkUtils.RateShape rateShape);
-
-  @Description("Initial overall event rate (in --rateUnit).")
-  @Nullable
-  Integer getFirstEventRate();
-
-  void setFirstEventRate(Integer firstEventRate);
-
-  @Description("Next overall event rate (in --rateUnit).")
-  @Nullable
-  Integer getNextEventRate();
-
-  void setNextEventRate(Integer nextEventRate);
-
-  @Description("Unit for rates.")
-  @Nullable
-  NexmarkUtils.RateUnit getRateUnit();
-
-  void setRateUnit(NexmarkUtils.RateUnit rateUnit);
-
-  @Description("Overall period of rate shape, in seconds.")
-  @Nullable
-  Integer getRatePeriodSec();
-
-  void setRatePeriodSec(Integer ratePeriodSec);
-
-  @Description("If true, relay events in real time in streaming mode.")
-  @Nullable
-  Boolean getIsRateLimited();
-
-  void setIsRateLimited(Boolean isRateLimited);
-
-  @Description("If true, use wallclock time as event time. Otherwise, use a deterministic"
-               + " time in the past so that multiple runs will see exactly the same event streams"
-               + " and should thus have exactly the same results.")
-  @Nullable
-  Boolean getUseWallclockEventTime();
-
-  void setUseWallclockEventTime(Boolean useWallclockEventTime);
-
-  @Description("Assert pipeline results match model results.")
-  @Nullable
-  boolean getAssertCorrectness();
-
-  void setAssertCorrectness(boolean assertCorrectness);
-
-  @Description("Log all input events.")
-  @Nullable
-  boolean getLogEvents();
-
-  void setLogEvents(boolean logEvents);
-
-  @Description("Log all query results.")
-  @Nullable
-  boolean getLogResults();
-
-  void setLogResults(boolean logResults);
-
-  @Description("Average size in bytes for a person record.")
-  @Nullable
-  Integer getAvgPersonByteSize();
-
-  void setAvgPersonByteSize(Integer avgPersonByteSize);
-
-  @Description("Average size in bytes for an auction record.")
-  @Nullable
-  Integer getAvgAuctionByteSize();
-
-  void setAvgAuctionByteSize(Integer avgAuctionByteSize);
-
-  @Description("Average size in bytes for a bid record.")
-  @Nullable
-  Integer getAvgBidByteSize();
-
-  void setAvgBidByteSize(Integer avgBidByteSize);
-
-  @Description("Ratio of bids for 'hot' auctions above the background.")
-  @Nullable
-  Integer getHotAuctionRatio();
-
-  void setHotAuctionRatio(Integer hotAuctionRatio);
-
-  @Description("Ratio of auctions for 'hot' sellers above the background.")
-  @Nullable
-  Integer getHotSellersRatio();
-
-  void setHotSellersRatio(Integer hotSellersRatio);
-
-  @Description("Ratio of auctions for 'hot' bidders above the background.")
-  @Nullable
-  Integer getHotBiddersRatio();
-
-  void setHotBiddersRatio(Integer hotBiddersRatio);
-
-  @Description("Window size in seconds.")
-  @Nullable
-  Long getWindowSizeSec();
-
-  void setWindowSizeSec(Long windowSizeSec);
-
-  @Description("Window period in seconds.")
-  @Nullable
-  Long getWindowPeriodSec();
-
-  void setWindowPeriodSec(Long windowPeriodSec);
-
-  @Description("If in streaming mode, the holdback for watermark in seconds.")
-  @Nullable
-  Long getWatermarkHoldbackSec();
-
-  void setWatermarkHoldbackSec(Long watermarkHoldbackSec);
-
-  @Description("Roughly how many auctions should be in flight for each generator.")
-  @Nullable
-  Integer getNumInFlightAuctions();
-
-  void setNumInFlightAuctions(Integer numInFlightAuctions);
-
-
-  @Description("Maximum number of people to consider as active for placing auctions or bids.")
-  @Nullable
-  Integer getNumActivePeople();
-
-  void setNumActivePeople(Integer numActivePeople);
-
-  @Description("Filename of perf data to append to.")
-  @Nullable
-  String getPerfFilename();
-
-  void setPerfFilename(String perfFilename);
-
-  @Description("Filename of baseline perf data to read from.")
-  @Nullable
-  String getBaselineFilename();
-
-  void setBaselineFilename(String baselineFilename);
-
-  @Description("Filename of summary perf data to append to.")
-  @Nullable
-  String getSummaryFilename();
-
-  void setSummaryFilename(String summaryFilename);
-
-  @Description("Filename for javascript capturing all perf data and any baselines.")
-  @Nullable
-  String getJavascriptFilename();
-
-  void setJavascriptFilename(String javascriptFilename);
-
-  @Description("If true, don't run the actual query. Instead, calculate the distribution "
-               + "of number of query results per (event time) minute according to the query model.")
-  @Nullable
-  boolean getJustModelResultRate();
-
-  void setJustModelResultRate(boolean justModelResultRate);
-
-  @Description("Coder strategy to use.")
-  @Nullable
-  NexmarkUtils.CoderStrategy getCoderStrategy();
-
-  void setCoderStrategy(NexmarkUtils.CoderStrategy coderStrategy);
-
-  @Description("Delay, in milliseconds, for each event. We will peg one core for this "
-               + "number of milliseconds to simulate CPU-bound computation.")
-  @Nullable
-  Long getCpuDelayMs();
-
-  void setCpuDelayMs(Long cpuDelayMs);
-
-  @Description("Extra data, in bytes, to save to persistent state for each event. "
-               + "This will force I/O all the way to durable storage to simulate an "
-               + "I/O-bound computation.")
-  @Nullable
-  Long getDiskBusyBytes();
-
-  void setDiskBusyBytes(Long diskBusyBytes);
-
-  @Description("Skip factor for query 2. We select bids for every {@code auctionSkip}'th auction")
-  @Nullable
-  Integer getAuctionSkip();
-
-  void setAuctionSkip(Integer auctionSkip);
-
-  @Description("Fanout for queries 4 (groups by category id) and 7 (finds a global maximum).")
-  @Nullable
-  Integer getFanout();
-
-  void setFanout(Integer fanout);
-
-  @Description("Maximum waiting time to clean personState in query3 "
-      + "(ie maximum waiting of the auctions related to person in state in seconds in event time).")
-  @Nullable
-  Integer getMaxAuctionsWaitingTime();
-
-  void setMaxAuctionsWaitingTime(Integer fanout);
-
-  @Description("Length of occasional delay to impose on events (in seconds).")
-  @Nullable
-  Long getOccasionalDelaySec();
-
-  void setOccasionalDelaySec(Long occasionalDelaySec);
-
-  @Description("Probability that an event will be delayed by delayS.")
-  @Nullable
-  Double getProbDelayedEvent();
-
-  void setProbDelayedEvent(Double probDelayedEvent);
-
-  @Description("Maximum size of each log file (in events). For Query10 only.")
-  @Nullable
-  Integer getMaxLogEvents();
-
-  void setMaxLogEvents(Integer maxLogEvents);
-
-  @Description("How to derive names of resources.")
-  @Default.Enum("QUERY_AND_SALT")
-  NexmarkUtils.ResourceNameMode getResourceNameMode();
-
-  void setResourceNameMode(NexmarkUtils.ResourceNameMode mode);
-
-  @Description("If true, manage the creation and cleanup of topics, subscriptions and gcs files.")
-  @Default.Boolean(true)
-  boolean getManageResources();
-
-  void setManageResources(boolean manageResources);
-
-  @Description("If true, use pub/sub publish time instead of event time.")
-  @Nullable
-  Boolean getUsePubsubPublishTime();
-
-  void setUsePubsubPublishTime(Boolean usePubsubPublishTime);
-
-  @Description("Number of events in out-of-order groups. 1 implies no out-of-order events. "
-               + "1000 implies every 1000 events per generator are emitted in pseudo-random order.")
-  @Nullable
-  Long getOutOfOrderGroupSize();
-
-  void setOutOfOrderGroupSize(Long outOfOrderGroupSize);
-
-  @Description("If false, do not add the Monitor and Snoop transforms.")
-  @Nullable
-  Boolean getDebug();
-
-  void setDebug(Boolean value);
-
-  @Description("If set, cancel running pipelines after this long")
-  @Nullable
-  Long getRunningTimeMinutes();
-
-  void setRunningTimeMinutes(Long value);
-
-  @Description("If set and --monitorJobs is true, check that the system watermark is never more "
-               + "than this far behind real time")
-  @Nullable
-  Long getMaxSystemLagSeconds();
-
-  void setMaxSystemLagSeconds(Long value);
-
-  @Description("If set and --monitorJobs is true, check that the data watermark is never more "
-               + "than this far behind real time")
-  @Nullable
-  Long getMaxDataLagSeconds();
-
-  void setMaxDataLagSeconds(Long value);
-
-  @Description("Only start validating watermarks after this many seconds")
-  @Nullable
-  Long getWatermarkValidationDelaySeconds();
-
-  void setWatermarkValidationDelaySeconds(Long value);
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/f4333df7/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkPerf.java
----------------------------------------------------------------------
diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkPerf.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkPerf.java
deleted file mode 100644
index e7f59c8..0000000
--- a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkPerf.java
+++ /dev/null
@@ -1,208 +0,0 @@
-/*
- * 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.beam.integration.nexmark;
-
-import com.fasterxml.jackson.annotation.JsonProperty;
-import com.fasterxml.jackson.core.JsonProcessingException;
-
-import java.io.IOException;
-import java.util.List;
-import javax.annotation.Nullable;
-
-/**
- * Summary of performance for a particular run of a configuration.
- */
-public class NexmarkPerf {
-  /**
-   * A sample of the number of events and number of results (if known) generated at
-   * a particular time.
-   */
-  public static class ProgressSnapshot {
-    /** Seconds since job was started (in wallclock time). */
-    @JsonProperty
-    double secSinceStart;
-
-    /** Job runtime in seconds (time from first event to last generated event or output result). */
-    @JsonProperty
-    double runtimeSec;
-
-    /** Cumulative number of events generated. -1 if not known. */
-    @JsonProperty
-    long numEvents;
-
-    /** Cumulative number of results emitted. -1 if not known. */
-    @JsonProperty
-    long numResults;
-
-    /**
-     * Return true if there looks to be activity between {@code this} and {@code that}
-     * snapshots.
-     */
-    public boolean anyActivity(ProgressSnapshot that) {
-      if (runtimeSec != that.runtimeSec) {
-        // An event or result end timestamp looks to have changed.
-        return true;
-      }
-      if (numEvents != that.numEvents) {
-        // Some more events were generated.
-        return true;
-      }
-      if (numResults != that.numResults) {
-        // Some more results were emitted.
-        return true;
-      }
-      return false;
-    }
-  }
-
-  /**
-   * Progess snapshots. Null if not yet calculated.
-   */
-  @JsonProperty
-  @Nullable
-  public List<ProgressSnapshot> snapshots = null;
-
-  /**
-   * Effective runtime, in seconds. Measured from timestamp of first generated event to latest of
-   * timestamp of last generated event and last emitted result. -1 if not known.
-   */
-  @JsonProperty
-  public double runtimeSec = -1.0;
-
-  /**
-   * Number of events generated. -1 if not known.
-   */
-  @JsonProperty
-  public long numEvents = -1;
-
-  /**
-   * Number of events generated per second of runtime. For batch this is number of events
-   * over the above runtime. For streaming this is the 'steady-state' event generation rate sampled
-   * over the lifetime of the job. -1 if not known.
-   */
-  @JsonProperty
-  public double eventsPerSec = -1.0;
-
-  /**
-   * Number of event bytes generated per second of runtime. -1 if not known.
-   */
-  @JsonProperty
-  public double eventBytesPerSec = -1.0;
-
-  /**
-   * Number of results emitted. -1 if not known.
-   */
-  @JsonProperty
-  public long numResults = -1;
-
-  /**
-   * Number of results generated per second of runtime. -1 if not known.
-   */
-  @JsonProperty
-  public double resultsPerSec = -1.0;
-
-  /**
-   * Number of result bytes generated per second of runtime. -1 if not known.
-   */
-  @JsonProperty
-  public double resultBytesPerSec = -1.0;
-
-  /**
-   * Delay between start of job and first event in second. -1 if not known.
-   */
-  @JsonProperty
-  public double startupDelaySec = -1.0;
-
-  /**
-   * Delay between first event and first result in seconds. -1 if not known.
-   */
-  @JsonProperty
-  public double processingDelaySec = -1.0;
-
-  /**
-   * Delay between last result and job completion in seconds. -1 if not known.
-   */
-  @JsonProperty
-  public double shutdownDelaySec = -1.0;
-
-  /**
-   * Time-dilation factor.  Calculate as event time advancement rate relative to real time.
-   * Greater than one implies we processed events faster than they would have been generated
-   * in real time. Less than one implies we could not keep up with events in real time.
-   * -1 if not known.
-   */
-  @JsonProperty
-  double timeDilation = -1.0;
-
-  /**
-   * List of errors encountered during job execution.
-   */
-  @JsonProperty
-  @Nullable
-  public List<String> errors = null;
-
-  /**
-   * The job id this perf was drawn from. Null if not known.
-   */
-  @JsonProperty
-  @Nullable
-  public String jobId = null;
-
-  /**
-   * Return a JSON representation of performance.
-   */
-  @Override
-  public String toString() {
-    try {
-      return NexmarkUtils.MAPPER.writeValueAsString(this);
-    } catch (JsonProcessingException e) {
-      throw new RuntimeException(e);
-    }
-  }
-
-  /**
-   * Parse a {@link NexmarkPerf} object from JSON {@code string}.
-   */
-  public static NexmarkPerf fromString(String string) {
-    try {
-      return NexmarkUtils.MAPPER.readValue(string, NexmarkPerf.class);
-    } catch (IOException e) {
-      throw new RuntimeException("Unable to parse nexmark perf: ", e);
-    }
-  }
-
-  /**
-   * Return true if there looks to be activity between {@code this} and {@code that}
-   * perf values.
-   */
-  public boolean anyActivity(NexmarkPerf that) {
-    if (runtimeSec != that.runtimeSec) {
-      // An event or result end timestamp looks to have changed.
-      return true;
-    }
-    if (numEvents != that.numEvents) {
-      // Some more events were generated.
-      return true;
-    }
-    if (numResults != that.numResults) {
-      // Some more results were emitted.
-      return true;
-    }
-    return false;
-  }
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/f4333df7/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkSuite.java
----------------------------------------------------------------------
diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkSuite.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkSuite.java
deleted file mode 100644
index 0d98a5d..0000000
--- a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkSuite.java
+++ /dev/null
@@ -1,112 +0,0 @@
-/*
- * 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.beam.integration.nexmark;
-
-import java.util.ArrayList;
-import java.util.LinkedHashSet;
-import java.util.List;
-import java.util.Set;
-
-/**
- * A set of {@link NexmarkConfiguration}s.
- */
-public enum NexmarkSuite {
-  /**
-   * The default.
-   */
-  DEFAULT(defaultConf()),
-
-  /**
-   * Sweep through all queries using the default configuration.
-   * 100k/10k events (depending on query).
-   */
-  SMOKE(smoke()),
-
-  /**
-   * As for SMOKE, but with 10m/1m events.
-   */
-  STRESS(stress()),
-
-  /**
-   * As for SMOKE, but with 1b/100m events.
-   */
-  FULL_THROTTLE(fullThrottle());
-
-  private static List<NexmarkConfiguration> defaultConf() {
-    List<NexmarkConfiguration> configurations = new ArrayList<>();
-    NexmarkConfiguration configuration = new NexmarkConfiguration();
-    configurations.add(configuration);
-    return configurations;
-  }
-
-  private static List<NexmarkConfiguration> smoke() {
-    List<NexmarkConfiguration> configurations = new ArrayList<>();
-    for (int query = 0; query <= 12; query++) {
-      NexmarkConfiguration configuration = NexmarkConfiguration.DEFAULT.copy();
-      configuration.query = query;
-      configuration.numEvents = 100_000;
-      if (query == 4 || query == 6 || query == 9) {
-        // Scale back so overall runtimes are reasonably close across all queries.
-        configuration.numEvents /= 10;
-      }
-      configurations.add(configuration);
-    }
-    return configurations;
-  }
-
-  private static List<NexmarkConfiguration> stress() {
-    List<NexmarkConfiguration> configurations = smoke();
-    for (NexmarkConfiguration configuration : configurations) {
-      if (configuration.numEvents >= 0) {
-        configuration.numEvents *= 1000;
-      }
-    }
-    return configurations;
-  }
-
-  private static List<NexmarkConfiguration> fullThrottle() {
-    List<NexmarkConfiguration> configurations = smoke();
-    for (NexmarkConfiguration configuration : configurations) {
-      if (configuration.numEvents >= 0) {
-        configuration.numEvents *= 1000;
-      }
-    }
-    return configurations;
-  }
-
-  private final List<NexmarkConfiguration> configurations;
-
-  NexmarkSuite(List<NexmarkConfiguration> configurations) {
-    this.configurations = configurations;
-  }
-
-  /**
-   * Return the configurations corresponding to this suite. We'll override each configuration
-   * with any set command line flags, except for --isStreaming which is only respected for
-   * the {@link #DEFAULT} suite.
-   */
-  public Iterable<NexmarkConfiguration> getConfigurations(NexmarkOptions options) {
-    Set<NexmarkConfiguration> results = new LinkedHashSet<>();
-    for (NexmarkConfiguration configuration : configurations) {
-      NexmarkConfiguration result = configuration.copy();
-      result.overrideFromOptions(options);
-      results.add(result);
-    }
-    return results;
-  }
-}


[34/55] [abbrv] beam git commit: Clean some code that is specific to Dataflow

Posted by ie...@apache.org.
Clean some code that is specific to Dataflow


Project: http://git-wip-us.apache.org/repos/asf/beam/repo
Commit: http://git-wip-us.apache.org/repos/asf/beam/commit/77eabbaa
Tree: http://git-wip-us.apache.org/repos/asf/beam/tree/77eabbaa
Diff: http://git-wip-us.apache.org/repos/asf/beam/diff/77eabbaa

Branch: refs/heads/master
Commit: 77eabbaaddad88784c8ce2e775b4b8e8fea3f868
Parents: 902050b
Author: Etienne Chauchot <ec...@gmail.com>
Authored: Fri May 5 15:19:07 2017 +0200
Committer: Ismaël Mejía <ie...@gmail.com>
Committed: Wed Aug 23 19:07:28 2017 +0200

----------------------------------------------------------------------
 .../beam/integration/nexmark/NexmarkRunner.java | 106 -------------------
 1 file changed, 106 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/beam/blob/77eabbaa/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkRunner.java
----------------------------------------------------------------------
diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkRunner.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkRunner.java
index 6df76f0..935544e 100644
--- a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkRunner.java
+++ b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkRunner.java
@@ -157,9 +157,6 @@ public class NexmarkRunner<OptionT extends NexmarkOptions> {
     this.options = options;
   }
 
-  // ================================================================================
-  // Overridden by each runner.
-  // ================================================================================
 
   /**
    * Is this query running in streaming mode?
@@ -414,7 +411,6 @@ public class NexmarkRunner<OptionT extends NexmarkOptions> {
       perf.shutdownDelaySec = (now - resultEnd) / 1000.0;
     }
 
-    perf.jobId = getJobId(result);
     // As soon as available, try to capture cumulative cost at this point too.
 
     NexmarkPerf.ProgressSnapshot snapshot = new NexmarkPerf.ProgressSnapshot();
@@ -429,105 +425,6 @@ public class NexmarkRunner<OptionT extends NexmarkOptions> {
     return perf;
   }
 
-  private String getJobId(PipelineResult job) {
-    return "";
-  }
-
-  // TODO specific to dataflow, see if we can find an equivalent
-/*
-  protected MetricType getMetricType(MetricUpdate metric) {
-    String metricName = metric.getKey().metricName().name();
-    if (metricName.endsWith("windmill-system-watermark")) {
-      return MetricType.SYSTEM_WATERMARK;
-    } else if (metricName.endsWith("windmill-data-watermark")) {
-      return MetricType.DATA_WATERMARK;
-    } else {
-      return MetricType.OTHER;
-    }
-  }
-*/
-
-  /**
-   * Check that watermarks are not too far behind.
-   *
-   * <p>Returns a list of errors detected.
-   */
-  // TODO specific to dataflow, see if we can find an equivalent
-  /*
-  private List<String> checkWatermarks(DataflowPipelineJob job, long startMsSinceEpoch) {
-    long now = System.currentTimeMillis();
-    List<String> errors = new ArrayList<>();
-    try {
-      JobMetrics metricResponse = job.getDataflowClient()
-                                     .projects()
-                                     .jobs()
-                                     .getMetrics(job.getProjectId(), job.getJobId())
-                                     .execute();
-          List<MetricUpdate> metrics = metricResponse.getMetrics();
-
-
-
-      if (metrics != null) {
-        boolean foundWatermarks = false;
-        for (MetricUpdate metric : metrics) {
-          MetricType type = getMetricType(metric);
-          if (type == MetricType.OTHER) {
-            continue;
-          }
-          foundWatermarks = true;
-          @SuppressWarnings("unchecked")
-          BigDecimal scalar = (BigDecimal) metric.getScalar();
-          if (scalar.signum() < 0) {
-            continue;
-          }
-          Instant value =
-                  new Instant(scalar.divideToIntegralValue(new BigDecimal(1000)).longValueExact());
-          Instant updateTime = Instant.parse(metric.getUpdateTime());
-
-          if (options.getWatermarkValidationDelaySeconds() == null
-                  || now > startMsSinceEpoch
-                  + Duration.standardSeconds(options.getWatermarkValidationDelaySeconds())
-                  .getMillis()) {
-            Duration threshold = null;
-            if (type == MetricType.SYSTEM_WATERMARK && options.getMaxSystemLagSeconds() != null) {
-              threshold = Duration.standardSeconds(options.getMaxSystemLagSeconds());
-            } else if (type == MetricType.DATA_WATERMARK
-                    && options.getMaxDataLagSeconds() != null) {
-              threshold = Duration.standardSeconds(options.getMaxDataLagSeconds());
-            }
-
-            if (threshold != null && value.isBefore(updateTime.minus(threshold))) {
-              String msg = String.format("High lag for %s: %s vs %s (allowed lag of %s)",
-                      metric.getKey().metricName().name(), value, updateTime, threshold);
-              errors.add(msg);
-              NexmarkUtils.console(msg);
-            }
-          }
-        }
-        if (!foundWatermarks) {
-          NexmarkUtils.console("No known watermarks in update: " + metrics);
-          if (now > startMsSinceEpoch + Duration.standardMinutes(5).getMillis()) {
-            errors.add("No known watermarks found.  Metrics were " + metrics);
-          }
-        }
-      }
-    } catch (IOException e) {
-      NexmarkUtils.console("Warning: failed to get JobMetrics: " + e);
-    }
-
-    return errors;
-  }
-*/
-
-  // TODO specific to dataflow, see if we can find an equivalent
-/*
-  enum MetricType {
-    SYSTEM_WATERMARK,
-    DATA_WATERMARK,
-    OTHER
-  }
-*/
-
   /**
    * Build and run a pipeline using specified options.
    */
@@ -643,9 +540,6 @@ public class NexmarkRunner<OptionT extends NexmarkOptions> {
               String.format("Streaming query was stuck for %d min", quietFor.getStandardMinutes()));
         }
 
-        // TODO specific to dataflow, see if we can find an equivalent
-//        errors.addAll(checkWatermarks(job, startMsSinceEpoch));
-
         if (waitingForShutdown) {
           try {
             job.cancel();


[42/55] [abbrv] beam git commit: Move module beam-integration-java-nexmark to beam-sdks-java-nexmark

Posted by ie...@apache.org.
http://git-wip-us.apache.org/repos/asf/beam/blob/f4333df7/sdks/java/nexmark/src/main/java/org/apache/beam/sdk/nexmark/queries/Query4.java
----------------------------------------------------------------------
diff --git a/sdks/java/nexmark/src/main/java/org/apache/beam/sdk/nexmark/queries/Query4.java b/sdks/java/nexmark/src/main/java/org/apache/beam/sdk/nexmark/queries/Query4.java
new file mode 100644
index 0000000..3c1cf3b
--- /dev/null
+++ b/sdks/java/nexmark/src/main/java/org/apache/beam/sdk/nexmark/queries/Query4.java
@@ -0,0 +1,116 @@
+/*
+ * 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.beam.sdk.nexmark.queries;
+
+import org.apache.beam.sdk.nexmark.Monitor;
+import org.apache.beam.sdk.nexmark.NexmarkConfiguration;
+import org.apache.beam.sdk.nexmark.NexmarkUtils;
+import org.apache.beam.sdk.nexmark.model.Auction;
+import org.apache.beam.sdk.nexmark.model.AuctionBid;
+import org.apache.beam.sdk.nexmark.model.Bid;
+import org.apache.beam.sdk.nexmark.model.CategoryPrice;
+import org.apache.beam.sdk.nexmark.model.Event;
+import org.apache.beam.sdk.nexmark.model.KnownSize;
+import org.apache.beam.sdk.transforms.DoFn;
+import org.apache.beam.sdk.transforms.Mean;
+import org.apache.beam.sdk.transforms.ParDo;
+import org.apache.beam.sdk.transforms.windowing.SlidingWindows;
+import org.apache.beam.sdk.transforms.windowing.Window;
+import org.apache.beam.sdk.values.KV;
+import org.apache.beam.sdk.values.PCollection;
+import org.joda.time.Duration;
+
+/**
+ * Query 4, 'Average Price for a Category'. Select the average of the wining bid prices for all
+ * closed auctions in each category. In CQL syntax:
+ *
+ * <pre>{@code
+ * SELECT Istream(AVG(Q.final))
+ * FROM Category C, (SELECT Rstream(MAX(B.price) AS final, A.category)
+ *                   FROM Auction A [ROWS UNBOUNDED], Bid B [ROWS UNBOUNDED]
+ *                   WHERE A.id=B.auction AND B.datetime < A.expires AND A.expires < CURRENT_TIME
+ *                   GROUP BY A.id, A.category) Q
+ * WHERE Q.category = C.id
+ * GROUP BY C.id;
+ * }</pre>
+ *
+ * <p>For extra spiciness our implementation differs slightly from the above:
+ * <ul>
+ * <li>We select both the average winning price and the category.
+ * <li>We don't bother joining with a static category table, since it's contents are never used.
+ * <li>We only consider bids which are above the auction's reserve price.
+ * <li>We accept the highest-price, earliest valid bid as the winner.
+ * <li>We calculate the averages oven a sliding window of size {@code windowSizeSec} and
+ * period {@code windowPeriodSec}.
+ * </ul>
+ */
+public class Query4 extends NexmarkQuery {
+  private final Monitor<AuctionBid> winningBidsMonitor;
+
+  public Query4(NexmarkConfiguration configuration) {
+    super(configuration, "Query4");
+    winningBidsMonitor = new Monitor<>(name + ".WinningBids", "winning");
+  }
+
+  private PCollection<CategoryPrice> applyTyped(PCollection<Event> events) {
+    PCollection<AuctionBid> winningBids =
+        events
+            // Find the winning bid for each closed auction.
+            .apply(new WinningBids(name + ".WinningBids", configuration));
+
+    // Monitor winning bids
+    winningBids = winningBids.apply(name + ".WinningBidsMonitor",
+            winningBidsMonitor.getTransform());
+
+    return winningBids
+        // Key the winning bid price by the auction category.
+        .apply(name + ".Rekey",
+            ParDo.of(new DoFn<AuctionBid, KV<Long, Long>>() {
+                  @ProcessElement
+                  public void processElement(ProcessContext c) {
+                    Auction auction = c.element().auction;
+                    Bid bid = c.element().bid;
+                    c.output(KV.of(auction.category, bid.price));
+                  }
+                }))
+
+        // Re-window so we can calculate a sliding average
+        .apply(Window.<KV<Long, Long>>into(
+            SlidingWindows.of(Duration.standardSeconds(configuration.windowSizeSec))
+                .every(Duration.standardSeconds(configuration.windowPeriodSec))))
+
+        // Find the average of the winning bids for each category.
+        // Make sure we share the work for each category between workers.
+        .apply(Mean.<Long, Long>perKey().withHotKeyFanout(configuration.fanout))
+
+        // For testing against Query4Model, capture which results are 'final'.
+        .apply(name + ".Project",
+            ParDo.of(new DoFn<KV<Long, Double>, CategoryPrice>() {
+                  @ProcessElement
+                  public void processElement(ProcessContext c) {
+                    c.output(new CategoryPrice(c.element().getKey(),
+                        Math.round(c.element().getValue()), c.pane().isLast()));
+                  }
+                }));
+  }
+
+  @Override
+  protected PCollection<KnownSize> applyPrim(PCollection<Event> events) {
+    return NexmarkUtils.castToKnownSize(name, applyTyped(events));
+  }
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/f4333df7/sdks/java/nexmark/src/main/java/org/apache/beam/sdk/nexmark/queries/Query4Model.java
----------------------------------------------------------------------
diff --git a/sdks/java/nexmark/src/main/java/org/apache/beam/sdk/nexmark/queries/Query4Model.java b/sdks/java/nexmark/src/main/java/org/apache/beam/sdk/nexmark/queries/Query4Model.java
new file mode 100644
index 0000000..84274a8
--- /dev/null
+++ b/sdks/java/nexmark/src/main/java/org/apache/beam/sdk/nexmark/queries/Query4Model.java
@@ -0,0 +1,186 @@
+/*
+ * 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.beam.sdk.nexmark.queries;
+
+import java.io.Serializable;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.TreeMap;
+import org.apache.beam.sdk.nexmark.NexmarkConfiguration;
+import org.apache.beam.sdk.nexmark.NexmarkUtils;
+import org.apache.beam.sdk.nexmark.model.Auction;
+import org.apache.beam.sdk.nexmark.model.AuctionBid;
+import org.apache.beam.sdk.nexmark.model.Bid;
+import org.apache.beam.sdk.nexmark.model.CategoryPrice;
+import org.apache.beam.sdk.nexmark.model.KnownSize;
+import org.apache.beam.sdk.transforms.windowing.BoundedWindow;
+import org.apache.beam.sdk.values.TimestampedValue;
+import org.joda.time.Duration;
+import org.joda.time.Instant;
+import org.junit.Assert;
+
+/**
+ * A direct implementation of {@link Query4}.
+ */
+public class Query4Model extends NexmarkQueryModel implements Serializable {
+  /**
+   * Simulator for query 4.
+   */
+  private class Simulator extends AbstractSimulator<AuctionBid, CategoryPrice> {
+    /** The prices and categories for all winning bids in the last window size. */
+    private final List<TimestampedValue<CategoryPrice>> winningPricesByCategory;
+
+    /** Timestamp of last result (ms since epoch). */
+    private Instant lastTimestamp;
+
+    /** When oldest active window starts. */
+    private Instant windowStart;
+
+    /** The last seen result for each category. */
+    private final Map<Long, TimestampedValue<CategoryPrice>> lastSeenResults;
+
+    public Simulator(NexmarkConfiguration configuration) {
+      super(new WinningBidsSimulator(configuration).results());
+      winningPricesByCategory = new ArrayList<>();
+      lastTimestamp = BoundedWindow.TIMESTAMP_MIN_VALUE;
+      windowStart = NexmarkUtils.BEGINNING_OF_TIME;
+      lastSeenResults = new TreeMap<>();
+    }
+
+    /**
+     * Calculate the average bid price for each category for all winning bids
+     * which are strictly before {@code end}.
+     */
+    private void averages(Instant end) {
+      Map<Long, Long> counts = new TreeMap<>();
+      Map<Long, Long> totals = new TreeMap<>();
+      for (TimestampedValue<CategoryPrice> value : winningPricesByCategory) {
+        if (!value.getTimestamp().isBefore(end)) {
+          continue;
+        }
+        long category = value.getValue().category;
+        long price = value.getValue().price;
+        Long count = counts.get(category);
+        if (count == null) {
+          count = 1L;
+        } else {
+          count += 1;
+        }
+        counts.put(category, count);
+        Long total = totals.get(category);
+        if (total == null) {
+          total = price;
+        } else {
+          total += price;
+        }
+        totals.put(category, total);
+      }
+      for (Map.Entry<Long, Long> entry : counts.entrySet()) {
+        long category = entry.getKey();
+        long count = entry.getValue();
+        long total = totals.get(category);
+        TimestampedValue<CategoryPrice> result = TimestampedValue.of(
+            new CategoryPrice(category, Math.round((double) total / count), true), lastTimestamp);
+        addIntermediateResult(result);
+        lastSeenResults.put(category, result);
+      }
+    }
+
+    /**
+     * Calculate averages for any windows which can now be retired. Also prune entries
+     * which can no longer contribute to any future window.
+     */
+    private void prune(Instant newWindowStart) {
+      while (!newWindowStart.equals(windowStart)) {
+        averages(windowStart.plus(Duration.standardSeconds(configuration.windowSizeSec)));
+        windowStart = windowStart.plus(Duration.standardSeconds(configuration.windowPeriodSec));
+        Iterator<TimestampedValue<CategoryPrice>> itr = winningPricesByCategory.iterator();
+        while (itr.hasNext()) {
+          if (itr.next().getTimestamp().isBefore(windowStart)) {
+            itr.remove();
+          }
+        }
+        if (winningPricesByCategory.isEmpty()) {
+          windowStart = newWindowStart;
+        }
+      }
+    }
+
+    /**
+     * Capture the winning bid.
+     */
+    private void captureWinningBid(Auction auction, Bid bid, Instant timestamp) {
+      winningPricesByCategory.add(
+          TimestampedValue.of(new CategoryPrice(auction.category, bid.price, false), timestamp));
+    }
+
+    @Override
+    protected void run() {
+      TimestampedValue<AuctionBid> timestampedWinningBid = nextInput();
+      if (timestampedWinningBid == null) {
+        prune(NexmarkUtils.END_OF_TIME);
+        for (TimestampedValue<CategoryPrice> result : lastSeenResults.values()) {
+          addResult(result);
+        }
+        allDone();
+        return;
+      }
+      lastTimestamp = timestampedWinningBid.getTimestamp();
+      Instant newWindowStart = windowStart(Duration.standardSeconds(configuration.windowSizeSec),
+          Duration.standardSeconds(configuration.windowPeriodSec), lastTimestamp);
+      prune(newWindowStart);
+      captureWinningBid(timestampedWinningBid.getValue().auction,
+          timestampedWinningBid.getValue().bid, lastTimestamp);
+    }
+  }
+
+  public Query4Model(NexmarkConfiguration configuration) {
+    super(configuration);
+  }
+
+  @Override
+  public AbstractSimulator<?, ?> simulator() {
+    return new Simulator(configuration);
+  }
+
+  @Override
+  protected Iterable<TimestampedValue<KnownSize>> relevantResults(
+      Iterable<TimestampedValue<KnownSize>> results) {
+    // Find the last (in processing time) reported average price for each category.
+    Map<Long, TimestampedValue<KnownSize>> finalAverages = new TreeMap<>();
+    for (TimestampedValue<KnownSize> obj : results) {
+      Assert.assertTrue("have CategoryPrice", obj.getValue() instanceof CategoryPrice);
+      CategoryPrice categoryPrice = (CategoryPrice) obj.getValue();
+      if (categoryPrice.isLast) {
+        finalAverages.put(
+            categoryPrice.category,
+            TimestampedValue.of((KnownSize) categoryPrice, obj.getTimestamp()));
+      }
+    }
+
+    return finalAverages.values();
+  }
+
+  @Override
+  protected <T> Collection<String> toCollection(Iterator<TimestampedValue<T>> itr) {
+    return toValue(itr);
+  }
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/f4333df7/sdks/java/nexmark/src/main/java/org/apache/beam/sdk/nexmark/queries/Query5.java
----------------------------------------------------------------------
diff --git a/sdks/java/nexmark/src/main/java/org/apache/beam/sdk/nexmark/queries/Query5.java b/sdks/java/nexmark/src/main/java/org/apache/beam/sdk/nexmark/queries/Query5.java
new file mode 100644
index 0000000..d027cb3
--- /dev/null
+++ b/sdks/java/nexmark/src/main/java/org/apache/beam/sdk/nexmark/queries/Query5.java
@@ -0,0 +1,138 @@
+/*
+ * 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.beam.sdk.nexmark.queries;
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+import org.apache.beam.sdk.nexmark.NexmarkConfiguration;
+import org.apache.beam.sdk.nexmark.NexmarkUtils;
+import org.apache.beam.sdk.nexmark.model.AuctionCount;
+import org.apache.beam.sdk.nexmark.model.Bid;
+import org.apache.beam.sdk.nexmark.model.Event;
+import org.apache.beam.sdk.nexmark.model.KnownSize;
+import org.apache.beam.sdk.transforms.Combine;
+import org.apache.beam.sdk.transforms.Count;
+import org.apache.beam.sdk.transforms.DoFn;
+import org.apache.beam.sdk.transforms.ParDo;
+import org.apache.beam.sdk.transforms.windowing.SlidingWindows;
+import org.apache.beam.sdk.transforms.windowing.Window;
+import org.apache.beam.sdk.values.KV;
+import org.apache.beam.sdk.values.PCollection;
+import org.joda.time.Duration;
+
+/**
+ * Query 5, 'Hot Items'. Which auctions have seen the most bids in the last hour (updated every
+ * minute). In CQL syntax:
+ *
+ * <pre>{@code
+ * SELECT Rstream(auction)
+ * FROM (SELECT B1.auction, count(*) AS num
+ *       FROM Bid [RANGE 60 MINUTE SLIDE 1 MINUTE] B1
+ *       GROUP BY B1.auction)
+ * WHERE num >= ALL (SELECT count(*)
+ *                   FROM Bid [RANGE 60 MINUTE SLIDE 1 MINUTE] B2
+ *                   GROUP BY B2.auction);
+ * }</pre>
+ *
+ * <p>To make things a bit more dynamic and easier to test we use much shorter windows, and
+ * we'll also preserve the bid counts.
+ */
+public class Query5 extends NexmarkQuery {
+  public Query5(NexmarkConfiguration configuration) {
+    super(configuration, "Query5");
+  }
+
+  private PCollection<AuctionCount> applyTyped(PCollection<Event> events) {
+    return events
+        // Only want the bid events.
+        .apply(JUST_BIDS)
+        // Window the bids into sliding windows.
+        .apply(
+            Window.<Bid>into(
+                SlidingWindows.of(Duration.standardSeconds(configuration.windowSizeSec))
+                    .every(Duration.standardSeconds(configuration.windowPeriodSec))))
+        // Project just the auction id.
+        .apply("BidToAuction", BID_TO_AUCTION)
+
+        // Count the number of bids per auction id.
+        .apply(Count.<Long>perElement())
+
+        // We'll want to keep all auctions with the maximal number of bids.
+        // Start by lifting each into a singleton list.
+        // need to do so because bellow combine returns a list of auctions in the key in case of
+        // equal number of bids. Combine needs to have same input type and return type.
+        .apply(
+            name + ".ToSingletons",
+            ParDo.of(
+                new DoFn<KV<Long, Long>, KV<List<Long>, Long>>() {
+                  @ProcessElement
+                  public void processElement(ProcessContext c) {
+                    c.output(
+                        KV.of(
+                            Collections.singletonList(c.element().getKey()),
+                            c.element().getValue()));
+                  }
+                }))
+
+        // Keep only the auction ids with the most bids.
+        .apply(
+            Combine.globally(
+                    new Combine.BinaryCombineFn<KV<List<Long>, Long>>() {
+                      @Override
+                      public KV<List<Long>, Long> apply(
+                          KV<List<Long>, Long> left, KV<List<Long>, Long> right) {
+                        List<Long> leftBestAuctions = left.getKey();
+                        long leftCount = left.getValue();
+                        List<Long> rightBestAuctions = right.getKey();
+                        long rightCount = right.getValue();
+                        if (leftCount > rightCount) {
+                          return left;
+                        } else if (leftCount < rightCount) {
+                          return right;
+                        } else {
+                          List<Long> newBestAuctions = new ArrayList<>();
+                          newBestAuctions.addAll(leftBestAuctions);
+                          newBestAuctions.addAll(rightBestAuctions);
+                          return KV.of(newBestAuctions, leftCount);
+                        }
+                      }
+                    })
+                .withoutDefaults()
+                .withFanout(configuration.fanout))
+
+        // Project into result.
+        .apply(
+            name + ".Select",
+            ParDo.of(
+                new DoFn<KV<List<Long>, Long>, AuctionCount>() {
+                  @ProcessElement
+                  public void processElement(ProcessContext c) {
+                    long count = c.element().getValue();
+                    for (long auction : c.element().getKey()) {
+                      c.output(new AuctionCount(auction, count));
+                    }
+                  }
+                }));
+  }
+
+  @Override
+  protected PCollection<KnownSize> applyPrim(PCollection<Event> events) {
+    return NexmarkUtils.castToKnownSize(name, applyTyped(events));
+  }
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/f4333df7/sdks/java/nexmark/src/main/java/org/apache/beam/sdk/nexmark/queries/Query5Model.java
----------------------------------------------------------------------
diff --git a/sdks/java/nexmark/src/main/java/org/apache/beam/sdk/nexmark/queries/Query5Model.java b/sdks/java/nexmark/src/main/java/org/apache/beam/sdk/nexmark/queries/Query5Model.java
new file mode 100644
index 0000000..7ed0709
--- /dev/null
+++ b/sdks/java/nexmark/src/main/java/org/apache/beam/sdk/nexmark/queries/Query5Model.java
@@ -0,0 +1,176 @@
+/*
+ * 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.beam.sdk.nexmark.queries;
+
+import java.io.Serializable;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.TreeMap;
+import org.apache.beam.sdk.nexmark.NexmarkConfiguration;
+import org.apache.beam.sdk.nexmark.NexmarkUtils;
+import org.apache.beam.sdk.nexmark.model.AuctionCount;
+import org.apache.beam.sdk.nexmark.model.Bid;
+import org.apache.beam.sdk.nexmark.model.Event;
+import org.apache.beam.sdk.values.TimestampedValue;
+import org.joda.time.Duration;
+import org.joda.time.Instant;
+
+/**
+ * A direct implementation of {@link Query5}.
+ */
+public class Query5Model extends NexmarkQueryModel implements Serializable {
+  /**
+   * Simulator for query 5.
+   */
+  private class Simulator extends AbstractSimulator<Event, AuctionCount> {
+    /** Time of bids still contributing to open windows, indexed by their auction id. */
+    private final Map<Long, List<Instant>> bids;
+
+    /** When oldest active window starts. */
+    private Instant windowStart;
+
+    public Simulator(NexmarkConfiguration configuration) {
+      super(NexmarkUtils.standardEventIterator(configuration));
+      bids = new TreeMap<>();
+      windowStart = NexmarkUtils.BEGINNING_OF_TIME;
+    }
+
+    /**
+     * Count bids per auction id for bids strictly before {@code end}. Add the auction ids with
+     * the maximum number of bids to results.
+     */
+    private void countBids(Instant end) {
+      Map<Long, Long> counts = new TreeMap<>();
+      long maxCount = 0L;
+      for (Map.Entry<Long, List<Instant>> entry : bids.entrySet()) {
+        long count = 0L;
+        long auction = entry.getKey();
+        for (Instant bid : entry.getValue()) {
+          if (bid.isBefore(end)) {
+            count++;
+          }
+        }
+        if (count > 0) {
+          counts.put(auction, count);
+          maxCount = Math.max(maxCount, count);
+        }
+      }
+      for (Map.Entry<Long, Long> entry : counts.entrySet()) {
+        long auction = entry.getKey();
+        long count = entry.getValue();
+        if (count == maxCount) {
+          AuctionCount result = new AuctionCount(auction, count);
+          addResult(TimestampedValue.of(result, end));
+        }
+      }
+    }
+
+    /**
+     * Retire bids which are strictly before {@code cutoff}. Return true if there are any bids
+     * remaining.
+     */
+    private boolean retireBids(Instant cutoff) {
+      boolean anyRemain = false;
+      for (Map.Entry<Long, List<Instant>> entry : bids.entrySet()) {
+        long auction = entry.getKey();
+        Iterator<Instant> itr = entry.getValue().iterator();
+        while (itr.hasNext()) {
+          Instant bid = itr.next();
+          if (bid.isBefore(cutoff)) {
+            NexmarkUtils.info("retire: %s for %s", bid, auction);
+            itr.remove();
+          } else {
+            anyRemain = true;
+          }
+        }
+      }
+      return anyRemain;
+    }
+
+    /**
+     * Retire active windows until we've reached {@code newWindowStart}.
+     */
+    private void retireWindows(Instant newWindowStart) {
+      while (!newWindowStart.equals(windowStart)) {
+        NexmarkUtils.info("retiring window %s, aiming for %s", windowStart, newWindowStart);
+        // Count bids in the window (windowStart, windowStart + size].
+        countBids(windowStart.plus(Duration.standardSeconds(configuration.windowSizeSec)));
+        // Advance the window.
+        windowStart = windowStart.plus(Duration.standardSeconds(configuration.windowPeriodSec));
+        // Retire bids which will never contribute to a future window.
+        if (!retireBids(windowStart)) {
+          // Can fast forward to latest window since no more outstanding bids.
+          windowStart = newWindowStart;
+        }
+      }
+    }
+
+    /**
+     * Add bid to state.
+     */
+    private void captureBid(Bid bid, Instant timestamp) {
+      List<Instant> existing = bids.get(bid.auction);
+      if (existing == null) {
+        existing = new ArrayList<>();
+        bids.put(bid.auction, existing);
+      }
+      existing.add(timestamp);
+    }
+
+    @Override
+    public void run() {
+      TimestampedValue<Event> timestampedEvent = nextInput();
+      if (timestampedEvent == null) {
+        // Drain the remaining windows.
+        retireWindows(NexmarkUtils.END_OF_TIME);
+        allDone();
+        return;
+      }
+
+      Event event = timestampedEvent.getValue();
+      if (event.bid == null) {
+        // Ignore non-bid events.
+        return;
+      }
+      Instant timestamp = timestampedEvent.getTimestamp();
+      Instant newWindowStart = windowStart(Duration.standardSeconds(configuration.windowSizeSec),
+          Duration.standardSeconds(configuration.windowPeriodSec), timestamp);
+      // Capture results from any windows we can now retire.
+      retireWindows(newWindowStart);
+      // Capture current bid.
+      captureBid(event.bid, timestamp);
+    }
+  }
+
+  public Query5Model(NexmarkConfiguration configuration) {
+    super(configuration);
+  }
+
+  @Override
+  public AbstractSimulator<?, ?> simulator() {
+    return new Simulator(configuration);
+  }
+
+  @Override
+  protected <T> Collection<String> toCollection(Iterator<TimestampedValue<T>> itr) {
+    return toValue(itr);
+  }
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/f4333df7/sdks/java/nexmark/src/main/java/org/apache/beam/sdk/nexmark/queries/Query6.java
----------------------------------------------------------------------
diff --git a/sdks/java/nexmark/src/main/java/org/apache/beam/sdk/nexmark/queries/Query6.java b/sdks/java/nexmark/src/main/java/org/apache/beam/sdk/nexmark/queries/Query6.java
new file mode 100644
index 0000000..bc6b12c
--- /dev/null
+++ b/sdks/java/nexmark/src/main/java/org/apache/beam/sdk/nexmark/queries/Query6.java
@@ -0,0 +1,155 @@
+/*
+ * 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.beam.sdk.nexmark.queries;
+
+import com.google.common.collect.Lists;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+import org.apache.beam.sdk.nexmark.NexmarkConfiguration;
+import org.apache.beam.sdk.nexmark.NexmarkUtils;
+import org.apache.beam.sdk.nexmark.model.Auction;
+import org.apache.beam.sdk.nexmark.model.AuctionBid;
+import org.apache.beam.sdk.nexmark.model.Bid;
+import org.apache.beam.sdk.nexmark.model.Event;
+import org.apache.beam.sdk.nexmark.model.KnownSize;
+import org.apache.beam.sdk.nexmark.model.SellerPrice;
+import org.apache.beam.sdk.transforms.Combine;
+import org.apache.beam.sdk.transforms.DoFn;
+import org.apache.beam.sdk.transforms.ParDo;
+import org.apache.beam.sdk.transforms.windowing.AfterPane;
+import org.apache.beam.sdk.transforms.windowing.GlobalWindows;
+import org.apache.beam.sdk.transforms.windowing.Repeatedly;
+import org.apache.beam.sdk.transforms.windowing.Window;
+import org.apache.beam.sdk.values.KV;
+import org.apache.beam.sdk.values.PCollection;
+import org.joda.time.Duration;
+
+/**
+ * Query 6, 'Average Selling Price by Seller'. Select the average selling price over the
+ * last 10 closed auctions by the same seller. In CQL syntax:
+ *
+ * <pre>{@code
+ * SELECT Istream(AVG(Q.final), Q.seller)
+ * FROM (SELECT Rstream(MAX(B.price) AS final, A.seller)
+ *       FROM Auction A [ROWS UNBOUNDED], Bid B [ROWS UNBOUNDED]
+ *       WHERE A.id=B.auction AND B.datetime < A.expires AND A.expires < CURRENT_TIME
+ *       GROUP BY A.id, A.seller) [PARTITION BY A.seller ROWS 10] Q
+ * GROUP BY Q.seller;
+ * }</pre>
+ *
+ * <p>We are a little more exact with selecting winning bids: see {@link WinningBids}.
+ */
+public class Query6 extends NexmarkQuery {
+  /**
+   * Combiner to keep track of up to {@code maxNumBids} of the most recent wining bids and calculate
+   * their average selling price.
+   */
+  private static class MovingMeanSellingPrice extends Combine.CombineFn<Bid, List<Bid>, Long> {
+    private final int maxNumBids;
+
+    public MovingMeanSellingPrice(int maxNumBids) {
+      this.maxNumBids = maxNumBids;
+    }
+
+    @Override
+    public List<Bid> createAccumulator() {
+      return new ArrayList<>();
+    }
+
+    @Override
+    public List<Bid> addInput(List<Bid> accumulator, Bid input) {
+      accumulator.add(input);
+      Collections.sort(accumulator, Bid.ASCENDING_TIME_THEN_PRICE);
+      if (accumulator.size() > maxNumBids) {
+        accumulator.remove(0);
+      }
+      return accumulator;
+    }
+
+    @Override
+    public List<Bid> mergeAccumulators(Iterable<List<Bid>> accumulators) {
+      List<Bid> result = new ArrayList<>();
+      for (List<Bid> accumulator : accumulators) {
+        result.addAll(accumulator);
+      }
+      Collections.sort(result, Bid.ASCENDING_TIME_THEN_PRICE);
+      if (result.size() > maxNumBids) {
+        result = Lists.newArrayList(result.listIterator(result.size() - maxNumBids));
+      }
+      return result;
+    }
+
+    @Override
+    public Long extractOutput(List<Bid> accumulator) {
+      if (accumulator.isEmpty()) {
+        return 0L;
+      }
+      long sumOfPrice = 0;
+      for (Bid bid : accumulator) {
+        sumOfPrice += bid.price;
+      }
+      return Math.round((double) sumOfPrice / accumulator.size());
+    }
+  }
+
+  public Query6(NexmarkConfiguration configuration) {
+    super(configuration, "Query6");
+  }
+
+  private PCollection<SellerPrice> applyTyped(PCollection<Event> events) {
+    return events
+        // Find the winning bid for each closed auction.
+        .apply(new WinningBids(name + ".WinningBids", configuration))
+
+        // Key the winning bid by the seller id.
+        .apply(name + ".Rekey",
+            ParDo.of(new DoFn<AuctionBid, KV<Long, Bid>>() {
+                  @ProcessElement
+                  public void processElement(ProcessContext c) {
+                    Auction auction = c.element().auction;
+                    Bid bid = c.element().bid;
+                    c.output(KV.of(auction.seller, bid));
+                  }
+                }))
+
+        // Re-window to update on every wining bid.
+        .apply(
+            Window.<KV<Long, Bid>>into(new GlobalWindows())
+                .triggering(Repeatedly.forever(AfterPane.elementCountAtLeast(1)))
+                .accumulatingFiredPanes()
+                .withAllowedLateness(Duration.ZERO))
+
+        // Find the average of last 10 winning bids for each seller.
+        .apply(Combine.<Long, Bid, Long>perKey(new MovingMeanSellingPrice(10)))
+
+        // Project into our datatype.
+        .apply(name + ".Select",
+            ParDo.of(new DoFn<KV<Long, Long>, SellerPrice>() {
+                  @ProcessElement
+                  public void processElement(ProcessContext c) {
+                    c.output(new SellerPrice(c.element().getKey(), c.element().getValue()));
+                  }
+                }));
+  }
+
+  @Override
+  protected PCollection<KnownSize> applyPrim(PCollection<Event> events) {
+    return NexmarkUtils.castToKnownSize(name, applyTyped(events));
+  }
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/f4333df7/sdks/java/nexmark/src/main/java/org/apache/beam/sdk/nexmark/queries/Query6Model.java
----------------------------------------------------------------------
diff --git a/sdks/java/nexmark/src/main/java/org/apache/beam/sdk/nexmark/queries/Query6Model.java b/sdks/java/nexmark/src/main/java/org/apache/beam/sdk/nexmark/queries/Query6Model.java
new file mode 100644
index 0000000..b5152d8
--- /dev/null
+++ b/sdks/java/nexmark/src/main/java/org/apache/beam/sdk/nexmark/queries/Query6Model.java
@@ -0,0 +1,133 @@
+/*
+ * 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.beam.sdk.nexmark.queries;
+
+import java.io.Serializable;
+import java.util.Collection;
+import java.util.Iterator;
+import java.util.Map;
+import java.util.TreeMap;
+import org.apache.beam.sdk.nexmark.NexmarkConfiguration;
+import org.apache.beam.sdk.nexmark.NexmarkUtils;
+import org.apache.beam.sdk.nexmark.model.Auction;
+import org.apache.beam.sdk.nexmark.model.AuctionBid;
+import org.apache.beam.sdk.nexmark.model.Bid;
+import org.apache.beam.sdk.nexmark.model.KnownSize;
+import org.apache.beam.sdk.nexmark.model.SellerPrice;
+import org.apache.beam.sdk.transforms.windowing.BoundedWindow;
+import org.apache.beam.sdk.values.TimestampedValue;
+import org.joda.time.Instant;
+import org.junit.Assert;
+
+/**
+ * A direct implementation of {@link Query6}.
+ */
+public class Query6Model extends NexmarkQueryModel implements Serializable {
+  /**
+   * Simulator for query 6.
+   */
+  private static class Simulator extends AbstractSimulator<AuctionBid, SellerPrice> {
+    /** The cumulative count of winning bids, indexed by seller id. */
+    private final Map<Long, Long> numWinningBidsPerSeller;
+
+    /** The cumulative total of winning bid prices, indexed by seller id. */
+    private final Map<Long, Long> totalWinningBidPricesPerSeller;
+
+    private Instant lastTimestamp;
+
+    public Simulator(NexmarkConfiguration configuration) {
+      super(new WinningBidsSimulator(configuration).results());
+      numWinningBidsPerSeller = new TreeMap<>();
+      totalWinningBidPricesPerSeller = new TreeMap<>();
+      lastTimestamp = BoundedWindow.TIMESTAMP_MIN_VALUE;
+    }
+
+    /**
+     * Update the per-seller running counts/sums.
+     */
+    private void captureWinningBid(Auction auction, Bid bid, Instant timestamp) {
+      NexmarkUtils.info("winning auction, bid: %s, %s", auction, bid);
+      Long count = numWinningBidsPerSeller.get(auction.seller);
+      if (count == null) {
+        count = 1L;
+      } else {
+        count += 1;
+      }
+      numWinningBidsPerSeller.put(auction.seller, count);
+      Long total = totalWinningBidPricesPerSeller.get(auction.seller);
+      if (total == null) {
+        total = bid.price;
+      } else {
+        total += bid.price;
+      }
+      totalWinningBidPricesPerSeller.put(auction.seller, total);
+      TimestampedValue<SellerPrice> intermediateResult = TimestampedValue.of(
+          new SellerPrice(auction.seller, Math.round((double) total / count)), timestamp);
+      addIntermediateResult(intermediateResult);
+    }
+
+
+    @Override
+    protected void run() {
+      TimestampedValue<AuctionBid> timestampedWinningBid = nextInput();
+      if (timestampedWinningBid == null) {
+        for (Map.Entry<Long, Long> entry : numWinningBidsPerSeller.entrySet()) {
+          long seller = entry.getKey();
+          long count = entry.getValue();
+          long total = totalWinningBidPricesPerSeller.get(seller);
+          addResult(TimestampedValue.of(
+              new SellerPrice(seller, Math.round((double) total / count)), lastTimestamp));
+        }
+        allDone();
+        return;
+      }
+
+      lastTimestamp = timestampedWinningBid.getTimestamp();
+      captureWinningBid(timestampedWinningBid.getValue().auction,
+          timestampedWinningBid.getValue().bid, lastTimestamp);
+    }
+  }
+
+  public Query6Model(NexmarkConfiguration configuration) {
+    super(configuration);
+  }
+
+  @Override
+  public AbstractSimulator<?, ?> simulator() {
+    return new Simulator(configuration);
+  }
+
+  @Override
+  protected Iterable<TimestampedValue<KnownSize>> relevantResults(
+      Iterable<TimestampedValue<KnownSize>> results) {
+    // Find the last (in processing time) reported average price for each seller.
+    Map<Long, TimestampedValue<KnownSize>> finalAverages = new TreeMap<>();
+    for (TimestampedValue<KnownSize> obj : results) {
+      Assert.assertTrue("have SellerPrice", obj.getValue() instanceof SellerPrice);
+      SellerPrice sellerPrice = (SellerPrice) obj.getValue();
+      finalAverages.put(
+          sellerPrice.seller, TimestampedValue.of((KnownSize) sellerPrice, obj.getTimestamp()));
+    }
+    return finalAverages.values();
+  }
+
+  @Override
+  protected <T> Collection<String> toCollection(Iterator<TimestampedValue<T>> itr) {
+    return toValue(itr);
+  }
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/f4333df7/sdks/java/nexmark/src/main/java/org/apache/beam/sdk/nexmark/queries/Query7.java
----------------------------------------------------------------------
diff --git a/sdks/java/nexmark/src/main/java/org/apache/beam/sdk/nexmark/queries/Query7.java b/sdks/java/nexmark/src/main/java/org/apache/beam/sdk/nexmark/queries/Query7.java
new file mode 100644
index 0000000..71b75c3
--- /dev/null
+++ b/sdks/java/nexmark/src/main/java/org/apache/beam/sdk/nexmark/queries/Query7.java
@@ -0,0 +1,90 @@
+/*
+ * 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.beam.sdk.nexmark.queries;
+
+import org.apache.beam.sdk.nexmark.NexmarkConfiguration;
+import org.apache.beam.sdk.nexmark.NexmarkUtils;
+import org.apache.beam.sdk.nexmark.model.Bid;
+import org.apache.beam.sdk.nexmark.model.Event;
+import org.apache.beam.sdk.nexmark.model.KnownSize;
+import org.apache.beam.sdk.transforms.DoFn;
+import org.apache.beam.sdk.transforms.Max;
+import org.apache.beam.sdk.transforms.ParDo;
+import org.apache.beam.sdk.transforms.windowing.FixedWindows;
+import org.apache.beam.sdk.transforms.windowing.Window;
+import org.apache.beam.sdk.values.PCollection;
+import org.apache.beam.sdk.values.PCollectionView;
+
+import org.joda.time.Duration;
+
+/**
+ * Query 7, 'Highest Bid'. Select the bids with the highest bid
+ * price in the last minute. In CQL syntax:
+ *
+ * <pre>
+ * SELECT Rstream(B.auction, B.price, B.bidder)
+ * FROM Bid [RANGE 1 MINUTE SLIDE 1 MINUTE] B
+ * WHERE B.price = (SELECT MAX(B1.price)
+ *                  FROM BID [RANGE 1 MINUTE SLIDE 1 MINUTE] B1);
+ * </pre>
+ *
+ * <p>We will use a shorter window to help make testing easier. We'll also implement this using
+ * a side-input in order to exercise that functionality. (A combiner, as used in Query 5, is
+ * a more efficient approach.).
+ */
+public class Query7 extends NexmarkQuery {
+  public Query7(NexmarkConfiguration configuration) {
+    super(configuration, "Query7");
+  }
+
+  private PCollection<Bid> applyTyped(PCollection<Event> events) {
+    // Window the bids.
+    PCollection<Bid> slidingBids = events.apply(JUST_BIDS).apply(
+        Window.<Bid>into(FixedWindows.of(Duration.standardSeconds(configuration.windowSizeSec))));
+
+    // Find the largest price in all bids.
+    // NOTE: It would be more efficient to write this query much as we did for Query5, using
+    // a binary combiner to accumulate the bids with maximal price. As written this query
+    // requires an additional scan per window, with the associated cost of snapshotted state and
+    // its I/O. We'll keep this implementation since it illustrates the use of side inputs.
+    final PCollectionView<Long> maxPriceView =
+        slidingBids
+            .apply("BidToPrice", BID_TO_PRICE)
+            .apply(Max.longsGlobally().withFanout(configuration.fanout).asSingletonView());
+
+    return slidingBids
+        // Select all bids which have that maximum price (there may be more than one).
+        .apply(name + ".Select", ParDo
+          .of(new DoFn<Bid, Bid>() {
+                @ProcessElement
+                public void processElement(ProcessContext c) {
+                  long maxPrice = c.sideInput(maxPriceView);
+                  Bid bid = c.element();
+                  if (bid.price == maxPrice) {
+                    c.output(bid);
+                  }
+                }
+              })
+          .withSideInputs(maxPriceView));
+  }
+
+  @Override
+  protected PCollection<KnownSize> applyPrim(PCollection<Event> events) {
+    return NexmarkUtils.castToKnownSize(name, applyTyped(events));
+  }
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/f4333df7/sdks/java/nexmark/src/main/java/org/apache/beam/sdk/nexmark/queries/Query7Model.java
----------------------------------------------------------------------
diff --git a/sdks/java/nexmark/src/main/java/org/apache/beam/sdk/nexmark/queries/Query7Model.java b/sdks/java/nexmark/src/main/java/org/apache/beam/sdk/nexmark/queries/Query7Model.java
new file mode 100644
index 0000000..4011746
--- /dev/null
+++ b/sdks/java/nexmark/src/main/java/org/apache/beam/sdk/nexmark/queries/Query7Model.java
@@ -0,0 +1,130 @@
+/*
+ * 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.beam.sdk.nexmark.queries;
+
+import java.io.Serializable;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Iterator;
+import java.util.List;
+
+import org.apache.beam.sdk.nexmark.NexmarkConfiguration;
+import org.apache.beam.sdk.nexmark.NexmarkUtils;
+import org.apache.beam.sdk.nexmark.model.Bid;
+import org.apache.beam.sdk.nexmark.model.Event;
+import org.apache.beam.sdk.transforms.windowing.BoundedWindow;
+import org.apache.beam.sdk.values.TimestampedValue;
+import org.joda.time.Duration;
+import org.joda.time.Instant;
+
+/**
+ * A direct implementation of {@link Query7}.
+ */
+public class Query7Model extends NexmarkQueryModel implements Serializable {
+  /**
+   * Simulator for query 7.
+   */
+  private class Simulator extends AbstractSimulator<Event, Bid> {
+    /** Bids with highest bid price seen in the current window. */
+    private final List<Bid> highestBids;
+
+    /** When current window started. */
+    private Instant windowStart;
+
+    private Instant lastTimestamp;
+
+    public Simulator(NexmarkConfiguration configuration) {
+      super(NexmarkUtils.standardEventIterator(configuration));
+      highestBids = new ArrayList<>();
+      windowStart = NexmarkUtils.BEGINNING_OF_TIME;
+      lastTimestamp = BoundedWindow.TIMESTAMP_MIN_VALUE;
+    }
+
+    /**
+     * Transfer the currently winning bids into results and retire them.
+     */
+    private void retireWindow(Instant timestamp) {
+      for (Bid bid : highestBids) {
+        addResult(TimestampedValue.of(bid, timestamp));
+      }
+      highestBids.clear();
+    }
+
+    /**
+     * Keep just the highest price bid.
+     */
+    private void captureBid(Bid bid) {
+      Iterator<Bid> itr = highestBids.iterator();
+      boolean isWinning = true;
+      while (itr.hasNext()) {
+        Bid existingBid = itr.next();
+        if (existingBid.price > bid.price) {
+          isWinning = false;
+          break;
+        }
+        NexmarkUtils.info("smaller price: %s", existingBid);
+        itr.remove();
+      }
+      if (isWinning) {
+        NexmarkUtils.info("larger price: %s", bid);
+        highestBids.add(bid);
+      }
+    }
+
+    @Override
+    protected void run() {
+      TimestampedValue<Event> timestampedEvent = nextInput();
+      if (timestampedEvent == null) {
+        // Capture all remaining bids in results.
+        retireWindow(lastTimestamp);
+        allDone();
+        return;
+      }
+
+      Event event = timestampedEvent.getValue();
+      if (event.bid == null) {
+        // Ignore non-bid events.
+        return;
+      }
+      lastTimestamp = timestampedEvent.getTimestamp();
+      Instant newWindowStart = windowStart(Duration.standardSeconds(configuration.windowSizeSec),
+          Duration.standardSeconds(configuration.windowSizeSec), lastTimestamp);
+      if (!newWindowStart.equals(windowStart)) {
+        // Capture highest priced bids in current window and retire it.
+        retireWindow(lastTimestamp);
+        windowStart = newWindowStart;
+      }
+      // Keep only the highest bids.
+      captureBid(event.bid);
+    }
+  }
+
+  public Query7Model(NexmarkConfiguration configuration) {
+    super(configuration);
+  }
+
+  @Override
+  public AbstractSimulator<?, ?> simulator() {
+    return new Simulator(configuration);
+  }
+
+  @Override
+  protected <T> Collection<String> toCollection(Iterator<TimestampedValue<T>> itr) {
+    return toValueOrder(itr);
+  }
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/f4333df7/sdks/java/nexmark/src/main/java/org/apache/beam/sdk/nexmark/queries/Query8.java
----------------------------------------------------------------------
diff --git a/sdks/java/nexmark/src/main/java/org/apache/beam/sdk/nexmark/queries/Query8.java b/sdks/java/nexmark/src/main/java/org/apache/beam/sdk/nexmark/queries/Query8.java
new file mode 100644
index 0000000..fa3dd86
--- /dev/null
+++ b/sdks/java/nexmark/src/main/java/org/apache/beam/sdk/nexmark/queries/Query8.java
@@ -0,0 +1,97 @@
+/*
+ * 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.beam.sdk.nexmark.queries;
+
+import org.apache.beam.sdk.nexmark.NexmarkConfiguration;
+import org.apache.beam.sdk.nexmark.NexmarkUtils;
+import org.apache.beam.sdk.nexmark.model.Auction;
+import org.apache.beam.sdk.nexmark.model.Event;
+import org.apache.beam.sdk.nexmark.model.IdNameReserve;
+import org.apache.beam.sdk.nexmark.model.KnownSize;
+import org.apache.beam.sdk.nexmark.model.Person;
+import org.apache.beam.sdk.transforms.DoFn;
+import org.apache.beam.sdk.transforms.ParDo;
+import org.apache.beam.sdk.transforms.join.CoGbkResult;
+import org.apache.beam.sdk.transforms.join.CoGroupByKey;
+import org.apache.beam.sdk.transforms.join.KeyedPCollectionTuple;
+import org.apache.beam.sdk.transforms.windowing.FixedWindows;
+import org.apache.beam.sdk.transforms.windowing.Window;
+import org.apache.beam.sdk.values.KV;
+import org.apache.beam.sdk.values.PCollection;
+import org.joda.time.Duration;
+
+/**
+ * Query 8, 'Monitor New Users'. Select people who have entered the system and created auctions
+ * in the last 12 hours, updated every 12 hours. In CQL syntax:
+ *
+ * <pre>
+ * SELECT Rstream(P.id, P.name, A.reserve)
+ * FROM Person [RANGE 12 HOUR] P, Auction [RANGE 12 HOUR] A
+ * WHERE P.id = A.seller;
+ * </pre>
+ *
+ * <p>To make things a bit more dynamic and easier to test we'll use a much shorter window.
+ */
+public class Query8 extends NexmarkQuery {
+  public Query8(NexmarkConfiguration configuration) {
+    super(configuration, "Query8");
+  }
+
+  private PCollection<IdNameReserve> applyTyped(PCollection<Event> events) {
+    // Window and key new people by their id.
+    PCollection<KV<Long, Person>> personsById =
+        events
+          .apply(JUST_NEW_PERSONS)
+          .apply("Query8.WindowPersons",
+            Window.<Person>into(
+              FixedWindows.of(Duration.standardSeconds(configuration.windowSizeSec))))
+            .apply("PersonById", PERSON_BY_ID);
+
+    // Window and key new auctions by their id.
+    PCollection<KV<Long, Auction>> auctionsBySeller =
+        events.apply(JUST_NEW_AUCTIONS)
+          .apply("Query8.WindowAuctions",
+            Window.<Auction>into(
+              FixedWindows.of(Duration.standardSeconds(configuration.windowSizeSec))))
+            .apply("AuctionBySeller", AUCTION_BY_SELLER);
+
+    // Join people and auctions and project the person id, name and auction reserve price.
+    return KeyedPCollectionTuple.of(PERSON_TAG, personsById)
+        .and(AUCTION_TAG, auctionsBySeller)
+        .apply(CoGroupByKey.<Long>create())
+        .apply(name + ".Select",
+            ParDo.of(new DoFn<KV<Long, CoGbkResult>, IdNameReserve>() {
+                  @ProcessElement
+                  public void processElement(ProcessContext c) {
+                    Person person = c.element().getValue().getOnly(PERSON_TAG, null);
+                    if (person == null) {
+                      // Person was not created in last window period.
+                      return;
+                    }
+                    for (Auction auction : c.element().getValue().getAll(AUCTION_TAG)) {
+                      c.output(new IdNameReserve(person.id, person.name, auction.reserve));
+                    }
+                  }
+                }));
+  }
+
+  @Override
+  protected PCollection<KnownSize> applyPrim(PCollection<Event> events) {
+    return NexmarkUtils.castToKnownSize(name, applyTyped(events));
+  }
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/f4333df7/sdks/java/nexmark/src/main/java/org/apache/beam/sdk/nexmark/queries/Query8Model.java
----------------------------------------------------------------------
diff --git a/sdks/java/nexmark/src/main/java/org/apache/beam/sdk/nexmark/queries/Query8Model.java b/sdks/java/nexmark/src/main/java/org/apache/beam/sdk/nexmark/queries/Query8Model.java
new file mode 100644
index 0000000..351cef7
--- /dev/null
+++ b/sdks/java/nexmark/src/main/java/org/apache/beam/sdk/nexmark/queries/Query8Model.java
@@ -0,0 +1,148 @@
+/*
+ * 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.beam.sdk.nexmark.queries;
+
+import com.google.common.collect.ArrayListMultimap;
+import com.google.common.collect.Multimap;
+import java.io.Serializable;
+import java.util.Collection;
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.Map;
+import org.apache.beam.sdk.nexmark.NexmarkConfiguration;
+import org.apache.beam.sdk.nexmark.NexmarkUtils;
+import org.apache.beam.sdk.nexmark.model.Auction;
+import org.apache.beam.sdk.nexmark.model.Event;
+import org.apache.beam.sdk.nexmark.model.IdNameReserve;
+import org.apache.beam.sdk.nexmark.model.Person;
+import org.apache.beam.sdk.values.TimestampedValue;
+import org.joda.time.Duration;
+import org.joda.time.Instant;
+
+/**
+ * A direct implementation of {@link Query8}.
+ */
+public class Query8Model extends NexmarkQueryModel implements Serializable {
+  /**
+   * Simulator for query 8.
+   */
+  private class Simulator extends AbstractSimulator<Event, IdNameReserve> {
+    /** New persons seen in the current window, indexed by id. */
+    private final Map<Long, Person> newPersons;
+
+    /** New auctions seen in the current window, indexed by seller id. */
+    private final Multimap<Long, Auction> newAuctions;
+
+    /** When did the current window start. */
+    private Instant windowStart;
+
+    public Simulator(NexmarkConfiguration configuration) {
+      super(NexmarkUtils.standardEventIterator(configuration));
+      newPersons = new HashMap<>();
+      newAuctions = ArrayListMultimap.create();
+      windowStart = NexmarkUtils.BEGINNING_OF_TIME;
+    }
+
+    /**
+     * Retire all persons added in last window.
+     */
+    private void retirePersons() {
+      for (Map.Entry<Long, Person> entry : newPersons.entrySet()) {
+        NexmarkUtils.info("retire: %s", entry.getValue());
+      }
+      newPersons.clear();
+    }
+
+    /**
+     * Retire all auctions added in last window.
+     */
+    private void retireAuctions() {
+      for (Map.Entry<Long, Auction> entry : newAuctions.entries()) {
+        NexmarkUtils.info("retire: %s", entry.getValue());
+      }
+      newAuctions.clear();
+    }
+
+    /**
+     * Capture new result.
+     */
+    private void addResult(Auction auction, Person person, Instant timestamp) {
+      addResult(TimestampedValue.of(
+          new IdNameReserve(person.id, person.name, auction.reserve), timestamp));
+    }
+
+    @Override
+    public void run() {
+      TimestampedValue<Event> timestampedEvent = nextInput();
+      if (timestampedEvent == null) {
+        allDone();
+        return;
+      }
+
+      Event event = timestampedEvent.getValue();
+      if (event.bid != null) {
+        // Ignore bid events.
+        // Keep looking for next events.
+        return;
+      }
+      Instant timestamp = timestampedEvent.getTimestamp();
+      Instant newWindowStart = windowStart(Duration.standardSeconds(configuration.windowSizeSec),
+          Duration.standardSeconds(configuration.windowSizeSec), timestamp);
+      if (!newWindowStart.equals(windowStart)) {
+        // Retire this window.
+        retirePersons();
+        retireAuctions();
+        windowStart = newWindowStart;
+      }
+
+      if (event.newAuction != null) {
+        // Join new auction with existing person, if any.
+        Person person = newPersons.get(event.newAuction.seller);
+        if (person != null) {
+          addResult(event.newAuction, person, timestamp);
+        } else {
+          // Remember auction for future new people.
+          newAuctions.put(event.newAuction.seller, event.newAuction);
+        }
+      } else { // event is not an auction, nor a bid, so it is a person
+        // Join new person with existing auctions.
+        for (Auction auction : newAuctions.get(event.newPerson.id)) {
+          addResult(auction, event.newPerson, timestamp);
+        }
+        // We'll never need these auctions again.
+        newAuctions.removeAll(event.newPerson.id);
+        // Remember person for future auctions.
+        newPersons.put(event.newPerson.id, event.newPerson);
+      }
+    }
+  }
+
+  public Query8Model(NexmarkConfiguration configuration) {
+    super(configuration);
+  }
+
+  @Override
+  public AbstractSimulator<?, ?> simulator() {
+    return new Simulator(configuration);
+  }
+
+  @Override
+  protected <T> Collection<String> toCollection(Iterator<TimestampedValue<T>> itr) {
+    return toValue(itr);
+  }
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/f4333df7/sdks/java/nexmark/src/main/java/org/apache/beam/sdk/nexmark/queries/Query9.java
----------------------------------------------------------------------
diff --git a/sdks/java/nexmark/src/main/java/org/apache/beam/sdk/nexmark/queries/Query9.java b/sdks/java/nexmark/src/main/java/org/apache/beam/sdk/nexmark/queries/Query9.java
new file mode 100644
index 0000000..5f11e4e
--- /dev/null
+++ b/sdks/java/nexmark/src/main/java/org/apache/beam/sdk/nexmark/queries/Query9.java
@@ -0,0 +1,44 @@
+/*
+ * 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.beam.sdk.nexmark.queries;
+
+import org.apache.beam.sdk.nexmark.NexmarkConfiguration;
+import org.apache.beam.sdk.nexmark.NexmarkUtils;
+import org.apache.beam.sdk.nexmark.model.AuctionBid;
+import org.apache.beam.sdk.nexmark.model.Event;
+import org.apache.beam.sdk.nexmark.model.KnownSize;
+import org.apache.beam.sdk.values.PCollection;
+
+/**
+ * Query "9", 'Winning bids'. Select just the winning bids. Not in original NEXMark suite, but
+ * handy for testing. See {@link WinningBids} for the details.
+ */
+public class Query9 extends NexmarkQuery {
+  public Query9(NexmarkConfiguration configuration) {
+    super(configuration, "Query9");
+  }
+
+  private PCollection<AuctionBid> applyTyped(PCollection<Event> events) {
+    return events.apply(new WinningBids(name, configuration));
+  }
+
+  @Override
+  protected PCollection<KnownSize> applyPrim(PCollection<Event> events) {
+    return NexmarkUtils.castToKnownSize(name, applyTyped(events));
+  }
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/f4333df7/sdks/java/nexmark/src/main/java/org/apache/beam/sdk/nexmark/queries/Query9Model.java
----------------------------------------------------------------------
diff --git a/sdks/java/nexmark/src/main/java/org/apache/beam/sdk/nexmark/queries/Query9Model.java b/sdks/java/nexmark/src/main/java/org/apache/beam/sdk/nexmark/queries/Query9Model.java
new file mode 100644
index 0000000..48d792e
--- /dev/null
+++ b/sdks/java/nexmark/src/main/java/org/apache/beam/sdk/nexmark/queries/Query9Model.java
@@ -0,0 +1,44 @@
+/*
+ * 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.beam.sdk.nexmark.queries;
+
+import java.io.Serializable;
+import java.util.Collection;
+import java.util.Iterator;
+
+import org.apache.beam.sdk.nexmark.NexmarkConfiguration;
+import org.apache.beam.sdk.values.TimestampedValue;
+
+/**
+ * A direct implementation of {@link Query9}.
+ */
+public class Query9Model extends NexmarkQueryModel implements Serializable {
+  public Query9Model(NexmarkConfiguration configuration) {
+    super(configuration);
+  }
+
+  @Override
+  public AbstractSimulator<?, ?> simulator() {
+    return new WinningBidsSimulator(configuration);
+  }
+
+  @Override
+  protected <T> Collection<String> toCollection(Iterator<TimestampedValue<T>> itr) {
+    return toValue(itr);
+  }
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/f4333df7/sdks/java/nexmark/src/main/java/org/apache/beam/sdk/nexmark/queries/WinningBids.java
----------------------------------------------------------------------
diff --git a/sdks/java/nexmark/src/main/java/org/apache/beam/sdk/nexmark/queries/WinningBids.java b/sdks/java/nexmark/src/main/java/org/apache/beam/sdk/nexmark/queries/WinningBids.java
new file mode 100644
index 0000000..816a81f
--- /dev/null
+++ b/sdks/java/nexmark/src/main/java/org/apache/beam/sdk/nexmark/queries/WinningBids.java
@@ -0,0 +1,412 @@
+/*
+ * 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.beam.sdk.nexmark.queries;
+
+import static com.google.common.base.Preconditions.checkState;
+
+import com.fasterxml.jackson.annotation.JsonCreator;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.TreeMap;
+import org.apache.beam.sdk.coders.Coder;
+import org.apache.beam.sdk.coders.CoderException;
+import org.apache.beam.sdk.coders.CustomCoder;
+import org.apache.beam.sdk.coders.VarIntCoder;
+import org.apache.beam.sdk.coders.VarLongCoder;
+import org.apache.beam.sdk.metrics.Counter;
+import org.apache.beam.sdk.metrics.Metrics;
+import org.apache.beam.sdk.nexmark.NexmarkConfiguration;
+import org.apache.beam.sdk.nexmark.NexmarkUtils;
+import org.apache.beam.sdk.nexmark.model.Auction;
+import org.apache.beam.sdk.nexmark.model.AuctionBid;
+import org.apache.beam.sdk.nexmark.model.Bid;
+import org.apache.beam.sdk.nexmark.model.Event;
+import org.apache.beam.sdk.nexmark.sources.GeneratorConfig;
+import org.apache.beam.sdk.transforms.DoFn;
+import org.apache.beam.sdk.transforms.PTransform;
+import org.apache.beam.sdk.transforms.ParDo;
+import org.apache.beam.sdk.transforms.join.CoGbkResult;
+import org.apache.beam.sdk.transforms.join.CoGroupByKey;
+import org.apache.beam.sdk.transforms.join.KeyedPCollectionTuple;
+import org.apache.beam.sdk.transforms.windowing.IntervalWindow;
+import org.apache.beam.sdk.transforms.windowing.Window;
+import org.apache.beam.sdk.transforms.windowing.WindowFn;
+import org.apache.beam.sdk.transforms.windowing.WindowMappingFn;
+import org.apache.beam.sdk.values.KV;
+import org.apache.beam.sdk.values.PCollection;
+import org.joda.time.Instant;
+
+/**
+ * A transform to find the winning bid for each closed auction. In pseudo CQL syntax:
+ *
+ * <pre>{@code
+ * SELECT Rstream(A.*, B.auction, B.bidder, MAX(B.price), B.dateTime)
+ * FROM Auction A [ROWS UNBOUNDED], Bid B [ROWS UNBOUNDED]
+ * WHERE A.id = B.auction AND B.datetime < A.expires AND A.expires < CURRENT_TIME
+ * GROUP BY A.id
+ * }</pre>
+ *
+ * <p>We will also check that the winning bid is above the auction reserve. Note that
+ * we ignore the auction opening bid value since it has no impact on which bid eventually wins,
+ * if any.
+ *
+ * <p>Our implementation will use a custom windowing function in order to bring bids and
+ * auctions together without requiring global state.
+ */
+public class WinningBids extends PTransform<PCollection<Event>, PCollection<AuctionBid>> {
+  /** Windows for open auctions and bids. */
+  private static class AuctionOrBidWindow extends IntervalWindow {
+    /** Id of auction this window is for. */
+    public final long auction;
+
+    /**
+     * True if this window represents an actual auction, and thus has a start/end
+     * time matching that of the auction. False if this window represents a bid, and
+     * thus has an unbounded start/end time.
+     */
+    public final boolean isAuctionWindow;
+
+    /** For avro only. */
+    private AuctionOrBidWindow() {
+      super(TIMESTAMP_MIN_VALUE, TIMESTAMP_MAX_VALUE);
+      auction = 0;
+      isAuctionWindow = false;
+    }
+
+    private AuctionOrBidWindow(
+        Instant start, Instant end, long auctionId, boolean isAuctionWindow) {
+      super(start, end);
+      this.auction = auctionId;
+      this.isAuctionWindow = isAuctionWindow;
+    }
+
+    /** Return an auction window for {@code auction}. */
+    public static AuctionOrBidWindow forAuction(Instant timestamp, Auction auction) {
+      return new AuctionOrBidWindow(timestamp, new Instant(auction.expires), auction.id, true);
+    }
+
+    /**
+     * Return a bid window for {@code bid}. It should later be merged into
+     * the corresponding auction window. However, it is possible this bid is for an already
+     * expired auction, or for an auction which the system has not yet seen. So we
+     * give the bid a bit of wiggle room in its interval.
+     */
+    public static AuctionOrBidWindow forBid(
+        long expectedAuctionDurationMs, Instant timestamp, Bid bid) {
+      // At this point we don't know which auctions are still valid, and the bid may
+      // be for an auction which won't start until some unknown time in the future
+      // (due to Generator.AUCTION_ID_LEAD in Generator.nextBid).
+      // A real system would atomically reconcile bids and auctions by a separate mechanism.
+      // If we give bids an unbounded window it is possible a bid for an auction which
+      // has already expired would cause the system watermark to stall, since that window
+      // would never be retired.
+      // Instead, we will just give the bid a finite window which expires at
+      // the upper bound of auctions assuming the auction starts at the same time as the bid,
+      // and assuming the system is running at its lowest event rate (as per interEventDelayUs).
+      return new AuctionOrBidWindow(
+          timestamp, timestamp.plus(expectedAuctionDurationMs * 2), bid.auction, false);
+    }
+
+    /** Is this an auction window? */
+    public boolean isAuctionWindow() {
+      return isAuctionWindow;
+    }
+
+    @Override
+    public String toString() {
+      return String.format("AuctionOrBidWindow{start:%s; end:%s; auction:%d; isAuctionWindow:%s}",
+          start(), end(), auction, isAuctionWindow);
+    }
+
+    @Override public boolean equals(Object o) {
+      if (this == o) {
+        return true;
+      }
+      if (o == null || getClass() != o.getClass()) {
+        return false;
+      }
+      if (!super.equals(o)) {
+        return false;
+      }
+      AuctionOrBidWindow that = (AuctionOrBidWindow) o;
+      return (isAuctionWindow == that.isAuctionWindow) && (auction == that.auction);
+    }
+
+    @Override public int hashCode() {
+      return Objects.hash(isAuctionWindow, auction);
+    }
+  }
+
+  /**
+   * Encodes an {@link AuctionOrBidWindow} as an {@link IntervalWindow} and an auction id long.
+   */
+  private static class AuctionOrBidWindowCoder extends CustomCoder<AuctionOrBidWindow> {
+    private static final AuctionOrBidWindowCoder INSTANCE = new AuctionOrBidWindowCoder();
+    private static final Coder<IntervalWindow> SUPER_CODER = IntervalWindow.getCoder();
+    private static final Coder<Long> ID_CODER = VarLongCoder.of();
+    private static final Coder<Integer> INT_CODER = VarIntCoder.of();
+
+    @JsonCreator
+    public static AuctionOrBidWindowCoder of() {
+      return INSTANCE;
+    }
+
+    @Override
+    public void encode(AuctionOrBidWindow window, OutputStream outStream)
+        throws IOException, CoderException {
+      SUPER_CODER.encode(window, outStream);
+      ID_CODER.encode(window.auction, outStream);
+      INT_CODER.encode(window.isAuctionWindow ? 1 : 0, outStream);
+    }
+
+    @Override
+    public AuctionOrBidWindow decode(InputStream inStream)
+        throws IOException, CoderException {
+      IntervalWindow superWindow = SUPER_CODER.decode(inStream);
+      long auction = ID_CODER.decode(inStream);
+      boolean isAuctionWindow = INT_CODER.decode(inStream) != 0;
+      return new AuctionOrBidWindow(
+          superWindow.start(), superWindow.end(), auction, isAuctionWindow);
+    }
+
+    @Override public void verifyDeterministic() throws NonDeterministicException {}
+  }
+
+  /** Assign events to auction windows and merges them intelligently. */
+  private static class AuctionOrBidWindowFn extends WindowFn<Event, AuctionOrBidWindow> {
+    /** Expected duration of auctions in ms. */
+    private final long expectedAuctionDurationMs;
+
+    public AuctionOrBidWindowFn(long expectedAuctionDurationMs) {
+      this.expectedAuctionDurationMs = expectedAuctionDurationMs;
+    }
+
+    @Override
+    public Collection<AuctionOrBidWindow> assignWindows(AssignContext c) {
+      Event event = c.element();
+      if (event.newAuction != null) {
+        // Assign auctions to an auction window which expires at the auction's close.
+        return Collections
+            .singletonList(AuctionOrBidWindow.forAuction(c.timestamp(), event.newAuction));
+      } else if (event.bid != null) {
+        // Assign bids to a temporary bid window which will later be merged into the appropriate
+        // auction window.
+        return Collections.singletonList(
+            AuctionOrBidWindow.forBid(expectedAuctionDurationMs, c.timestamp(), event.bid));
+      } else {
+        // Don't assign people to any window. They will thus be dropped.
+        return Collections.emptyList();
+      }
+    }
+
+    @Override
+    public void mergeWindows(MergeContext c) throws Exception {
+      // Split and index the auction and bid windows by auction id.
+      Map<Long, AuctionOrBidWindow> idToTrueAuctionWindow = new TreeMap<>();
+      Map<Long, List<AuctionOrBidWindow>> idToBidAuctionWindows = new TreeMap<>();
+      for (AuctionOrBidWindow window : c.windows()) {
+        if (window.isAuctionWindow()) {
+          idToTrueAuctionWindow.put(window.auction, window);
+        } else {
+          List<AuctionOrBidWindow> bidWindows = idToBidAuctionWindows.get(window.auction);
+          if (bidWindows == null) {
+            bidWindows = new ArrayList<>();
+            idToBidAuctionWindows.put(window.auction, bidWindows);
+          }
+          bidWindows.add(window);
+        }
+      }
+
+      // Merge all 'bid' windows into their corresponding 'auction' window, provided the
+      // auction has not expired.
+      for (Map.Entry<Long, AuctionOrBidWindow> entry : idToTrueAuctionWindow.entrySet()) {
+        long auction = entry.getKey();
+        AuctionOrBidWindow auctionWindow = entry.getValue();
+        List<AuctionOrBidWindow> bidWindows = idToBidAuctionWindows.get(auction);
+        if (bidWindows != null) {
+          List<AuctionOrBidWindow> toBeMerged = new ArrayList<>();
+          for (AuctionOrBidWindow bidWindow : bidWindows) {
+            if (bidWindow.start().isBefore(auctionWindow.end())) {
+              toBeMerged.add(bidWindow);
+            }
+            // else: This bid window will remain until its expire time, at which point it
+            // will expire without ever contributing to an output.
+          }
+          if (!toBeMerged.isEmpty()) {
+            toBeMerged.add(auctionWindow);
+            c.merge(toBeMerged, auctionWindow);
+          }
+        }
+      }
+    }
+
+    @Override
+    public boolean isCompatible(WindowFn<?, ?> other) {
+      return other instanceof AuctionOrBidWindowFn;
+    }
+
+    @Override
+    public Coder<AuctionOrBidWindow> windowCoder() {
+      return AuctionOrBidWindowCoder.of();
+    }
+
+    @Override
+    public WindowMappingFn<AuctionOrBidWindow> getDefaultWindowMappingFn() {
+      throw new UnsupportedOperationException("AuctionWindowFn not supported for side inputs");
+    }
+
+    /**
+     * Below we will GBK auctions and bids on their auction ids. Then we will reduce those
+     * per id to emit {@code (auction, winning bid)} pairs for auctions which have expired with at
+     * least one valid bid. We would like those output pairs to have a timestamp of the auction's
+     * expiry (since that's the earliest we know for sure we have the correct winner). We would
+     * also like to make that winning results are available to following stages at the auction's
+     * expiry.
+     *
+     * <p>Each result of the GBK will have a timestamp of the min of the result of this object's
+     * assignOutputTime over all records which end up in one of its iterables. Thus we get the
+     * desired behavior if we ignore each record's timestamp and always return the auction window's
+     * 'maxTimestamp', which will correspond to the auction's expiry.
+     *
+     * <p>In contrast, if this object's assignOutputTime were to return 'inputTimestamp'
+     * (the usual implementation), then each GBK record will take as its timestamp the minimum of
+     * the timestamps of all bids and auctions within it, which will always be the auction's
+     * timestamp. An auction which expires well into the future would thus hold up the watermark
+     * of the GBK results until that auction expired. That in turn would hold up all winning pairs.
+     */
+    @Override
+    public Instant getOutputTime(
+        Instant inputTimestamp, AuctionOrBidWindow window) {
+      return window.maxTimestamp();
+    }
+  }
+
+  private final AuctionOrBidWindowFn auctionOrBidWindowFn;
+
+  public WinningBids(String name, NexmarkConfiguration configuration) {
+    super(name);
+    // What's the expected auction time (when the system is running at the lowest event rate).
+    long[] interEventDelayUs = configuration.rateShape.interEventDelayUs(
+        configuration.firstEventRate, configuration.nextEventRate,
+        configuration.rateUnit, configuration.numEventGenerators);
+    long longestDelayUs = 0;
+    for (long interEventDelayU : interEventDelayUs) {
+      longestDelayUs = Math.max(longestDelayUs, interEventDelayU);
+    }
+    // Adjust for proportion of auction events amongst all events.
+    longestDelayUs =
+        (longestDelayUs * GeneratorConfig.PROPORTION_DENOMINATOR)
+        / GeneratorConfig.AUCTION_PROPORTION;
+    // Adjust for number of in-flight auctions.
+    longestDelayUs = longestDelayUs * configuration.numInFlightAuctions;
+    long expectedAuctionDurationMs = (longestDelayUs + 999) / 1000;
+    NexmarkUtils.console("Expected auction duration is %d ms", expectedAuctionDurationMs);
+    auctionOrBidWindowFn = new AuctionOrBidWindowFn(expectedAuctionDurationMs);
+  }
+
+  @Override
+  public PCollection<AuctionBid> expand(PCollection<Event> events) {
+    // Window auctions and bids into custom auction windows. New people events will be discarded.
+    // This will allow us to bring bids and auctions together irrespective of how long
+    // each auction is open for.
+    events = events.apply("Window", Window.into(auctionOrBidWindowFn));
+
+    // Key auctions by their id.
+    PCollection<KV<Long, Auction>> auctionsById =
+        events.apply(NexmarkQuery.JUST_NEW_AUCTIONS)
+              .apply("AuctionById:", NexmarkQuery.AUCTION_BY_ID);
+
+    // Key bids by their auction id.
+    PCollection<KV<Long, Bid>> bidsByAuctionId =
+        events.apply(NexmarkQuery.JUST_BIDS).apply("BidByAuction", NexmarkQuery.BID_BY_AUCTION);
+
+    // Find the highest price valid bid for each closed auction.
+    return
+      // Join auctions and bids.
+      KeyedPCollectionTuple.of(NexmarkQuery.AUCTION_TAG, auctionsById)
+        .and(NexmarkQuery.BID_TAG, bidsByAuctionId)
+        .apply(CoGroupByKey.<Long>create())
+        // Filter and select.
+        .apply(name + ".Join",
+          ParDo.of(new DoFn<KV<Long, CoGbkResult>, AuctionBid>() {
+            private final Counter noAuctionCounter = Metrics.counter(name, "noAuction");
+            private final Counter underReserveCounter = Metrics.counter(name, "underReserve");
+            private final Counter noValidBidsCounter = Metrics.counter(name, "noValidBids");
+
+            @ProcessElement
+            public void processElement(ProcessContext c) {
+              Auction auction =
+                  c.element().getValue().getOnly(NexmarkQuery.AUCTION_TAG, null);
+              if (auction == null) {
+                // We have bids without a matching auction. Give up.
+                noAuctionCounter.inc();
+                return;
+              }
+              // Find the current winning bid for auction.
+              // The earliest bid with the maximum price above the reserve wins.
+              Bid bestBid = null;
+              for (Bid bid : c.element().getValue().getAll(NexmarkQuery.BID_TAG)) {
+                // Bids too late for their auction will have been
+                // filtered out by the window merge function.
+                checkState(bid.dateTime < auction.expires);
+                if (bid.price < auction.reserve) {
+                  // Bid price is below auction reserve.
+                  underReserveCounter.inc();
+                  continue;
+                }
+
+                if (bestBid == null
+                    || Bid.PRICE_THEN_DESCENDING_TIME.compare(bid, bestBid) > 0) {
+                  bestBid = bid;
+                }
+              }
+              if (bestBid == null) {
+                // We don't have any valid bids for auction.
+                noValidBidsCounter.inc();
+                return;
+              }
+              c.output(new AuctionBid(auction, bestBid));
+            }
+          }
+        ));
+  }
+
+  @Override
+  public int hashCode() {
+    return Objects.hash(auctionOrBidWindowFn);
+  }
+
+  @Override public boolean equals(Object o) {
+    if (this == o) {
+      return true;
+    }
+    if (o == null || getClass() != o.getClass()) {
+      return false;
+    }
+
+    WinningBids that = (WinningBids) o;
+    return auctionOrBidWindowFn.equals(that.auctionOrBidWindowFn);
+  }
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/f4333df7/sdks/java/nexmark/src/main/java/org/apache/beam/sdk/nexmark/queries/WinningBidsSimulator.java
----------------------------------------------------------------------
diff --git a/sdks/java/nexmark/src/main/java/org/apache/beam/sdk/nexmark/queries/WinningBidsSimulator.java b/sdks/java/nexmark/src/main/java/org/apache/beam/sdk/nexmark/queries/WinningBidsSimulator.java
new file mode 100644
index 0000000..69b64c0
--- /dev/null
+++ b/sdks/java/nexmark/src/main/java/org/apache/beam/sdk/nexmark/queries/WinningBidsSimulator.java
@@ -0,0 +1,206 @@
+/*
+ * 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.beam.sdk.nexmark.queries;
+
+import java.util.ArrayList;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.TreeMap;
+import java.util.TreeSet;
+import javax.annotation.Nullable;
+
+import org.apache.beam.sdk.nexmark.NexmarkConfiguration;
+import org.apache.beam.sdk.nexmark.NexmarkUtils;
+import org.apache.beam.sdk.nexmark.model.Auction;
+import org.apache.beam.sdk.nexmark.model.AuctionBid;
+import org.apache.beam.sdk.nexmark.model.Bid;
+import org.apache.beam.sdk.nexmark.model.Event;
+import org.apache.beam.sdk.transforms.windowing.BoundedWindow;
+import org.apache.beam.sdk.values.TimestampedValue;
+import org.joda.time.Instant;
+
+/**
+ * A simulator of the {@code WinningBids} query.
+ */
+public class WinningBidsSimulator extends AbstractSimulator<Event, AuctionBid> {
+  /** Auctions currently still open, indexed by auction id. */
+  private final Map<Long, Auction> openAuctions;
+
+  /** The ids of auctions known to be closed. */
+  private final Set<Long> closedAuctions;
+
+  /** Current best valid bids for open auctions, indexed by auction id. */
+  private final Map<Long, Bid> bestBids;
+
+  /** Bids for auctions we havn't seen yet. */
+  private final List<Bid> bidsWithoutAuctions;
+
+  /**
+   * Timestamp of last new auction or bid event (ms since epoch).
+   */
+  private long lastTimestamp;
+
+  public WinningBidsSimulator(NexmarkConfiguration configuration) {
+    super(NexmarkUtils.standardEventIterator(configuration));
+    openAuctions = new TreeMap<>();
+    closedAuctions = new TreeSet<>();
+    bestBids = new TreeMap<>();
+    bidsWithoutAuctions = new ArrayList<>();
+    lastTimestamp = BoundedWindow.TIMESTAMP_MIN_VALUE.getMillis();
+  }
+
+  /**
+   * Try to account for {@code bid} in state. Return true if bid has now been
+   * accounted for by {@code bestBids}.
+   */
+  private boolean captureBestBid(Bid bid, boolean shouldLog) {
+    if (closedAuctions.contains(bid.auction)) {
+      // Ignore bids for known, closed auctions.
+      if (shouldLog) {
+        NexmarkUtils.info("closed auction: %s", bid);
+      }
+      return true;
+    }
+    Auction auction = openAuctions.get(bid.auction);
+    if (auction == null) {
+      // We don't have an auction for this bid yet, so can't determine if it is
+      // winning or not.
+      if (shouldLog) {
+        NexmarkUtils.info("pending auction: %s", bid);
+      }
+      return false;
+    }
+    if (bid.price < auction.reserve) {
+      // Bid price is too low.
+      if (shouldLog) {
+        NexmarkUtils.info("below reserve: %s", bid);
+      }
+      return true;
+    }
+    Bid existingBid = bestBids.get(bid.auction);
+    if (existingBid == null || Bid.PRICE_THEN_DESCENDING_TIME.compare(existingBid, bid) < 0) {
+      // We've found a (new) best bid for a known auction.
+      bestBids.put(bid.auction, bid);
+      if (shouldLog) {
+        NexmarkUtils.info("new winning bid: %s", bid);
+      }
+    } else {
+      if (shouldLog) {
+        NexmarkUtils.info("ignoring low bid: %s", bid);
+      }
+    }
+    return true;
+  }
+
+  /**
+   * Try to match bids without auctions to auctions.
+   */
+  private void flushBidsWithoutAuctions() {
+    Iterator<Bid> itr = bidsWithoutAuctions.iterator();
+    while (itr.hasNext()) {
+      Bid bid = itr.next();
+      if (captureBestBid(bid, false)) {
+        NexmarkUtils.info("bid now accounted for: %s", bid);
+        itr.remove();
+      }
+    }
+  }
+
+  /**
+   * Return the next winning bid for an expired auction relative to {@code timestamp}.
+   * Return null if no more winning bids, in which case all expired auctions will
+   * have been removed from our state. Retire auctions in order of expire time.
+   */
+  @Nullable
+  private TimestampedValue<AuctionBid> nextWinningBid(long timestamp) {
+    Map<Long, List<Long>> toBeRetired = new TreeMap<>();
+    for (Map.Entry<Long, Auction> entry : openAuctions.entrySet()) {
+      if (entry.getValue().expires <= timestamp) {
+        List<Long> idsAtTime = toBeRetired.get(entry.getValue().expires);
+        if (idsAtTime == null) {
+          idsAtTime = new ArrayList<>();
+          toBeRetired.put(entry.getValue().expires, idsAtTime);
+        }
+        idsAtTime.add(entry.getKey());
+      }
+    }
+    for (Map.Entry<Long, List<Long>> entry : toBeRetired.entrySet()) {
+      for (long id : entry.getValue()) {
+        Auction auction = openAuctions.get(id);
+        NexmarkUtils.info("retiring auction: %s", auction);
+        openAuctions.remove(id);
+        Bid bestBid = bestBids.get(id);
+        if (bestBid != null) {
+          TimestampedValue<AuctionBid> result =
+              TimestampedValue.of(new AuctionBid(auction, bestBid), new Instant(auction.expires));
+          NexmarkUtils.info("winning: %s", result);
+          return result;
+        }
+      }
+    }
+    return null;
+  }
+
+  @Override
+  protected void run() {
+    if (lastTimestamp > BoundedWindow.TIMESTAMP_MIN_VALUE.getMillis()) {
+      // We may have finally seen the auction a bid was intended for.
+      flushBidsWithoutAuctions();
+      TimestampedValue<AuctionBid> result = nextWinningBid(lastTimestamp);
+      if (result != null) {
+        addResult(result);
+        return;
+      }
+    }
+
+    TimestampedValue<Event> timestampedEvent = nextInput();
+    if (timestampedEvent == null) {
+      // No more events. Flush any still open auctions.
+      TimestampedValue<AuctionBid> result =
+          nextWinningBid(BoundedWindow.TIMESTAMP_MAX_VALUE.getMillis());
+      if (result == null) {
+        // We are done.
+        allDone();
+        return;
+      }
+      addResult(result);
+      return;
+    }
+
+    Event event = timestampedEvent.getValue();
+    if (event.newPerson != null) {
+      // Ignore new person events.
+      return;
+    }
+
+    lastTimestamp = timestampedEvent.getTimestamp().getMillis();
+    if (event.newAuction != null) {
+      // Add this new open auction to our state.
+      openAuctions.put(event.newAuction.id, event.newAuction);
+    } else {
+      if (!captureBestBid(event.bid, true)) {
+        // We don't know what to do with this bid yet.
+        NexmarkUtils.info("bid not yet accounted for: %s", event.bid);
+        bidsWithoutAuctions.add(event.bid);
+      }
+    }
+    // Keep looking for winning bids.
+  }
+}


[21/55] [abbrv] beam git commit: Improve queries tests

Posted by ie...@apache.org.
Improve queries tests

Fix Runner categories in tests

Add streaming unit tests and corresponding labels
issue #37

Update numEvents: results are no more linked to the number of events

issue #22


Project: http://git-wip-us.apache.org/repos/asf/beam/repo
Commit: http://git-wip-us.apache.org/repos/asf/beam/commit/7ef49dc3
Tree: http://git-wip-us.apache.org/repos/asf/beam/tree/7ef49dc3
Diff: http://git-wip-us.apache.org/repos/asf/beam/diff/7ef49dc3

Branch: refs/heads/master
Commit: 7ef49dc3706c3a2543284e17eb39782c783d30cf
Parents: 7c28b49
Author: Etienne Chauchot <ec...@gmail.com>
Authored: Mon Apr 3 16:50:51 2017 +0200
Committer: Ismaël Mejía <ie...@gmail.com>
Committed: Wed Aug 23 19:07:27 2017 +0200

----------------------------------------------------------------------
 .../nexmark/src/main/resources/log4j.properties |   2 +-
 .../integration/nexmark/queries/QueryTest.java  | 142 ++++++++++++++-----
 2 files changed, 110 insertions(+), 34 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/beam/blob/7ef49dc3/integration/java/nexmark/src/main/resources/log4j.properties
----------------------------------------------------------------------
diff --git a/integration/java/nexmark/src/main/resources/log4j.properties b/integration/java/nexmark/src/main/resources/log4j.properties
index 30d0a9d..7dd57b5 100644
--- a/integration/java/nexmark/src/main/resources/log4j.properties
+++ b/integration/java/nexmark/src/main/resources/log4j.properties
@@ -27,7 +27,7 @@ log4j.logger.org.apache.beam.runners.direct=WARN
 log4j.logger.org.apache.beam.sdk=WARN
 
 # Nexmark specific
-log4j.logger.org.apache.beam.integration.nexmark=ALL
+log4j.logger.org.apache.beam.integration.nexmark=WARN
 
 # Settings to quiet third party logs that are too verbose
 log4j.logger.org.spark_project.jetty=WARN

http://git-wip-us.apache.org/repos/asf/beam/blob/7ef49dc3/integration/java/nexmark/src/test/java/org/apache/beam/integration/nexmark/queries/QueryTest.java
----------------------------------------------------------------------
diff --git a/integration/java/nexmark/src/test/java/org/apache/beam/integration/nexmark/queries/QueryTest.java b/integration/java/nexmark/src/test/java/org/apache/beam/integration/nexmark/queries/QueryTest.java
index dca2887..284aa7e 100644
--- a/integration/java/nexmark/src/test/java/org/apache/beam/integration/nexmark/queries/QueryTest.java
+++ b/integration/java/nexmark/src/test/java/org/apache/beam/integration/nexmark/queries/QueryTest.java
@@ -23,6 +23,7 @@ import org.apache.beam.integration.nexmark.NexmarkQueryModel;
 import org.apache.beam.integration.nexmark.NexmarkUtils;
 import org.apache.beam.integration.nexmark.model.KnownSize;
 import org.apache.beam.sdk.PipelineResult;
+import org.apache.beam.sdk.testing.NeedsRunner;
 import org.apache.beam.sdk.testing.PAssert;
 import org.apache.beam.sdk.testing.TestPipeline;
 import org.apache.beam.sdk.testing.UsesStatefulParDo;
@@ -35,81 +36,156 @@ import org.junit.experimental.categories.Category;
 import org.junit.runner.RunWith;
 import org.junit.runners.JUnit4;
 
-/**
- * Test the various NEXMark queries yield results coherent with their models.
- */
+/** Test the various NEXMark queries yield results coherent with their models. */
 @RunWith(JUnit4.class)
 public class QueryTest {
   private static final NexmarkConfiguration CONFIG = NexmarkConfiguration.DEFAULT.clone();
-  @Rule
-  public TestPipeline p = TestPipeline.create();
 
   static {
-    //careful, results of tests are linked to numEvents value
+    // careful, results of tests are linked to numEventGenerators because of timestamp generation
     CONFIG.numEventGenerators = 1;
-    CONFIG.numEvents = 100;
+    CONFIG.numEvents = 1000;
   }
 
+  @Rule public TestPipeline p = TestPipeline.create();
+
   /** Test {@code query} matches {@code model}. */
-  private void queryMatchesModel(String name, NexmarkQuery query, NexmarkQueryModel model) {
+  private void queryMatchesModel(
+      String name, NexmarkQuery query, NexmarkQueryModel model, boolean streamingMode) {
     NexmarkUtils.setupPipeline(NexmarkUtils.CoderStrategy.HAND, p);
-    PCollection<TimestampedValue<KnownSize>> results =
-        p.apply(name + ".ReadBounded", NexmarkUtils.batchEventsSource(CONFIG)).apply(query);
-    //TODO Ismael this should not be called explicitly
-    results.setIsBoundedInternal(PCollection.IsBounded.BOUNDED);
+    PCollection<TimestampedValue<KnownSize>> results;
+    if (streamingMode) {
+      results =
+          p.apply(name + ".ReadUnBounded", NexmarkUtils.streamEventsSource(CONFIG)).apply(query);
+      //TODO Ismael this should not be called explicitly
+      results.setIsBoundedInternal(PCollection.IsBounded.UNBOUNDED);
+    } else {
+      results = p.apply(name + ".ReadBounded", NexmarkUtils.batchEventsSource(CONFIG)).apply(query);
+      //TODO Ismael this should not be called explicitly
+      results.setIsBoundedInternal(PCollection.IsBounded.BOUNDED);
+    }
     PAssert.that(results).satisfies(model.assertionFor());
     PipelineResult result = p.run();
     result.waitUntilFinish();
   }
 
   @Test
-  public void query0MatchesModel() {
-    queryMatchesModel("Query0Test", new Query0(CONFIG), new Query0Model(CONFIG));
+  @Category(NeedsRunner.class)
+  public void query0MatchesModelBatch() {
+    queryMatchesModel("Query0TestBatch", new Query0(CONFIG), new Query0Model(CONFIG), false);
+  }
+
+  @Test
+  @Category(NeedsRunner.class)
+  public void query0MatchesModelStreaming() {
+    queryMatchesModel("Query0TestStreaming", new Query0(CONFIG), new Query0Model(CONFIG), true);
+  }
+
+  @Test
+  @Category(NeedsRunner.class)
+  public void query1MatchesModelBatch() {
+    queryMatchesModel("Query1TestBatch", new Query1(CONFIG), new Query1Model(CONFIG), false);
+  }
+
+  @Test
+  @Category(NeedsRunner.class)
+  public void query1MatchesModelStreaming() {
+    queryMatchesModel("Query1TestStreaming", new Query1(CONFIG), new Query1Model(CONFIG), true);
+  }
+
+  @Test
+  @Category(NeedsRunner.class)
+  public void query2MatchesModelBatch() {
+    queryMatchesModel("Query2TestBatch", new Query2(CONFIG), new Query2Model(CONFIG), false);
+  }
+
+  @Test
+  @Category(NeedsRunner.class)
+  public void query2MatchesModelStreaming() {
+    queryMatchesModel("Query2TestStreaming", new Query2(CONFIG), new Query2Model(CONFIG), true);
+  }
+
+  @Test
+  @Category({NeedsRunner.class, UsesStatefulParDo.class, UsesTimersInParDo.class})
+  public void query3MatchesModelBatch() {
+    queryMatchesModel("Query3TestBatch", new Query3(CONFIG), new Query3Model(CONFIG), false);
+  }
+
+  @Test
+  @Category({NeedsRunner.class, UsesStatefulParDo.class, UsesTimersInParDo.class})
+  public void query3MatchesModelStreaming() {
+    queryMatchesModel("Query3TestStreaming", new Query3(CONFIG), new Query3Model(CONFIG), true);
+  }
+
+  @Test
+  @Category(NeedsRunner.class)
+  public void query4MatchesModelBatch() {
+    queryMatchesModel("Query4TestBatch", new Query4(CONFIG), new Query4Model(CONFIG), false);
+  }
+
+  @Test
+  @Category(NeedsRunner.class)
+  public void query4MatchesModelStreaming() {
+    queryMatchesModel("Query4TestStreaming", new Query4(CONFIG), new Query4Model(CONFIG), true);
+  }
+
+  @Test
+  @Category(NeedsRunner.class)
+  public void query5MatchesModelBatch() {
+    queryMatchesModel("Query5TestBatch", new Query5(CONFIG), new Query5Model(CONFIG), false);
   }
 
   @Test
-  public void query1MatchesModel() {
-    queryMatchesModel("Query1Test", new Query1(CONFIG), new Query1Model(CONFIG));
+  @Category(NeedsRunner.class)
+  public void query5MatchesModelStreaming() {
+    queryMatchesModel("Query5TestStreaming", new Query5(CONFIG), new Query5Model(CONFIG), true);
   }
 
   @Test
-  public void query2MatchesModel() {
-    queryMatchesModel("Query2Test", new Query2(CONFIG), new Query2Model(CONFIG));
+  @Category(NeedsRunner.class)
+  public void query6MatchesModelBatch() {
+    queryMatchesModel("Query6TestBatch", new Query6(CONFIG), new Query6Model(CONFIG), false);
   }
 
   @Test
-  public void query3MatchesModel() {
-    queryMatchesModel("Query3Test", new Query3(CONFIG), new Query3Model(CONFIG));
+  @Category(NeedsRunner.class)
+  public void query6MatchesModelStreaming() {
+    queryMatchesModel("Query6TestStreaming", new Query6(CONFIG), new Query6Model(CONFIG), true);
   }
 
   @Test
-  public void query4MatchesModel() {
-    queryMatchesModel("Query4Test", new Query4(CONFIG), new Query4Model(CONFIG));
+  @Category(NeedsRunner.class)
+  public void query7MatchesModelBatch() {
+    queryMatchesModel("Query7TestBatch", new Query7(CONFIG), new Query7Model(CONFIG), false);
   }
 
   @Test
-  public void query5MatchesModel() {
-    queryMatchesModel("Query5Test", new Query5(CONFIG), new Query5Model(CONFIG));
+  @Category(NeedsRunner.class)
+  public void query7MatchesModelStreaming() {
+    queryMatchesModel("Query7TestStreaming", new Query7(CONFIG), new Query7Model(CONFIG), true);
   }
 
   @Test
-  public void query6MatchesModel() {
-    queryMatchesModel("Query6Test", new Query6(CONFIG), new Query6Model(CONFIG));
+  @Category(NeedsRunner.class)
+  public void query8MatchesModelBatch() {
+    queryMatchesModel("Query8TestBatch", new Query8(CONFIG), new Query8Model(CONFIG), false);
   }
 
   @Test
-  @Category({UsesStatefulParDo.class, UsesTimersInParDo.class})
-  public void query7MatchesModel() {
-    queryMatchesModel("Query7Test", new Query7(CONFIG), new Query7Model(CONFIG));
+  @Category(NeedsRunner.class)
+  public void query8MatchesModelStreaming() {
+    queryMatchesModel("Query8TestStreaming", new Query8(CONFIG), new Query8Model(CONFIG), true);
   }
 
   @Test
-  public void query8MatchesModel() {
-    queryMatchesModel("Query8Test", new Query8(CONFIG), new Query8Model(CONFIG));
+  @Category(NeedsRunner.class)
+  public void query9MatchesModelBatch() {
+    queryMatchesModel("Query9TestBatch", new Query9(CONFIG), new Query9Model(CONFIG), false);
   }
 
   @Test
-  public void query9MatchesModel() {
-    queryMatchesModel("Query9Test", new Query9(CONFIG), new Query9Model(CONFIG));
+  @Category(NeedsRunner.class)
+  public void query9MatchesModelStreaming() {
+    queryMatchesModel("Query9TestStreaming", new Query9(CONFIG), new Query9Model(CONFIG), true);
   }
 }


[26/55] [abbrv] beam git commit: Remove NexmarkDrivers and make execution runner-agnostic

Posted by ie...@apache.org.
http://git-wip-us.apache.org/repos/asf/beam/blob/a6dbdfa5/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/io/PubsubJsonClient.java
----------------------------------------------------------------------
diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/io/PubsubJsonClient.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/io/PubsubJsonClient.java
deleted file mode 100644
index afddbd8..0000000
--- a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/io/PubsubJsonClient.java
+++ /dev/null
@@ -1,318 +0,0 @@
-/*
- * 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.beam.integration.nexmark.io;
-
-import static com.google.common.base.Preconditions.checkState;
-
-import com.google.api.client.http.HttpRequestInitializer;
-import com.google.api.services.pubsub.Pubsub;
-import com.google.api.services.pubsub.Pubsub.Builder;
-import com.google.api.services.pubsub.model.AcknowledgeRequest;
-import com.google.api.services.pubsub.model.ListSubscriptionsResponse;
-import com.google.api.services.pubsub.model.ListTopicsResponse;
-import com.google.api.services.pubsub.model.ModifyAckDeadlineRequest;
-import com.google.api.services.pubsub.model.PublishRequest;
-import com.google.api.services.pubsub.model.PublishResponse;
-import com.google.api.services.pubsub.model.PubsubMessage;
-import com.google.api.services.pubsub.model.PullRequest;
-import com.google.api.services.pubsub.model.PullResponse;
-import com.google.api.services.pubsub.model.ReceivedMessage;
-import com.google.api.services.pubsub.model.Subscription;
-import com.google.api.services.pubsub.model.Topic;
-import com.google.auth.Credentials;
-import com.google.auth.http.HttpCredentialsAdapter;
-import com.google.cloud.hadoop.util.ChainingHttpRequestInitializer;
-import com.google.common.annotations.VisibleForTesting;
-import com.google.common.base.Strings;
-import com.google.common.collect.ImmutableList;
-import java.io.IOException;
-import java.util.ArrayList;
-import java.util.List;
-import java.util.Map;
-import java.util.TreeMap;
-import javax.annotation.Nullable;
-import org.apache.beam.sdk.io.gcp.pubsub.PubsubOptions;
-import org.apache.beam.sdk.util.RetryHttpRequestInitializer;
-import org.apache.beam.sdk.util.Transport;
-
-/**
- * A Pubsub client using JSON transport.
- */
-class PubsubJsonClient extends PubsubClient {
-
-  private static class PubsubJsonClientFactory implements PubsubClientFactory {
-    private static HttpRequestInitializer chainHttpRequestInitializer(
-        Credentials credential, HttpRequestInitializer httpRequestInitializer) {
-      if (credential == null) {
-        return httpRequestInitializer;
-      } else {
-        return new ChainingHttpRequestInitializer(
-            new HttpCredentialsAdapter(credential),
-            httpRequestInitializer);
-      }
-    }
-
-    @Override
-    public PubsubClient newClient(
-        @Nullable String timestampLabel, @Nullable String idLabel, PubsubOptions options)
-        throws IOException {
-      Pubsub pubsub = new Builder(
-          Transport.getTransport(),
-          Transport.getJsonFactory(),
-          chainHttpRequestInitializer(
-              options.getGcpCredential(),
-              // Do not log 404. It clutters the output and is possibly even required by the caller.
-              new RetryHttpRequestInitializer(ImmutableList.of(404))))
-          .setRootUrl(options.getPubsubRootUrl())
-          .setApplicationName(options.getAppName())
-          .setGoogleClientRequestInitializer(options.getGoogleApiTrace())
-          .build();
-      return new PubsubJsonClient(timestampLabel, idLabel, pubsub);
-    }
-
-    @Override
-    public String getKind() {
-      return "Json";
-    }
-  }
-
-  /**
-   * Factory for creating Pubsub clients using Json transport.
-   */
-  public static final PubsubClientFactory FACTORY = new PubsubJsonClientFactory();
-
-  /**
-   * Label to use for custom timestamps, or {@literal null} if should use Pubsub publish time
-   * instead.
-   */
-  @Nullable
-  private final String timestampLabel;
-
-  /**
-   * Label to use for custom ids, or {@literal null} if should use Pubsub provided ids.
-   */
-  @Nullable
-  private final String idLabel;
-
-  /**
-   * Underlying JSON transport.
-   */
-  private Pubsub pubsub;
-
-  @VisibleForTesting PubsubJsonClient(
-      @Nullable String timestampLabel,
-      @Nullable String idLabel,
-      Pubsub pubsub) {
-    this.timestampLabel = timestampLabel;
-    this.idLabel = idLabel;
-    this.pubsub = pubsub;
-  }
-
-  @Override
-  public void close() {
-    // Nothing to close.
-  }
-
-  @Override
-  public int publish(TopicPath topic, List<OutgoingMessage> outgoingMessages)
-      throws IOException {
-    List<PubsubMessage> pubsubMessages = new ArrayList<>(outgoingMessages.size());
-    for (OutgoingMessage outgoingMessage : outgoingMessages) {
-      PubsubMessage pubsubMessage = new PubsubMessage().encodeData(outgoingMessage.elementBytes);
-
-      Map<String, String> attributes = outgoingMessage.attributes;
-      if ((timestampLabel != null || idLabel != null) && attributes == null) {
-        attributes = new TreeMap<>();
-      }
-      if (attributes != null) {
-        pubsubMessage.setAttributes(attributes);
-      }
-
-      if (timestampLabel != null) {
-        attributes.put(timestampLabel, String.valueOf(outgoingMessage.timestampMsSinceEpoch));
-      }
-
-      if (idLabel != null && !Strings.isNullOrEmpty(outgoingMessage.recordId)) {
-        attributes.put(idLabel, outgoingMessage.recordId);
-      }
-
-      pubsubMessages.add(pubsubMessage);
-    }
-    PublishRequest request = new PublishRequest().setMessages(pubsubMessages);
-    PublishResponse response = pubsub.projects()
-                                     .topics()
-                                     .publish(topic.getPath(), request)
-                                     .execute();
-    return response.getMessageIds().size();
-  }
-
-  @Override
-  public List<IncomingMessage> pull(
-      long requestTimeMsSinceEpoch,
-      SubscriptionPath subscription,
-      int batchSize,
-      boolean returnImmediately) throws IOException {
-    PullRequest request = new PullRequest()
-        .setReturnImmediately(returnImmediately)
-        .setMaxMessages(batchSize);
-    PullResponse response = pubsub.projects()
-                                  .subscriptions()
-                                  .pull(subscription.getPath(), request)
-                                  .execute();
-    if (response.getReceivedMessages() == null || response.getReceivedMessages().size() == 0) {
-      return ImmutableList.of();
-    }
-    List<IncomingMessage> incomingMessages = new ArrayList<>(response.getReceivedMessages().size());
-    for (ReceivedMessage message : response.getReceivedMessages()) {
-      PubsubMessage pubsubMessage = message.getMessage();
-      @Nullable Map<String, String> attributes = pubsubMessage.getAttributes();
-
-      // Payload.
-      byte[] elementBytes = pubsubMessage.decodeData();
-
-      // Timestamp.
-      long timestampMsSinceEpoch =
-          extractTimestamp(timestampLabel, message.getMessage().getPublishTime(), attributes);
-
-      // Ack id.
-      String ackId = message.getAckId();
-      checkState(!Strings.isNullOrEmpty(ackId));
-
-      // Record id, if any.
-      @Nullable String recordId = null;
-      if (idLabel != null && attributes != null) {
-        recordId = attributes.get(idLabel);
-      }
-      if (Strings.isNullOrEmpty(recordId)) {
-        // Fall back to the Pubsub provided message id.
-        recordId = pubsubMessage.getMessageId();
-      }
-
-      incomingMessages.add(new IncomingMessage(elementBytes, attributes, timestampMsSinceEpoch,
-                                               requestTimeMsSinceEpoch, ackId, recordId));
-    }
-
-    return incomingMessages;
-  }
-
-  @Override
-  public void acknowledge(SubscriptionPath subscription, List<String> ackIds) throws IOException {
-    AcknowledgeRequest request = new AcknowledgeRequest().setAckIds(ackIds);
-    pubsub.projects()
-          .subscriptions()
-          .acknowledge(subscription.getPath(), request)
-          .execute(); // ignore Empty result.
-  }
-
-  @Override
-  public void modifyAckDeadline(
-      SubscriptionPath subscription, List<String> ackIds, int deadlineSeconds)
-      throws IOException {
-    ModifyAckDeadlineRequest request =
-        new ModifyAckDeadlineRequest().setAckIds(ackIds)
-                                      .setAckDeadlineSeconds(deadlineSeconds);
-    pubsub.projects()
-          .subscriptions()
-          .modifyAckDeadline(subscription.getPath(), request)
-          .execute(); // ignore Empty result.
-  }
-
-  @Override
-  public void createTopic(TopicPath topic) throws IOException {
-    pubsub.projects()
-          .topics()
-          .create(topic.getPath(), new Topic())
-          .execute(); // ignore Topic result.
-  }
-
-  @Override
-  public void deleteTopic(TopicPath topic) throws IOException {
-    pubsub.projects()
-          .topics()
-          .delete(topic.getPath())
-          .execute(); // ignore Empty result.
-  }
-
-  @Override
-  public List<TopicPath> listTopics(ProjectPath project) throws IOException {
-    ListTopicsResponse response = pubsub.projects()
-                                        .topics()
-                                        .list(project.getPath())
-                                        .execute();
-    if (response.getTopics() == null || response.getTopics().isEmpty()) {
-      return ImmutableList.of();
-    }
-    List<TopicPath> topics = new ArrayList<>(response.getTopics().size());
-    for (Topic topic : response.getTopics()) {
-      topics.add(topicPathFromPath(topic.getName()));
-    }
-    return topics;
-  }
-
-  @Override
-  public void createSubscription(
-      TopicPath topic, SubscriptionPath subscription,
-      int ackDeadlineSeconds) throws IOException {
-    Subscription request = new Subscription()
-        .setTopic(topic.getPath())
-        .setAckDeadlineSeconds(ackDeadlineSeconds);
-    pubsub.projects()
-          .subscriptions()
-          .create(subscription.getPath(), request)
-          .execute(); // ignore Subscription result.
-  }
-
-  @Override
-  public void deleteSubscription(SubscriptionPath subscription) throws IOException {
-    pubsub.projects()
-          .subscriptions()
-          .delete(subscription.getPath())
-          .execute(); // ignore Empty result.
-  }
-
-  @Override
-  public List<SubscriptionPath> listSubscriptions(ProjectPath project, TopicPath topic)
-      throws IOException {
-    ListSubscriptionsResponse response = pubsub.projects()
-                                               .subscriptions()
-                                               .list(project.getPath())
-                                               .execute();
-    if (response.getSubscriptions() == null || response.getSubscriptions().isEmpty()) {
-      return ImmutableList.of();
-    }
-    List<SubscriptionPath> subscriptions = new ArrayList<>(response.getSubscriptions().size());
-    for (Subscription subscription : response.getSubscriptions()) {
-      if (subscription.getTopic().equals(topic.getPath())) {
-        subscriptions.add(subscriptionPathFromPath(subscription.getName()));
-      }
-    }
-    return subscriptions;
-  }
-
-  @Override
-  public int ackDeadlineSeconds(SubscriptionPath subscription) throws IOException {
-    Subscription response = pubsub.projects().subscriptions().get(subscription.getPath()).execute();
-    return response.getAckDeadlineSeconds();
-  }
-
-  @Override
-  public boolean isEOF() {
-    return false;
-  }
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/a6dbdfa5/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/io/PubsubTestClient.java
----------------------------------------------------------------------
diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/io/PubsubTestClient.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/io/PubsubTestClient.java
deleted file mode 100644
index 69ba2b0..0000000
--- a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/io/PubsubTestClient.java
+++ /dev/null
@@ -1,436 +0,0 @@
-/*
- * 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.beam.integration.nexmark.io;
-
-import static com.google.common.base.Preconditions.checkState;
-
-import com.google.api.client.util.Clock;
-import com.google.common.collect.Lists;
-import com.google.common.collect.Sets;
-import java.io.Closeable;
-import java.io.IOException;
-import java.io.Serializable;
-import java.util.ArrayList;
-import java.util.HashMap;
-import java.util.Iterator;
-import java.util.List;
-import java.util.Map;
-import java.util.Set;
-import javax.annotation.Nullable;
-import org.apache.beam.sdk.io.gcp.pubsub.PubsubOptions;
-
-/**
- * A (partial) implementation of {@link PubsubClient} for use by unit tests. Only suitable for
- * testing {@link #publish}, {@link #pull}, {@link #acknowledge} and {@link #modifyAckDeadline}
- * methods. Relies on statics to mimic the Pubsub service, though we try to hide that.
- */
-class PubsubTestClient extends PubsubClient implements Serializable {
-  /**
-   * Mimic the state of the simulated Pubsub 'service'.
-   *
-   * <p>Note that the {@link PubsubTestClientFactory} is serialized/deserialized even when running
-   * test pipelines. Meanwhile it is valid for multiple {@link PubsubTestClient}s to be created
-   * from the same client factory and run in parallel. Thus we can't enforce aliasing of the
-   * following data structures over all clients and must resort to a static.
-   */
-  private static class State {
-    /**
-     * True if has been primed for a test but not yet validated.
-     */
-    boolean isActive;
-
-    /**
-     * Publish mode only: Only publish calls for this topic are allowed.
-     */
-    @Nullable
-    TopicPath expectedTopic;
-
-    /**
-     * Publish mode only: Messages yet to seen in a {@link #publish} call.
-     */
-    @Nullable
-    Set<OutgoingMessage> remainingExpectedOutgoingMessages;
-
-    /**
-     * Publish mode only: Messages which should throw when first sent to simulate transient publish
-     * failure.
-     */
-    @Nullable
-    Set<OutgoingMessage> remainingFailingOutgoingMessages;
-
-    /**
-     * Pull mode only: Clock from which to get current time.
-     */
-    @Nullable
-    Clock clock;
-
-    /**
-     * Pull mode only: Only pull calls for this subscription are allowed.
-     */
-    @Nullable
-    SubscriptionPath expectedSubscription;
-
-    /**
-     * Pull mode only: Timeout to simulate.
-     */
-    int ackTimeoutSec;
-
-    /**
-     * Pull mode only: Messages waiting to be received by a {@link #pull} call.
-     */
-    @Nullable
-    List<IncomingMessage> remainingPendingIncomingMessages;
-
-    /**
-     * Pull mode only: Messages which have been returned from a {@link #pull} call and
-     * not yet ACKed by an {@link #acknowledge} call.
-     */
-    @Nullable
-    Map<String, IncomingMessage> pendingAckIncomingMessages;
-
-    /**
-     * Pull mode only: When above messages are due to have their ACK deadlines expire.
-     */
-    @Nullable
-    Map<String, Long> ackDeadline;
-  }
-
-  private static final State STATE = new State();
-
-  /** Closing the factory will validate all expected messages were processed. */
-  public interface PubsubTestClientFactory
-          extends PubsubClientFactory, Closeable, Serializable {
-  }
-
-  /**
-   * Return a factory for testing publishers. Only one factory may be in-flight at a time.
-   * The factory must be closed when the test is complete, at which point final validation will
-   * occur.
-   */
-  static PubsubTestClientFactory createFactoryForPublish(
-      final TopicPath expectedTopic,
-      final Iterable<OutgoingMessage> expectedOutgoingMessages,
-      final Iterable<OutgoingMessage> failingOutgoingMessages) {
-    synchronized (STATE) {
-      checkState(!STATE.isActive, "Test still in flight");
-      STATE.expectedTopic = expectedTopic;
-      STATE.remainingExpectedOutgoingMessages = Sets.newHashSet(expectedOutgoingMessages);
-      STATE.remainingFailingOutgoingMessages = Sets.newHashSet(failingOutgoingMessages);
-      STATE.isActive = true;
-    }
-    return new PubsubTestClientFactory() {
-      @Override
-      public PubsubClient newClient(
-          @Nullable String timestampLabel, @Nullable String idLabel, PubsubOptions options)
-          throws IOException {
-        return new PubsubTestClient();
-      }
-
-      @Override
-      public String getKind() {
-        return "PublishTest";
-      }
-
-      @Override
-      public void close() {
-        synchronized (STATE) {
-          checkState(STATE.isActive, "No test still in flight");
-          checkState(STATE.remainingExpectedOutgoingMessages.isEmpty(),
-                     "Still waiting for %s messages to be published",
-                     STATE.remainingExpectedOutgoingMessages.size());
-          STATE.isActive = false;
-          STATE.remainingExpectedOutgoingMessages = null;
-        }
-      }
-    };
-  }
-
-  /**
-   * Return a factory for testing subscribers. Only one factory may be in-flight at a time.
-   * The factory must be closed when the test in complete
-   */
-  public static PubsubTestClientFactory createFactoryForPull(
-      final Clock clock,
-      final SubscriptionPath expectedSubscription,
-      final int ackTimeoutSec,
-      final Iterable<IncomingMessage> expectedIncomingMessages) {
-    synchronized (STATE) {
-      checkState(!STATE.isActive, "Test still in flight");
-      STATE.clock = clock;
-      STATE.expectedSubscription = expectedSubscription;
-      STATE.ackTimeoutSec = ackTimeoutSec;
-      STATE.remainingPendingIncomingMessages = Lists.newArrayList(expectedIncomingMessages);
-      STATE.pendingAckIncomingMessages = new HashMap<>();
-      STATE.ackDeadline = new HashMap<>();
-      STATE.isActive = true;
-    }
-    return new PubsubTestClientFactory() {
-      @Override
-      public PubsubClient newClient(
-          @Nullable String timestampLabel, @Nullable String idLabel, PubsubOptions options)
-          throws IOException {
-        return new PubsubTestClient();
-      }
-
-      @Override
-      public String getKind() {
-        return "PullTest";
-      }
-
-      @Override
-      public void close() {
-        synchronized (STATE) {
-          checkState(STATE.isActive, "No test still in flight");
-          checkState(STATE.remainingPendingIncomingMessages.isEmpty(),
-                     "Still waiting for %s messages to be pulled",
-                     STATE.remainingPendingIncomingMessages.size());
-          checkState(STATE.pendingAckIncomingMessages.isEmpty(),
-                     "Still waiting for %s messages to be ACKed",
-                     STATE.pendingAckIncomingMessages.size());
-          checkState(STATE.ackDeadline.isEmpty(),
-                     "Still waiting for %s messages to be ACKed",
-                     STATE.ackDeadline.size());
-          STATE.isActive = false;
-          STATE.remainingPendingIncomingMessages = null;
-          STATE.pendingAckIncomingMessages = null;
-          STATE.ackDeadline = null;
-        }
-      }
-    };
-  }
-
-  public static PubsubTestClientFactory createFactoryForCreateSubscription() {
-    return new PubsubTestClientFactory() {
-      int numCalls = 0;
-
-      @Override
-      public void close() throws IOException {
-        checkState(
-            numCalls == 1, "Expected exactly one subscription to be created, got %s", numCalls);
-      }
-
-      @Override
-      public PubsubClient newClient(
-          @Nullable String timestampLabel, @Nullable String idLabel, PubsubOptions options)
-          throws IOException {
-        return new PubsubTestClient() {
-          @Override
-          public void createSubscription(
-              TopicPath topic, SubscriptionPath subscription, int ackDeadlineSeconds)
-              throws IOException {
-            checkState(numCalls == 0, "Expected at most one subscription to be created");
-            numCalls++;
-          }
-        };
-      }
-
-      @Override
-      public String getKind() {
-        return "CreateSubscriptionTest";
-      }
-    };
-  }
-
-  /**
-   * Return true if in pull mode.
-   */
-  private boolean inPullMode() {
-    checkState(STATE.isActive, "No test is active");
-    return STATE.expectedSubscription != null;
-  }
-
-  /**
-   * Return true if in publish mode.
-   */
-  private boolean inPublishMode() {
-    checkState(STATE.isActive, "No test is active");
-    return STATE.expectedTopic != null;
-  }
-
-  /**
-   * For subscription mode only:
-   * Track progression of time according to the {@link Clock} passed . This will simulate Pubsub
-   * expiring
-   * outstanding ACKs.
-   */
-  public void advance() {
-    synchronized (STATE) {
-      checkState(inPullMode(), "Can only advance in pull mode");
-      // Any messages who's ACKs timed out are available for re-pulling.
-      Iterator<Map.Entry<String, Long>> deadlineItr = STATE.ackDeadline.entrySet().iterator();
-      while (deadlineItr.hasNext()) {
-        Map.Entry<String, Long> entry = deadlineItr.next();
-        if (entry.getValue() <= STATE.clock.currentTimeMillis()) {
-          STATE.remainingPendingIncomingMessages.add(
-              STATE.pendingAckIncomingMessages.remove(entry.getKey()));
-          deadlineItr.remove();
-        }
-      }
-    }
-  }
-
-  @Override
-  public void close() {
-  }
-
-  @Override
-  public int publish(
-      TopicPath topic, List<OutgoingMessage> outgoingMessages) throws IOException {
-    synchronized (STATE) {
-      checkState(inPublishMode(), "Can only publish in publish mode");
-      checkState(topic.equals(STATE.expectedTopic), "Topic %s does not match expected %s", topic,
-                 STATE.expectedTopic);
-      for (OutgoingMessage outgoingMessage : outgoingMessages) {
-        if (STATE.remainingFailingOutgoingMessages.remove(outgoingMessage)) {
-          throw new RuntimeException("Simulating failure for " + outgoingMessage);
-        }
-        checkState(STATE.remainingExpectedOutgoingMessages.remove(outgoingMessage),
-                   "Unexpected outgoing message %s", outgoingMessage);
-      }
-      return outgoingMessages.size();
-    }
-  }
-
-  @Override
-  public List<IncomingMessage> pull(
-      long requestTimeMsSinceEpoch, SubscriptionPath subscription, int batchSize,
-      boolean returnImmediately) throws IOException {
-    synchronized (STATE) {
-      checkState(inPullMode(), "Can only pull in pull mode");
-      long now = STATE.clock.currentTimeMillis();
-      checkState(requestTimeMsSinceEpoch == now,
-                 "Simulated time %s does not match request time %s", now, requestTimeMsSinceEpoch);
-      checkState(subscription.equals(STATE.expectedSubscription),
-                 "Subscription %s does not match expected %s", subscription,
-                 STATE.expectedSubscription);
-      checkState(returnImmediately, "Pull only supported if returning immediately");
-
-      List<IncomingMessage> incomingMessages = new ArrayList<>();
-      Iterator<IncomingMessage> pendItr = STATE.remainingPendingIncomingMessages.iterator();
-      while (pendItr.hasNext()) {
-        IncomingMessage incomingMessage = pendItr.next();
-        pendItr.remove();
-        IncomingMessage incomingMessageWithRequestTime =
-            incomingMessage.withRequestTime(requestTimeMsSinceEpoch);
-        incomingMessages.add(incomingMessageWithRequestTime);
-        STATE.pendingAckIncomingMessages.put(incomingMessageWithRequestTime.ackId,
-                                             incomingMessageWithRequestTime);
-        STATE.ackDeadline.put(incomingMessageWithRequestTime.ackId,
-                              requestTimeMsSinceEpoch + STATE.ackTimeoutSec * 1000);
-        if (incomingMessages.size() >= batchSize) {
-          break;
-        }
-      }
-      return incomingMessages;
-    }
-  }
-
-  @Override
-  public void acknowledge(
-      SubscriptionPath subscription,
-      List<String> ackIds) throws IOException {
-    synchronized (STATE) {
-      checkState(inPullMode(), "Can only acknowledge in pull mode");
-      checkState(subscription.equals(STATE.expectedSubscription),
-                 "Subscription %s does not match expected %s", subscription,
-                 STATE.expectedSubscription);
-
-      for (String ackId : ackIds) {
-        checkState(STATE.ackDeadline.remove(ackId) != null,
-                   "No message with ACK id %s is waiting for an ACK", ackId);
-        checkState(STATE.pendingAckIncomingMessages.remove(ackId) != null,
-                   "No message with ACK id %s is waiting for an ACK", ackId);
-      }
-    }
-  }
-
-  @Override
-  public void modifyAckDeadline(
-      SubscriptionPath subscription, List<String> ackIds, int deadlineSeconds) throws IOException {
-    synchronized (STATE) {
-      checkState(inPullMode(), "Can only modify ack deadline in pull mode");
-      checkState(subscription.equals(STATE.expectedSubscription),
-                 "Subscription %s does not match expected %s", subscription,
-                 STATE.expectedSubscription);
-
-      for (String ackId : ackIds) {
-        if (deadlineSeconds > 0) {
-          checkState(STATE.ackDeadline.remove(ackId) != null,
-                     "No message with ACK id %s is waiting for an ACK", ackId);
-          checkState(STATE.pendingAckIncomingMessages.containsKey(ackId),
-                     "No message with ACK id %s is waiting for an ACK", ackId);
-          STATE.ackDeadline.put(ackId, STATE.clock.currentTimeMillis() + deadlineSeconds * 1000);
-        } else {
-          checkState(STATE.ackDeadline.remove(ackId) != null,
-                     "No message with ACK id %s is waiting for an ACK", ackId);
-          IncomingMessage message = STATE.pendingAckIncomingMessages.remove(ackId);
-          checkState(message != null, "No message with ACK id %s is waiting for an ACK", ackId);
-          STATE.remainingPendingIncomingMessages.add(message);
-        }
-      }
-    }
-  }
-
-  @Override
-  public void createTopic(TopicPath topic) throws IOException {
-    throw new UnsupportedOperationException();
-  }
-
-  @Override
-  public void deleteTopic(TopicPath topic) throws IOException {
-    throw new UnsupportedOperationException();
-  }
-
-  @Override
-  public List<TopicPath> listTopics(ProjectPath project) throws IOException {
-    throw new UnsupportedOperationException();
-  }
-
-  @Override
-  public void createSubscription(
-      TopicPath topic, SubscriptionPath subscription, int ackDeadlineSeconds) throws IOException {
-    throw new UnsupportedOperationException();
-  }
-
-  @Override
-  public void deleteSubscription(SubscriptionPath subscription) throws IOException {
-    throw new UnsupportedOperationException();
-  }
-
-  @Override
-  public List<SubscriptionPath> listSubscriptions(
-      ProjectPath project, TopicPath topic) throws IOException {
-    throw new UnsupportedOperationException();
-  }
-
-  @Override
-  public int ackDeadlineSeconds(SubscriptionPath subscription) throws IOException {
-    synchronized (STATE) {
-      return STATE.ackTimeoutSec;
-    }
-  }
-
-  @Override
-  public boolean isEOF() {
-    synchronized (STATE) {
-      checkState(inPullMode(), "Can only check EOF in pull mode");
-      return STATE.remainingPendingIncomingMessages.isEmpty();
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/a6dbdfa5/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/io/package-info.java
----------------------------------------------------------------------
diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/io/package-info.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/io/package-info.java
deleted file mode 100644
index 1161f3e..0000000
--- a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/io/package-info.java
+++ /dev/null
@@ -1,22 +0,0 @@
-/*
- * 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.
- */
-
-/**
- * Nexmark Beam IO related utilities.
- */
-package org.apache.beam.integration.nexmark.io;

http://git-wip-us.apache.org/repos/asf/beam/blob/a6dbdfa5/integration/java/nexmark/src/test/java/org/apache/beam/integration/nexmark/sources/BoundedEventSourceTest.java
----------------------------------------------------------------------
diff --git a/integration/java/nexmark/src/test/java/org/apache/beam/integration/nexmark/sources/BoundedEventSourceTest.java b/integration/java/nexmark/src/test/java/org/apache/beam/integration/nexmark/sources/BoundedEventSourceTest.java
index c5d7725..d95461a 100644
--- a/integration/java/nexmark/src/test/java/org/apache/beam/integration/nexmark/sources/BoundedEventSourceTest.java
+++ b/integration/java/nexmark/src/test/java/org/apache/beam/integration/nexmark/sources/BoundedEventSourceTest.java
@@ -18,7 +18,7 @@
 package org.apache.beam.integration.nexmark.sources;
 
 import org.apache.beam.integration.nexmark.NexmarkConfiguration;
-import org.apache.beam.runners.dataflow.options.DataflowPipelineOptions;
+import org.apache.beam.integration.nexmark.NexmarkOptions;
 import org.apache.beam.sdk.options.PipelineOptionsFactory;
 import org.apache.beam.sdk.testing.SourceTestUtils;
 
@@ -38,7 +38,7 @@ public class BoundedEventSourceTest {
 
   @Test
   public void sourceAndReadersWork() throws Exception {
-    DataflowPipelineOptions options = PipelineOptionsFactory.as(DataflowPipelineOptions.class);
+    NexmarkOptions options = PipelineOptionsFactory.as(NexmarkOptions.class);
     long n = 200L;
     BoundedEventSource source = new BoundedEventSource(makeConfig(n), 1);
 
@@ -48,7 +48,7 @@ public class BoundedEventSourceTest {
 
   @Test
   public void splitAtFractionRespectsContract() throws Exception {
-    DataflowPipelineOptions options = PipelineOptionsFactory.as(DataflowPipelineOptions.class);
+    NexmarkOptions options = PipelineOptionsFactory.as(NexmarkOptions.class);
     long n = 20L;
     BoundedEventSource source = new BoundedEventSource(makeConfig(n), 1);
 
@@ -62,7 +62,7 @@ public class BoundedEventSourceTest {
 
   @Test
   public void splitIntoBundlesRespectsContract() throws Exception {
-    DataflowPipelineOptions options = PipelineOptionsFactory.as(DataflowPipelineOptions.class);
+    NexmarkOptions options = PipelineOptionsFactory.as(NexmarkOptions.class);
     long n = 200L;
     BoundedEventSource source = new BoundedEventSource(makeConfig(n), 1);
     SourceTestUtils.assertSourcesEqualReferenceSource(


[33/55] [abbrv] beam git commit: Migrate to Beam 2.1.0-SNAPSHOT

Posted by ie...@apache.org.
Migrate to Beam 2.1.0-SNAPSHOT


Project: http://git-wip-us.apache.org/repos/asf/beam/repo
Commit: http://git-wip-us.apache.org/repos/asf/beam/commit/3d5c3d00
Tree: http://git-wip-us.apache.org/repos/asf/beam/tree/3d5c3d00
Diff: http://git-wip-us.apache.org/repos/asf/beam/diff/3d5c3d00

Branch: refs/heads/master
Commit: 3d5c3d009b441a8085189f9d4ed1926a4042f816
Parents: 69953a0
Author: Etienne Chauchot <ec...@gmail.com>
Authored: Tue May 9 15:25:54 2017 +0200
Committer: Ismaël Mejía <ie...@gmail.com>
Committed: Wed Aug 23 19:07:28 2017 +0200

----------------------------------------------------------------------
 integration/java/nexmark/pom.xml                |  2 +-
 .../integration/nexmark/NexmarkLauncher.java    |  4 +-
 .../beam/integration/nexmark/NexmarkUtils.java  | 43 +++++++++----------
 .../beam/integration/nexmark/model/Auction.java | 45 ++++++++++----------
 .../integration/nexmark/model/AuctionBid.java   | 13 +++---
 .../integration/nexmark/model/AuctionCount.java | 14 +++---
 .../integration/nexmark/model/AuctionPrice.java | 13 +++---
 .../beam/integration/nexmark/model/Bid.java     | 25 ++++++-----
 .../nexmark/model/BidsPerSession.java           | 13 +++---
 .../nexmark/model/CategoryPrice.java            | 18 ++++----
 .../beam/integration/nexmark/model/Done.java    | 10 ++---
 .../beam/integration/nexmark/model/Event.java   | 24 +++++------
 .../nexmark/model/IdNameReserve.java            | 17 ++++----
 .../nexmark/model/NameCityStateId.java          | 22 +++++-----
 .../beam/integration/nexmark/model/Person.java  | 38 ++++++++---------
 .../integration/nexmark/model/SellerPrice.java  | 13 +++---
 .../nexmark/queries/WinningBids.java            | 16 +++----
 .../integration/nexmark/sources/Generator.java  | 19 +++------
 integration/java/pom.xml                        |  2 +-
 integration/pom.xml                             |  2 +-
 20 files changed, 163 insertions(+), 190 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/beam/blob/3d5c3d00/integration/java/nexmark/pom.xml
----------------------------------------------------------------------
diff --git a/integration/java/nexmark/pom.xml b/integration/java/nexmark/pom.xml
index 35fe0f3..86b88bd 100644
--- a/integration/java/nexmark/pom.xml
+++ b/integration/java/nexmark/pom.xml
@@ -23,7 +23,7 @@
   <parent>
     <groupId>org.apache.beam</groupId>
     <artifactId>beam-integration-java-parent</artifactId>
-    <version>0.7.0-SNAPSHOT</version>
+    <version>2.1.0-SNAPSHOT</version>
     <relativePath>../pom.xml</relativePath>
   </parent>
 

http://git-wip-us.apache.org/repos/asf/beam/blob/3d5c3d00/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkLauncher.java
----------------------------------------------------------------------
diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkLauncher.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkLauncher.java
index ea4ff58..db53191 100644
--- a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkLauncher.java
+++ b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkLauncher.java
@@ -739,7 +739,7 @@ public class NexmarkLauncher<OptionT extends NexmarkOptions> {
     NexmarkUtils.console("Reading events from Pubsub %s", shortSubscription);
 
     PubsubIO.Read<PubsubMessage> io =
-        PubsubIO.readPubsubMessagesWithAttributes().fromSubscription(shortSubscription)
+        PubsubIO.readMessagesWithAttributes().fromSubscription(shortSubscription)
             .withIdAttribute(NexmarkUtils.PUBSUB_ID);
     if (!configuration.usePubsubPublishTime) {
       io = io.withTimestampAttribute(NexmarkUtils.PUBSUB_TIMESTAMP);
@@ -784,7 +784,7 @@ public class NexmarkLauncher<OptionT extends NexmarkOptions> {
     NexmarkUtils.console("Writing events to Pubsub %s", shortTopic);
 
     PubsubIO.Write<PubsubMessage> io =
-        PubsubIO.writePubsubMessages().to(shortTopic)
+        PubsubIO.writeMessages().to(shortTopic)
             .withIdAttribute(NexmarkUtils.PUBSUB_ID);
     if (!configuration.usePubsubPublishTime) {
       io = io.withTimestampAttribute(NexmarkUtils.PUBSUB_TIMESTAMP);

http://git-wip-us.apache.org/repos/asf/beam/blob/3d5c3d00/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkUtils.java
----------------------------------------------------------------------
diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkUtils.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkUtils.java
index ea851af..7707429 100644
--- a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkUtils.java
+++ b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkUtils.java
@@ -351,25 +351,25 @@ public class NexmarkUtils {
     CoderRegistry registry = p.getCoderRegistry();
     switch (coderStrategy) {
       case HAND:
-        registry.registerCoder(Auction.class, Auction.CODER);
-        registry.registerCoder(AuctionBid.class, AuctionBid.CODER);
-        registry.registerCoder(AuctionCount.class, AuctionCount.CODER);
-        registry.registerCoder(AuctionPrice.class, AuctionPrice.CODER);
-        registry.registerCoder(Bid.class, Bid.CODER);
-        registry.registerCoder(CategoryPrice.class, CategoryPrice.CODER);
-        registry.registerCoder(Event.class, Event.CODER);
-        registry.registerCoder(IdNameReserve.class, IdNameReserve.CODER);
-        registry.registerCoder(NameCityStateId.class, NameCityStateId.CODER);
-        registry.registerCoder(Person.class, Person.CODER);
-        registry.registerCoder(SellerPrice.class, SellerPrice.CODER);
-        registry.registerCoder(Done.class, Done.CODER);
-        registry.registerCoder(BidsPerSession.class, BidsPerSession.CODER);
+        registry.registerCoderForClass(Auction.class, Auction.CODER);
+        registry.registerCoderForClass(AuctionBid.class, AuctionBid.CODER);
+        registry.registerCoderForClass(AuctionCount.class, AuctionCount.CODER);
+        registry.registerCoderForClass(AuctionPrice.class, AuctionPrice.CODER);
+        registry.registerCoderForClass(Bid.class, Bid.CODER);
+        registry.registerCoderForClass(CategoryPrice.class, CategoryPrice.CODER);
+        registry.registerCoderForClass(Event.class, Event.CODER);
+        registry.registerCoderForClass(IdNameReserve.class, IdNameReserve.CODER);
+        registry.registerCoderForClass(NameCityStateId.class, NameCityStateId.CODER);
+        registry.registerCoderForClass(Person.class, Person.CODER);
+        registry.registerCoderForClass(SellerPrice.class, SellerPrice.CODER);
+        registry.registerCoderForClass(Done.class, Done.CODER);
+        registry.registerCoderForClass(BidsPerSession.class, BidsPerSession.CODER);
         break;
       case AVRO:
-        registry.setFallbackCoderProvider(AvroCoder.PROVIDER);
+        registry.registerCoderProvider(AvroCoder.getCoderProvider());
         break;
       case JAVA:
-        registry.setFallbackCoderProvider(SerializableCoder.PROVIDER);
+        registry.registerCoderProvider(SerializableCoder.getCoderProvider());
         break;
     }
   }
@@ -621,22 +621,17 @@ public class NexmarkUtils {
     }
 
     @Override
-    public void encode(KnownSize value, OutputStream outStream, Context context)
+    public void encode(KnownSize value, OutputStream outStream)
         throws CoderException, IOException {
       @SuppressWarnings("unchecked")
       T typedValue = (T) value;
-      trueCoder.encode(typedValue, outStream, context);
+      trueCoder.encode(typedValue, outStream);
     }
 
     @Override
-    public KnownSize decode(InputStream inStream, Context context)
+    public KnownSize decode(InputStream inStream)
         throws CoderException, IOException {
-      return trueCoder.decode(inStream, context);
-    }
-
-    @Override
-    public List<? extends Coder<?>> getComponents() {
-      return ImmutableList.of(trueCoder);
+      return trueCoder.decode(inStream);
     }
   }
 

http://git-wip-us.apache.org/repos/asf/beam/blob/3d5c3d00/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/model/Auction.java
----------------------------------------------------------------------
diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/model/Auction.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/model/Auction.java
index 5c018dc..9f5d7c0 100644
--- a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/model/Auction.java
+++ b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/model/Auction.java
@@ -39,35 +39,34 @@ public class Auction implements KnownSize, Serializable {
 
   public static final Coder<Auction> CODER = new CustomCoder<Auction>() {
     @Override
-    public void encode(Auction value, OutputStream outStream,
-        Coder.Context context)
+    public void encode(Auction value, OutputStream outStream)
         throws CoderException, IOException {
-      LONG_CODER.encode(value.id, outStream, Context.NESTED);
-      STRING_CODER.encode(value.itemName, outStream, Context.NESTED);
-      STRING_CODER.encode(value.description, outStream, Context.NESTED);
-      LONG_CODER.encode(value.initialBid, outStream, Context.NESTED);
-      LONG_CODER.encode(value.reserve, outStream, Context.NESTED);
-      LONG_CODER.encode(value.dateTime, outStream, Context.NESTED);
-      LONG_CODER.encode(value.expires, outStream, Context.NESTED);
-      LONG_CODER.encode(value.seller, outStream, Context.NESTED);
-      LONG_CODER.encode(value.category, outStream, Context.NESTED);
-      STRING_CODER.encode(value.extra, outStream, Context.NESTED);
+      LONG_CODER.encode(value.id, outStream);
+      STRING_CODER.encode(value.itemName, outStream);
+      STRING_CODER.encode(value.description, outStream);
+      LONG_CODER.encode(value.initialBid, outStream);
+      LONG_CODER.encode(value.reserve, outStream);
+      LONG_CODER.encode(value.dateTime, outStream);
+      LONG_CODER.encode(value.expires, outStream);
+      LONG_CODER.encode(value.seller, outStream);
+      LONG_CODER.encode(value.category, outStream);
+      STRING_CODER.encode(value.extra, outStream);
     }
 
     @Override
     public Auction decode(
-        InputStream inStream, Coder.Context context)
+        InputStream inStream)
         throws CoderException, IOException {
-      long id = LONG_CODER.decode(inStream, Context.NESTED);
-      String itemName = STRING_CODER.decode(inStream, Context.NESTED);
-      String description = STRING_CODER.decode(inStream, Context.NESTED);
-      long initialBid = LONG_CODER.decode(inStream, Context.NESTED);
-      long reserve = LONG_CODER.decode(inStream, Context.NESTED);
-      long dateTime = LONG_CODER.decode(inStream, Context.NESTED);
-      long expires = LONG_CODER.decode(inStream, Context.NESTED);
-      long seller = LONG_CODER.decode(inStream, Context.NESTED);
-      long category = LONG_CODER.decode(inStream, Context.NESTED);
-      String extra = STRING_CODER.decode(inStream, Context.NESTED);
+      long id = LONG_CODER.decode(inStream);
+      String itemName = STRING_CODER.decode(inStream);
+      String description = STRING_CODER.decode(inStream);
+      long initialBid = LONG_CODER.decode(inStream);
+      long reserve = LONG_CODER.decode(inStream);
+      long dateTime = LONG_CODER.decode(inStream);
+      long expires = LONG_CODER.decode(inStream);
+      long seller = LONG_CODER.decode(inStream);
+      long category = LONG_CODER.decode(inStream);
+      String extra = STRING_CODER.decode(inStream);
       return new Auction(
           id, itemName, description, initialBid, reserve, dateTime, expires, seller, category,
           extra);

http://git-wip-us.apache.org/repos/asf/beam/blob/3d5c3d00/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/model/AuctionBid.java
----------------------------------------------------------------------
diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/model/AuctionBid.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/model/AuctionBid.java
index b1d9ec2..b9d79db 100644
--- a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/model/AuctionBid.java
+++ b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/model/AuctionBid.java
@@ -34,19 +34,18 @@ import org.apache.beam.sdk.coders.CustomCoder;
 public class AuctionBid implements KnownSize, Serializable {
   public static final Coder<AuctionBid> CODER = new CustomCoder<AuctionBid>() {
     @Override
-    public void encode(AuctionBid value, OutputStream outStream,
-        Coder.Context context)
+    public void encode(AuctionBid value, OutputStream outStream)
         throws CoderException, IOException {
-      Auction.CODER.encode(value.auction, outStream, Context.NESTED);
-      Bid.CODER.encode(value.bid, outStream, Context.NESTED);
+      Auction.CODER.encode(value.auction, outStream);
+      Bid.CODER.encode(value.bid, outStream);
     }
 
     @Override
     public AuctionBid decode(
-        InputStream inStream, Coder.Context context)
+        InputStream inStream)
         throws CoderException, IOException {
-      Auction auction = Auction.CODER.decode(inStream, Context.NESTED);
-      Bid bid = Bid.CODER.decode(inStream, Context.NESTED);
+      Auction auction = Auction.CODER.decode(inStream);
+      Bid bid = Bid.CODER.decode(inStream);
       return new AuctionBid(auction, bid);
     }
   };

http://git-wip-us.apache.org/repos/asf/beam/blob/3d5c3d00/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/model/AuctionCount.java
----------------------------------------------------------------------
diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/model/AuctionCount.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/model/AuctionCount.java
index c83a455..0e643ff 100644
--- a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/model/AuctionCount.java
+++ b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/model/AuctionCount.java
@@ -37,19 +37,17 @@ public class AuctionCount implements KnownSize, Serializable {
 
   public static final Coder<AuctionCount> CODER = new CustomCoder<AuctionCount>() {
     @Override
-    public void encode(AuctionCount value, OutputStream outStream,
-        Coder.Context context)
+    public void encode(AuctionCount value, OutputStream outStream)
         throws CoderException, IOException {
-      LONG_CODER.encode(value.auction, outStream, Context.NESTED);
-      LONG_CODER.encode(value.count, outStream, Context.NESTED);
+      LONG_CODER.encode(value.auction, outStream);
+      LONG_CODER.encode(value.count, outStream);
     }
 
     @Override
-    public AuctionCount decode(
-        InputStream inStream, Coder.Context context)
+    public AuctionCount decode(InputStream inStream)
         throws CoderException, IOException {
-      long auction = LONG_CODER.decode(inStream, Context.NESTED);
-      long count = LONG_CODER.decode(inStream, Context.NESTED);
+      long auction = LONG_CODER.decode(inStream);
+      long count = LONG_CODER.decode(inStream);
       return new AuctionCount(auction, count);
     }
   };

http://git-wip-us.apache.org/repos/asf/beam/blob/3d5c3d00/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/model/AuctionPrice.java
----------------------------------------------------------------------
diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/model/AuctionPrice.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/model/AuctionPrice.java
index 43d0b27..7d51a21 100644
--- a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/model/AuctionPrice.java
+++ b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/model/AuctionPrice.java
@@ -37,19 +37,18 @@ public class AuctionPrice implements KnownSize, Serializable {
 
   public static final Coder<AuctionPrice> CODER = new CustomCoder<AuctionPrice>() {
     @Override
-    public void encode(AuctionPrice value, OutputStream outStream,
-        Coder.Context context)
+    public void encode(AuctionPrice value, OutputStream outStream)
         throws CoderException, IOException {
-      LONG_CODER.encode(value.auction, outStream, Context.NESTED);
-      LONG_CODER.encode(value.price, outStream, Context.NESTED);
+      LONG_CODER.encode(value.auction, outStream);
+      LONG_CODER.encode(value.price, outStream);
     }
 
     @Override
     public AuctionPrice decode(
-        InputStream inStream, Coder.Context context)
+        InputStream inStream)
         throws CoderException, IOException {
-      long auction = LONG_CODER.decode(inStream, Context.NESTED);
-      long price = LONG_CODER.decode(inStream, Context.NESTED);
+      long auction = LONG_CODER.decode(inStream);
+      long price = LONG_CODER.decode(inStream);
       return new AuctionPrice(auction, price);
     }
   };

http://git-wip-us.apache.org/repos/asf/beam/blob/3d5c3d00/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/model/Bid.java
----------------------------------------------------------------------
diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/model/Bid.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/model/Bid.java
index faeb928..4fa9ea0 100644
--- a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/model/Bid.java
+++ b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/model/Bid.java
@@ -40,25 +40,24 @@ public class Bid implements KnownSize, Serializable {
 
   public static final Coder<Bid> CODER = new CustomCoder<Bid>() {
     @Override
-    public void encode(Bid value, OutputStream outStream,
-        Coder.Context context)
+    public void encode(Bid value, OutputStream outStream)
         throws CoderException, IOException {
-      LONG_CODER.encode(value.auction, outStream, Context.NESTED);
-      LONG_CODER.encode(value.bidder, outStream, Context.NESTED);
-      LONG_CODER.encode(value.price, outStream, Context.NESTED);
-      LONG_CODER.encode(value.dateTime, outStream, Context.NESTED);
-      STRING_CODER.encode(value.extra, outStream, Context.NESTED);
+      LONG_CODER.encode(value.auction, outStream);
+      LONG_CODER.encode(value.bidder, outStream);
+      LONG_CODER.encode(value.price, outStream);
+      LONG_CODER.encode(value.dateTime, outStream);
+      STRING_CODER.encode(value.extra, outStream);
     }
 
     @Override
     public Bid decode(
-        InputStream inStream, Coder.Context context)
+        InputStream inStream)
         throws CoderException, IOException {
-      long auction = LONG_CODER.decode(inStream, Context.NESTED);
-      long bidder = LONG_CODER.decode(inStream, Context.NESTED);
-      long price = LONG_CODER.decode(inStream, Context.NESTED);
-      long dateTime = LONG_CODER.decode(inStream, Context.NESTED);
-      String extra = STRING_CODER.decode(inStream, Context.NESTED);
+      long auction = LONG_CODER.decode(inStream);
+      long bidder = LONG_CODER.decode(inStream);
+      long price = LONG_CODER.decode(inStream);
+      long dateTime = LONG_CODER.decode(inStream);
+      String extra = STRING_CODER.decode(inStream);
       return new Bid(auction, bidder, price, dateTime, extra);
     }
 

http://git-wip-us.apache.org/repos/asf/beam/blob/3d5c3d00/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/model/BidsPerSession.java
----------------------------------------------------------------------
diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/model/BidsPerSession.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/model/BidsPerSession.java
index 6dddf34..3211456 100644
--- a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/model/BidsPerSession.java
+++ b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/model/BidsPerSession.java
@@ -37,19 +37,18 @@ public class BidsPerSession implements KnownSize, Serializable {
 
   public static final Coder<BidsPerSession> CODER = new CustomCoder<BidsPerSession>() {
     @Override
-    public void encode(BidsPerSession value, OutputStream outStream,
-        Coder.Context context)
+    public void encode(BidsPerSession value, OutputStream outStream)
         throws CoderException, IOException {
-      LONG_CODER.encode(value.personId, outStream, Context.NESTED);
-      LONG_CODER.encode(value.bidsPerSession, outStream, Context.NESTED);
+      LONG_CODER.encode(value.personId, outStream);
+      LONG_CODER.encode(value.bidsPerSession, outStream);
     }
 
     @Override
     public BidsPerSession decode(
-        InputStream inStream, Coder.Context context)
+        InputStream inStream)
         throws CoderException, IOException {
-      long personId = LONG_CODER.decode(inStream, Context.NESTED);
-      long bidsPerSession = LONG_CODER.decode(inStream, Context.NESTED);
+      long personId = LONG_CODER.decode(inStream);
+      long bidsPerSession = LONG_CODER.decode(inStream);
       return new BidsPerSession(personId, bidsPerSession);
     }
     @Override public void verifyDeterministic() throws NonDeterministicException {}

http://git-wip-us.apache.org/repos/asf/beam/blob/3d5c3d00/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/model/CategoryPrice.java
----------------------------------------------------------------------
diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/model/CategoryPrice.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/model/CategoryPrice.java
index ccb2bc7..2678198 100644
--- a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/model/CategoryPrice.java
+++ b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/model/CategoryPrice.java
@@ -39,21 +39,19 @@ public class CategoryPrice implements KnownSize, Serializable {
 
   public static final Coder<CategoryPrice> CODER = new CustomCoder<CategoryPrice>() {
     @Override
-    public void encode(CategoryPrice value, OutputStream outStream,
-        Coder.Context context)
+    public void encode(CategoryPrice value, OutputStream outStream)
         throws CoderException, IOException {
-      LONG_CODER.encode(value.category, outStream, Context.NESTED);
-      LONG_CODER.encode(value.price, outStream, Context.NESTED);
-      INT_CODER.encode(value.isLast ? 1 : 0, outStream, Context.NESTED);
+      LONG_CODER.encode(value.category, outStream);
+      LONG_CODER.encode(value.price, outStream);
+      INT_CODER.encode(value.isLast ? 1 : 0, outStream);
     }
 
     @Override
-    public CategoryPrice decode(
-        InputStream inStream, Coder.Context context)
+    public CategoryPrice decode(InputStream inStream)
         throws CoderException, IOException {
-      long category = LONG_CODER.decode(inStream, Context.NESTED);
-      long price = LONG_CODER.decode(inStream, Context.NESTED);
-      boolean isLast = INT_CODER.decode(inStream, context) != 0;
+      long category = LONG_CODER.decode(inStream);
+      long price = LONG_CODER.decode(inStream);
+      boolean isLast = INT_CODER.decode(inStream) != 0;
       return new CategoryPrice(category, price, isLast);
     }
     @Override public void verifyDeterministic() throws NonDeterministicException {}

http://git-wip-us.apache.org/repos/asf/beam/blob/3d5c3d00/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/model/Done.java
----------------------------------------------------------------------
diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/model/Done.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/model/Done.java
index 0c14e8f..b0a88d4 100644
--- a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/model/Done.java
+++ b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/model/Done.java
@@ -37,17 +37,15 @@ public class Done implements KnownSize, Serializable {
 
   public static final Coder<Done> CODER = new CustomCoder<Done>() {
     @Override
-    public void encode(Done value, OutputStream outStream,
-        Coder.Context context)
+    public void encode(Done value, OutputStream outStream)
         throws CoderException, IOException {
-      STRING_CODER.encode(value.message, outStream, Context.NESTED);
+      STRING_CODER.encode(value.message, outStream);
     }
 
     @Override
-    public Done decode(
-        InputStream inStream, Coder.Context context)
+    public Done decode(InputStream inStream)
         throws CoderException, IOException {
-      String message = STRING_CODER.decode(inStream, Context.NESTED);
+      String message = STRING_CODER.decode(inStream);
       return new Done(message);
     }
     @Override public void verifyDeterministic() throws NonDeterministicException {}

http://git-wip-us.apache.org/repos/asf/beam/blob/3d5c3d00/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/model/Event.java
----------------------------------------------------------------------
diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/model/Event.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/model/Event.java
index 1f1f096..d813833 100644
--- a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/model/Event.java
+++ b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/model/Event.java
@@ -36,17 +36,17 @@ public class Event implements KnownSize, Serializable {
 
   public static final Coder<Event> CODER = new CustomCoder<Event>() {
     @Override
-    public void encode(Event value, OutputStream outStream, Coder.Context context)
+    public void encode(Event value, OutputStream outStream)
         throws CoderException, IOException {
       if (value.newPerson != null) {
-        INT_CODER.encode(0, outStream, Context.NESTED);
-        Person.CODER.encode(value.newPerson, outStream, Context.NESTED);
+        INT_CODER.encode(0, outStream);
+        Person.CODER.encode(value.newPerson, outStream);
       } else if (value.newAuction != null) {
-        INT_CODER.encode(1, outStream, Context.NESTED);
-        Auction.CODER.encode(value.newAuction, outStream, Context.NESTED);
+        INT_CODER.encode(1, outStream);
+        Auction.CODER.encode(value.newAuction, outStream);
       } else if (value.bid != null) {
-        INT_CODER.encode(2, outStream, Context.NESTED);
-        Bid.CODER.encode(value.bid, outStream, Context.NESTED);
+        INT_CODER.encode(2, outStream);
+        Bid.CODER.encode(value.bid, outStream);
       } else {
         throw new RuntimeException("invalid event");
       }
@@ -54,17 +54,17 @@ public class Event implements KnownSize, Serializable {
 
     @Override
     public Event decode(
-        InputStream inStream, Coder.Context context)
+        InputStream inStream)
         throws CoderException, IOException {
-      int tag = INT_CODER.decode(inStream, context);
+      int tag = INT_CODER.decode(inStream);
       if (tag == 0) {
-        Person person = Person.CODER.decode(inStream, Context.NESTED);
+        Person person = Person.CODER.decode(inStream);
         return new Event(person);
       } else if (tag == 1) {
-        Auction auction = Auction.CODER.decode(inStream, Context.NESTED);
+        Auction auction = Auction.CODER.decode(inStream);
         return new Event(auction);
       } else if (tag == 2) {
-        Bid bid = Bid.CODER.decode(inStream, Context.NESTED);
+        Bid bid = Bid.CODER.decode(inStream);
         return new Event(bid);
       } else {
         throw new RuntimeException("invalid event encoding");

http://git-wip-us.apache.org/repos/asf/beam/blob/3d5c3d00/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/model/IdNameReserve.java
----------------------------------------------------------------------
diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/model/IdNameReserve.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/model/IdNameReserve.java
index 17b8c4a..8cade4e 100644
--- a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/model/IdNameReserve.java
+++ b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/model/IdNameReserve.java
@@ -39,21 +39,20 @@ public class IdNameReserve implements KnownSize, Serializable {
 
   public static final Coder<IdNameReserve> CODER = new CustomCoder<IdNameReserve>() {
     @Override
-    public void encode(IdNameReserve value, OutputStream outStream,
-        Coder.Context context)
+    public void encode(IdNameReserve value, OutputStream outStream)
         throws CoderException, IOException {
-      LONG_CODER.encode(value.id, outStream, Context.NESTED);
-      STRING_CODER.encode(value.name, outStream, Context.NESTED);
-      LONG_CODER.encode(value.reserve, outStream, Context.NESTED);
+      LONG_CODER.encode(value.id, outStream);
+      STRING_CODER.encode(value.name, outStream);
+      LONG_CODER.encode(value.reserve, outStream);
     }
 
     @Override
     public IdNameReserve decode(
-        InputStream inStream, Coder.Context context)
+        InputStream inStream)
         throws CoderException, IOException {
-      long id = LONG_CODER.decode(inStream, Context.NESTED);
-      String name = STRING_CODER.decode(inStream, Context.NESTED);
-      long reserve = LONG_CODER.decode(inStream, Context.NESTED);
+      long id = LONG_CODER.decode(inStream);
+      String name = STRING_CODER.decode(inStream);
+      long reserve = LONG_CODER.decode(inStream);
       return new IdNameReserve(id, name, reserve);
     }
     @Override public void verifyDeterministic() throws NonDeterministicException {}

http://git-wip-us.apache.org/repos/asf/beam/blob/3d5c3d00/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/model/NameCityStateId.java
----------------------------------------------------------------------
diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/model/NameCityStateId.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/model/NameCityStateId.java
index 28f25cd..37bd3c6 100644
--- a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/model/NameCityStateId.java
+++ b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/model/NameCityStateId.java
@@ -39,23 +39,21 @@ public class NameCityStateId implements KnownSize, Serializable {
 
   public static final Coder<NameCityStateId> CODER = new CustomCoder<NameCityStateId>() {
     @Override
-    public void encode(NameCityStateId value, OutputStream outStream,
-        Coder.Context context)
+    public void encode(NameCityStateId value, OutputStream outStream)
         throws CoderException, IOException {
-      STRING_CODER.encode(value.name, outStream, Context.NESTED);
-      STRING_CODER.encode(value.city, outStream, Context.NESTED);
-      STRING_CODER.encode(value.state, outStream, Context.NESTED);
-      LONG_CODER.encode(value.id, outStream, Context.NESTED);
+      STRING_CODER.encode(value.name, outStream);
+      STRING_CODER.encode(value.city, outStream);
+      STRING_CODER.encode(value.state, outStream);
+      LONG_CODER.encode(value.id, outStream);
     }
 
     @Override
-    public NameCityStateId decode(
-        InputStream inStream, Coder.Context context)
+    public NameCityStateId decode(InputStream inStream)
         throws CoderException, IOException {
-      String name = STRING_CODER.decode(inStream, Context.NESTED);
-      String city = STRING_CODER.decode(inStream, Context.NESTED);
-      String state = STRING_CODER.decode(inStream, Context.NESTED);
-      long id = LONG_CODER.decode(inStream, Context.NESTED);
+      String name = STRING_CODER.decode(inStream);
+      String city = STRING_CODER.decode(inStream);
+      String state = STRING_CODER.decode(inStream);
+      long id = LONG_CODER.decode(inStream);
       return new NameCityStateId(name, city, state, id);
     }
     @Override public void verifyDeterministic() throws NonDeterministicException {}

http://git-wip-us.apache.org/repos/asf/beam/blob/3d5c3d00/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/model/Person.java
----------------------------------------------------------------------
diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/model/Person.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/model/Person.java
index c690fd4..bde587d 100644
--- a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/model/Person.java
+++ b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/model/Person.java
@@ -38,31 +38,29 @@ public class Person implements KnownSize, Serializable {
   private static final Coder<String> STRING_CODER = StringUtf8Coder.of();
   public static final Coder<Person> CODER = new CustomCoder<Person>() {
     @Override
-    public void encode(Person value, OutputStream outStream,
-        Coder.Context context)
+    public void encode(Person value, OutputStream outStream)
         throws CoderException, IOException {
-      LONG_CODER.encode(value.id, outStream, Context.NESTED);
-      STRING_CODER.encode(value.name, outStream, Context.NESTED);
-      STRING_CODER.encode(value.emailAddress, outStream, Context.NESTED);
-      STRING_CODER.encode(value.creditCard, outStream, Context.NESTED);
-      STRING_CODER.encode(value.city, outStream, Context.NESTED);
-      STRING_CODER.encode(value.state, outStream, Context.NESTED);
-      LONG_CODER.encode(value.dateTime, outStream, Context.NESTED);
-      STRING_CODER.encode(value.extra, outStream, Context.NESTED);
+      LONG_CODER.encode(value.id, outStream);
+      STRING_CODER.encode(value.name, outStream);
+      STRING_CODER.encode(value.emailAddress, outStream);
+      STRING_CODER.encode(value.creditCard, outStream);
+      STRING_CODER.encode(value.city, outStream);
+      STRING_CODER.encode(value.state, outStream);
+      LONG_CODER.encode(value.dateTime, outStream);
+      STRING_CODER.encode(value.extra, outStream);
     }
 
     @Override
-    public Person decode(
-        InputStream inStream, Coder.Context context)
+    public Person decode(InputStream inStream)
         throws CoderException, IOException {
-      long id = LONG_CODER.decode(inStream, Context.NESTED);
-      String name = STRING_CODER.decode(inStream, Context.NESTED);
-      String emailAddress = STRING_CODER.decode(inStream, Context.NESTED);
-      String creditCard = STRING_CODER.decode(inStream, Context.NESTED);
-      String city = STRING_CODER.decode(inStream, Context.NESTED);
-      String state = STRING_CODER.decode(inStream, Context.NESTED);
-      long dateTime = LONG_CODER.decode(inStream, Context.NESTED);
-      String extra = STRING_CODER.decode(inStream, Context.NESTED);
+      long id = LONG_CODER.decode(inStream);
+      String name = STRING_CODER.decode(inStream);
+      String emailAddress = STRING_CODER.decode(inStream);
+      String creditCard = STRING_CODER.decode(inStream);
+      String city = STRING_CODER.decode(inStream);
+      String state = STRING_CODER.decode(inStream);
+      long dateTime = LONG_CODER.decode(inStream);
+      String extra = STRING_CODER.decode(inStream);
       return new Person(id, name, emailAddress, creditCard, city, state, dateTime, extra);
     }
     @Override public void verifyDeterministic() throws NonDeterministicException {}

http://git-wip-us.apache.org/repos/asf/beam/blob/3d5c3d00/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/model/SellerPrice.java
----------------------------------------------------------------------
diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/model/SellerPrice.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/model/SellerPrice.java
index 52ff540..61537f6 100644
--- a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/model/SellerPrice.java
+++ b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/model/SellerPrice.java
@@ -37,19 +37,18 @@ public class SellerPrice implements KnownSize, Serializable {
 
   public static final Coder<SellerPrice> CODER = new CustomCoder<SellerPrice>() {
     @Override
-    public void encode(SellerPrice value, OutputStream outStream,
-        Coder.Context context)
+    public void encode(SellerPrice value, OutputStream outStream)
         throws CoderException, IOException {
-      LONG_CODER.encode(value.seller, outStream, Context.NESTED);
-      LONG_CODER.encode(value.price, outStream, Context.NESTED);
+      LONG_CODER.encode(value.seller, outStream);
+      LONG_CODER.encode(value.price, outStream);
     }
 
     @Override
     public SellerPrice decode(
-        InputStream inStream, Coder.Context context)
+        InputStream inStream)
         throws CoderException, IOException {
-      long seller = LONG_CODER.decode(inStream, Context.NESTED);
-      long price = LONG_CODER.decode(inStream, Context.NESTED);
+      long seller = LONG_CODER.decode(inStream);
+      long price = LONG_CODER.decode(inStream);
       return new SellerPrice(seller, price);
     }
     @Override public void verifyDeterministic() throws NonDeterministicException {}

http://git-wip-us.apache.org/repos/asf/beam/blob/3d5c3d00/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/WinningBids.java
----------------------------------------------------------------------
diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/WinningBids.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/WinningBids.java
index 52891a7..bd6c2ed 100644
--- a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/WinningBids.java
+++ b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/WinningBids.java
@@ -156,19 +156,19 @@ public class WinningBids extends PTransform<PCollection<Event>, PCollection<Auct
     }
 
     @Override
-    public void encode(AuctionOrBidWindow window, OutputStream outStream, Coder.Context context)
+    public void encode(AuctionOrBidWindow window, OutputStream outStream)
         throws IOException, CoderException {
-      SUPER_CODER.encode(window, outStream, Coder.Context.NESTED);
-      ID_CODER.encode(window.auction, outStream, Coder.Context.NESTED);
-      INT_CODER.encode(window.isAuctionWindow ? 1 : 0, outStream, Coder.Context.NESTED);
+      SUPER_CODER.encode(window, outStream);
+      ID_CODER.encode(window.auction, outStream);
+      INT_CODER.encode(window.isAuctionWindow ? 1 : 0, outStream);
     }
 
     @Override
-    public AuctionOrBidWindow decode(InputStream inStream, Coder.Context context)
+    public AuctionOrBidWindow decode(InputStream inStream)
         throws IOException, CoderException {
-      IntervalWindow superWindow = SUPER_CODER.decode(inStream, Coder.Context.NESTED);
-      long auction = ID_CODER.decode(inStream, Coder.Context.NESTED);
-      boolean isAuctionWindow = INT_CODER.decode(inStream, Context.NESTED) != 0;
+      IntervalWindow superWindow = SUPER_CODER.decode(inStream);
+      long auction = ID_CODER.decode(inStream);
+      boolean isAuctionWindow = INT_CODER.decode(inStream) != 0;
       return new AuctionOrBidWindow(
           superWindow.start(), superWindow.end(), auction, isAuctionWindow);
     }

http://git-wip-us.apache.org/repos/asf/beam/blob/3d5c3d00/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/sources/Generator.java
----------------------------------------------------------------------
diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/sources/Generator.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/sources/Generator.java
index 2a2732b..4f548cd 100644
--- a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/sources/Generator.java
+++ b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/sources/Generator.java
@@ -102,22 +102,17 @@ public class Generator implements Iterator<TimestampedValue<Event>>, Serializabl
     /** Coder for this class. */
     public static final Coder<Checkpoint> CODER_INSTANCE =
         new CustomCoder<Checkpoint>() {
-          @Override
-          public void encode(
-              Checkpoint value,
-              OutputStream outStream,
-              Coder.Context context)
-              throws CoderException, IOException {
-            LONG_CODER.encode(value.numEvents, outStream, Context.NESTED);
-            LONG_CODER.encode(value.wallclockBaseTime, outStream, Context.NESTED);
+          @Override public void encode(Checkpoint value, OutputStream outStream)
+          throws CoderException, IOException {
+            LONG_CODER.encode(value.numEvents, outStream);
+            LONG_CODER.encode(value.wallclockBaseTime, outStream);
           }
 
           @Override
-          public Checkpoint decode(
-              InputStream inStream, Coder.Context context)
+          public Checkpoint decode(InputStream inStream)
               throws CoderException, IOException {
-            long numEvents = LONG_CODER.decode(inStream, Context.NESTED);
-            long wallclockBaseTime = LONG_CODER.decode(inStream, Context.NESTED);
+            long numEvents = LONG_CODER.decode(inStream);
+            long wallclockBaseTime = LONG_CODER.decode(inStream);
             return new Checkpoint(numEvents, wallclockBaseTime);
           }
           @Override public void verifyDeterministic() throws NonDeterministicException {}

http://git-wip-us.apache.org/repos/asf/beam/blob/3d5c3d00/integration/java/pom.xml
----------------------------------------------------------------------
diff --git a/integration/java/pom.xml b/integration/java/pom.xml
index dcad4c3..b0c3853 100644
--- a/integration/java/pom.xml
+++ b/integration/java/pom.xml
@@ -22,7 +22,7 @@
   <parent>
     <groupId>org.apache.beam</groupId>
     <artifactId>beam-integration-parent</artifactId>
-    <version>0.7.0-SNAPSHOT</version>
+    <version>2.1.0-SNAPSHOT</version>
     <relativePath>../pom.xml</relativePath>
   </parent>
 

http://git-wip-us.apache.org/repos/asf/beam/blob/3d5c3d00/integration/pom.xml
----------------------------------------------------------------------
diff --git a/integration/pom.xml b/integration/pom.xml
index 31f293e..4254819 100644
--- a/integration/pom.xml
+++ b/integration/pom.xml
@@ -22,7 +22,7 @@
   <parent>
     <groupId>org.apache.beam</groupId>
     <artifactId>beam-parent</artifactId>
-    <version>0.7.0-SNAPSHOT</version>
+    <version>2.1.0-SNAPSHOT</version>
     <relativePath>../pom.xml</relativePath>
   </parent>
 


[37/55] [abbrv] beam git commit: Fix query10 log messages

Posted by ie...@apache.org.
Fix query10 log messages

issue #5 and issue #51


Project: http://git-wip-us.apache.org/repos/asf/beam/repo
Commit: http://git-wip-us.apache.org/repos/asf/beam/commit/ee500b28
Tree: http://git-wip-us.apache.org/repos/asf/beam/tree/ee500b28
Diff: http://git-wip-us.apache.org/repos/asf/beam/diff/ee500b28

Branch: refs/heads/master
Commit: ee500b28086f1261101395dc0b7b23f197ba19d9
Parents: 3d5c3d0
Author: Etienne Chauchot <ec...@gmail.com>
Authored: Tue May 30 18:00:00 2017 +0100
Committer: Ismaël Mejía <ie...@gmail.com>
Committed: Wed Aug 23 19:07:28 2017 +0200

----------------------------------------------------------------------
 integration/java/nexmark/pom.xml                |  6 +++
 .../integration/nexmark/queries/Query10.java    | 39 ++++++++------------
 2 files changed, 22 insertions(+), 23 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/beam/blob/ee500b28/integration/java/nexmark/pom.xml
----------------------------------------------------------------------
diff --git a/integration/java/nexmark/pom.xml b/integration/java/nexmark/pom.xml
index 86b88bd..664a410 100644
--- a/integration/java/nexmark/pom.xml
+++ b/integration/java/nexmark/pom.xml
@@ -255,6 +255,12 @@
     </dependency>
 
     <dependency>
+      <groupId>org.slf4j</groupId>
+      <artifactId>slf4j-jdk14</artifactId>
+      <scope>runtime</scope>
+    </dependency>
+
+    <dependency>
       <groupId>com.google.code.findbugs</groupId>
       <artifactId>jsr305</artifactId>
     </dependency>

http://git-wip-us.apache.org/repos/asf/beam/blob/ee500b28/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query10.java
----------------------------------------------------------------------
diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query10.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query10.java
index c868666..378d01e 100644
--- a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query10.java
+++ b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query10.java
@@ -17,9 +17,6 @@
  */
 package org.apache.beam.integration.nexmark.queries;
 
-import static com.google.common.base.Preconditions.checkState;
-
-import com.google.cloud.hadoop.gcsio.GoogleCloudStorageWriteChannel;
 import java.io.IOException;
 import java.io.OutputStream;
 import java.io.Serializable;
@@ -33,9 +30,9 @@ import org.apache.beam.integration.nexmark.model.Done;
 import org.apache.beam.integration.nexmark.model.Event;
 import org.apache.beam.integration.nexmark.model.KnownSize;
 import org.apache.beam.sdk.coders.Coder;
+import org.apache.beam.sdk.extensions.gcp.options.GcsOptions;
 import org.apache.beam.sdk.metrics.Counter;
 import org.apache.beam.sdk.metrics.Metrics;
-import org.apache.beam.sdk.extensions.gcp.options.GcsOptions;
 import org.apache.beam.sdk.transforms.DoFn;
 import org.apache.beam.sdk.transforms.GroupByKey;
 import org.apache.beam.sdk.transforms.ParDo;
@@ -57,7 +54,6 @@ import org.joda.time.Instant;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-
 /**
  * Query "10", 'Log to sharded files' (Not in original suite.)
  *
@@ -132,12 +128,9 @@ public class Query10 extends NexmarkQuery {
    */
   private WritableByteChannel openWritableGcsFile(GcsOptions options, String filename)
       throws IOException {
-    //TODO Decide what to do about this one
-//    WritableByteChannel channel =
-//            GcsIOChannelFactory.fromOptions(options).create(filename, "text/plain");
-//    checkState(channel instanceof GoogleCloudStorageWriteChannel);
-//    ((GoogleCloudStorageWriteChannel) channel).setUploadBufferSize(CHANNEL_BUFFER);
-//    return channel;
+    //TODO
+    // Fix after PR: right now this is a specific Google added use case
+    // Discuss it on ML: shall we keep GCS or use HDFS or use a generic beam filesystem way.
     throw new UnsupportedOperationException("Disabled after removal of GcsIOChannelFactory");
   }
 
@@ -192,7 +185,7 @@ public class Query10 extends NexmarkQuery {
           public void processElement(ProcessContext c) {
             if (c.element().hasAnnotation("LATE")) {
               lateCounter.inc();
-              LOG.error("Observed late: %s", c.element());
+              LOG.info("Observed late: %s", c.element());
             } else {
               onTimeCounter.inc();
             }
@@ -240,11 +233,11 @@ public class Query10 extends NexmarkQuery {
               }
             }
             String shard = c.element().getKey();
-            LOG.error(
+            LOG.info(String.format(
                 "%s with timestamp %s has %d actually late and %d on-time "
                     + "elements in pane %s for window %s",
                 shard, c.timestamp(), numLate, numOnTime, c.pane(),
-                window.maxTimestamp());
+                window.maxTimestamp()));
             if (c.pane().getTiming() == PaneInfo.Timing.LATE) {
               if (numLate == 0) {
                 LOG.error(
@@ -283,11 +276,11 @@ public class Query10 extends NexmarkQuery {
               String shard = c.element().getKey();
               GcsOptions options = c.getPipelineOptions().as(GcsOptions.class);
               OutputFile outputFile = outputFileFor(window, shard, c.pane());
-              LOG.error(
+              LOG.info(String.format(
                   "Writing %s with record timestamp %s, window timestamp %s, pane %s",
-                  shard, c.timestamp(), window.maxTimestamp(), c.pane());
+                  shard, c.timestamp(), window.maxTimestamp(), c.pane()));
               if (outputFile.filename != null) {
-                LOG.error("Beginning write to '%s'", outputFile.filename);
+                LOG.info("Beginning write to '%s'", outputFile.filename);
                 int n = 0;
                 try (OutputStream output =
                          Channels.newOutputStream(openWritableGcsFile(options, outputFile
@@ -296,12 +289,12 @@ public class Query10 extends NexmarkQuery {
                     Event.CODER.encode(event, output, Coder.Context.OUTER);
                     writtenRecordsCounter.inc();
                     if (++n % 10000 == 0) {
-                      LOG.error("So far written %d records to '%s'", n,
+                      LOG.info("So far written %d records to '%s'", n,
                           outputFile.filename);
                     }
                   }
                 }
-                LOG.error("Written all %d records to '%s'", n, outputFile.filename);
+                LOG.info("Written all %d records to '%s'", n, outputFile.filename);
               }
               savedFileCounter.inc();
               c.output(KV.<Void, OutputFile>of(null, outputFile));
@@ -341,23 +334,23 @@ public class Query10 extends NexmarkQuery {
               LOG.error("ERROR! Unexpected ON_TIME pane index: %s", c.pane());
             } else {
               GcsOptions options = c.getPipelineOptions().as(GcsOptions.class);
-              LOG.error(
+              LOG.info(
                   "Index with record timestamp %s, window timestamp %s, pane %s",
                   c.timestamp(), window.maxTimestamp(), c.pane());
 
               @Nullable String filename = indexPathFor(window);
               if (filename != null) {
-                LOG.error("Beginning write to '%s'", filename);
+                LOG.info("Beginning write to '%s'", filename);
                 int n = 0;
                 try (OutputStream output =
                          Channels.newOutputStream(
                              openWritableGcsFile(options, filename))) {
                   for (OutputFile outputFile : c.element().getValue()) {
-                    output.write(outputFile.toString().getBytes());
+                    output.write(outputFile.toString().getBytes("UTF-8"));
                     n++;
                   }
                 }
-                LOG.error("Written all %d lines to '%s'", n, filename);
+                LOG.info("Written all %d lines to '%s'", n, filename);
               }
               c.output(
                   new Done("written for timestamp " + window.maxTimestamp()));


[39/55] [abbrv] beam git commit: Change benchmark workload settings

Posted by ie...@apache.org.
Change benchmark workload settings

Update configuration of events generation to add some variation
Update execution matrix (issue #45)


Project: http://git-wip-us.apache.org/repos/asf/beam/repo
Commit: http://git-wip-us.apache.org/repos/asf/beam/commit/dbd1b155
Tree: http://git-wip-us.apache.org/repos/asf/beam/tree/dbd1b155
Diff: http://git-wip-us.apache.org/repos/asf/beam/diff/dbd1b155

Branch: refs/heads/master
Commit: dbd1b155c32c19ce7a6d0c0f0dffb318c9ccdde7
Parents: 683680b
Author: Etienne Chauchot <ec...@gmail.com>
Authored: Tue May 9 11:48:00 2017 +0200
Committer: Ismaël Mejía <ie...@gmail.com>
Committed: Wed Aug 23 19:07:28 2017 +0200

----------------------------------------------------------------------
 integration/java/nexmark/README.md              | 207 +++++++++++--------
 .../nexmark/NexmarkConfiguration.java           |  10 +-
 2 files changed, 128 insertions(+), 89 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/beam/blob/dbd1b155/integration/java/nexmark/README.md
----------------------------------------------------------------------
diff --git a/integration/java/nexmark/README.md b/integration/java/nexmark/README.md
index a3549f4..a9acd63 100644
--- a/integration/java/nexmark/README.md
+++ b/integration/java/nexmark/README.md
@@ -30,14 +30,14 @@ These are multiple queries over a three entities model representing on online au
  - **Auction** represents an item under auction.
  - **Bid** represents a bid for an item under auction.
 
-The queries exercise many aspects of dataflow model on Beam:
+The queries exercise many aspects of Beam model:
 
 * **Query1**: What are the bid values in Euro's?
   Illustrates a simple map.
 * **Query2**: What are the auctions with particular auction numbers?
   Illustrates a simple filter.
 * **Query3**: Who is selling in particular US states?
-  Illustrates an incremental join (using per-key state) and filter.
+  Illustrates an incremental join (using per-key state and timer) and filter.
 * **Query4**: What is the average selling price for each auction
   category?
   Illustrates complex join (using custom window functions) and
@@ -71,19 +71,17 @@ We have augmented the original queries with five more:
   compared with event time in non-Global windows for all the other
   queries.
 
-The queries can be executed using a 'Driver' for a given backend.
-Currently the supported drivers are:
+We can specify the Beam runner to use with maven profiles, available profiles are:
 
-* **NexmarkApexDriver** for running via the Apex runner.
-* **NexmarkDirectDriver** for running locally on a single machine.
-* **NexmarkGoogleDriver** for running on the Google Cloud Dataflow service.
-  Requires a Google Cloud account.
-* **NexmarkFlinkDriver** for running on a Flink cluster. Requires the
-  cluster to be established and the Nexmark jar to be distributed to
-  each worker.
-* **NexmarkSparkDriver** for running on a Spark cluster.
+* direct-runner
+* spark-runner
+* flink-runner
+* apex-runner
+
+The runner must also be specified like in any other Beam pipeline using
+
+    --runner
 
-Other drivers are straightforward.
 
 Test data is deterministically synthesized on demand. The test
 data may be synthesized in the same pipeline as the query itself,
@@ -97,11 +95,6 @@ The query results may be:
 * Send to BigQuery.
 * Discarded.
 
-Options are provided for measuring progress, measuring overall
-pipeline performance, and comparing that performance against a known
-baseline. However that machinery has only been implemented against
-the Google Cloud Dataflow driver.
-
 # Configuration
 
 ## Common configuration parameters
@@ -119,45 +112,48 @@ Run query N
     --query=N
 
 ## Available Suites
+The suite to run can be chosen using this configuration parameter:
 
-- DEFAULT: Test default configuration with query 0.
-- SMOKE: Run the 12 default configurations.
-- STRESS: Like smoke but for 1m events.
-- FULL_THROTTLE: Like SMOKE but 100m events.
+    --suite=SUITE
 
-        --suite=SMOKE
+Available suites are:
+* DEFAULT: Test default configuration with query 0.
+* SMOKE: Run the 12 default configurations.
+* STRESS: Like smoke but for 1m events.
+* FULL_THROTTLE: Like SMOKE but 100m events.
 
-### Apex specific configuration
+   
 
-    --suite=SMOKE --manageResources=false --monitorJobs=true
+## Apex specific configuration
 
-### Dataflow specific configuration
+    --manageResources=false --monitorJobs=false
 
-    --query=0 --suite=SMOKE --manageResources=false --monitorJobs=true \
+## Dataflow specific configuration
+
+    --manageResources=false --monitorJobs=true \
     --enforceEncodability=false --enforceImmutability=false
     --project=<your project> \
     --zone=<your zone> \
     --workerMachineType=n1-highmem-8 \
-    --stagingLocation=<a gs path for staging>
-
-    --runner=BlockingDataflowRunner \
+    --stagingLocation=<a gs path for staging> \
+    --runner=DataflowRunner \
     --tempLocation=gs://talend-imejia/nexmark/temp/ \
     --stagingLocation=gs://talend-imejia/nexmark/temp/staging/ \
     --filesToStage=target/beam-integration-java-0.7.0-SNAPSHOT.jar
 
-### Direct specific configuration
+## Direct specific configuration
 
-    --suite=SMOKE --manageResources=false --monitorJobs=true \
+    --manageResources=false --monitorJobs=true \
     --enforceEncodability=false --enforceImmutability=false
 
-### Flink specific configuration
+## Flink specific configuration
 
-    --suite=SMOKE --manageResources=false --monitorJobs=true \
-    --flinkMaster=[local] --parallelism=#numcores
+    --manageResources=false --monitorJobs=true \
+    --flinkMaster=local --parallelism=#numcores
 
-### Spark specific configuration
+## Spark specific configuration
 
-    --suite=SMOKE --manageResources=false --monitorJobs=true \
+    --manageResources=false --monitorJobs=true \
     --sparkMaster=local \
     -Dspark.ui.enabled=false -DSPARK_LOCAL_IP=localhost -Dsun.io.serialization.extendedDebugInfo=true
 
@@ -167,39 +163,39 @@ Open issues are tracked [here](https://github.com../../../../../issues):
 
 ## Batch / Synthetic / Local
 
-| Query | Direct                         | Spark                          | Flink                          | Apex                            |
-| ----: | ------------------------------ | ------------------------------ | ------------------------------ | ------------------------------- |
-|     0 | ok                             | ok                             | ok                             | ok                              |
-|     1 | ok                             | ok                             | ok                             | ok                              |
-|     2 | ok                             | ok                             | ok                             | ok                              |
-|     3 | [#7](../../../../../issues/7)  | [#7](../../../../../issues/7)  | [#7](../../../../../issues/7)  | [#7](../../../../../issues/7)   |
-|     4 | ok                             | ok                             | [#2](../../../../../issues/2)  | ok                              |
-|     5 | ok                             | ok                             | ok                             | ok                              |
-|     6 | ok                             | ok                             | [#2](../../../../../issues/2)  | ok                              |
-|     7 | ok                             | ok                             | ok                             | [#24](../../../../../issues/24) |
-|     8 | ok                             | ok                             | ok                             | ok                              |
-|     9 | ok                             | ok                             | [#2](../../../../../issues/2)  | ok                              |
-|    10 | [#5](../../../../../issues/5)  | ok                             | ok                             | ok                              |
-|    11 | ok                             | ok                             | ok                             | ok                              |
-|    12 | ok                             | ok                             | ok                             | ok                              |
+| Query | Direct | Spark                                                        | Flink                                                      | Apex                                                         |
+| ----: | ------ | ------------------------------------------------------------ | ---------------------------------------------------------- | ------------------------------------------------------------ |
+|     0 | ok     | ok                                                           | ok                                                         | ok                                                           |
+|     1 | ok     | ok                                                           | ok                                                         | ok                                                           |
+|     2 | ok     | ok                                                           | ok                                                         | ok                                                           |
+|     3 | ok     | [BEAM-1035](https://issues.apache.org/jira/browse/BEAM-1035) | ok                                                         | [BEAM-1037](https://issues.apache.org/jira/browse/BEAM-1037) |
+|     4 | ok     | ok                                                           | ok                                                         | ok                                                           |
+|     5 | ok     | ok                                                           | ok                                                         | ok                                                           |
+|     6 | ok     | ok                                                           | ok                                                         | ok                                                           |
+|     7 | ok     | ok                                                           | ok                                                         | ok                                                           |
+|     8 | ok     | ok                                                           | ok                                                         | ok                                                           |
+|     9 | ok     | ok                                                           | ok                                                         | ok                                                           |
+|    10 | ok     | ok                                                           | ok                                                         | ok                                                           |
+|    11 | ok     | ok                                                           | ok                                                         | ok                                                           |
+|    12 | ok     | ok                                                           | ok                                                         | ok                                                           |
 
 ## Streaming / Synthetic / Local
 
-| Query | Direct                         | Spark                          | Flink                          | Apex                           |
-| ----: | ------------------------------ | ------------------------------ | ------------------------------ | ------------------------------ |
-|     0 | ok                             |                                |                                | ok                             |
-|     1 | ok                             |                                |                                | ok                             |
-|     2 | ok                             |                                |                                | ok                             |
-|     3 | [#7](../../../../../issues/7)  |                                |                                | [#7](../../../../../issues/7)  |
-|     4 | ok                             |                                |                                | ok                             |
-|     5 | ok                             |                                |                                | ok                             |
-|     6 | ok                             |                                |                                | ok                             |
-|     7 | ok                             |                                |                                | ?                              |
-|     8 | ok                             |                                |                                | ok                             |
-|     9 | ok                             |                                |                                | ok                             |
-|    10 | [#5](../../../../../issues/5)  |                                |                                | ?                              |
-|    11 | ok                             |                                |                                | Ok                             |
-|    12 | ok                             |                                |                                | Ok                             |
+| Query | Direct | Spark                                                        | Flink                                                      | Apex                                                         |
+| ----: | ------ | ------------------------------------------------------------ | ---------------------------------------------------------- | ------------------------------------------------------------ |
+|     0 | ok     | ok                                                           | [BEAM-593](https://issues.apache.org/jira/browse/BEAM-593) | ok                                                           |
+|     1 | ok     | ok                                                           | [BEAM-593](https://issues.apache.org/jira/browse/BEAM-593) | ok                                                           |
+|     2 | ok     | ok                                                           | [BEAM-593](https://issues.apache.org/jira/browse/BEAM-593) | ok                                                           |
+|     3 | ok     | [BEAM-1035](https://issues.apache.org/jira/browse/BEAM-1035) | [BEAM-593](https://issues.apache.org/jira/browse/BEAM-593) | [BEAM-1037](https://issues.apache.org/jira/browse/BEAM-1037) |
+|     4 | ok     | ok                                                           | [BEAM-593](https://issues.apache.org/jira/browse/BEAM-593) | ok                                                           |
+|     5 | ok     | ok                                                           | [BEAM-593](https://issues.apache.org/jira/browse/BEAM-593) | ok                                                           |
+|     6 | ok     | ok                                                           | [BEAM-593](https://issues.apache.org/jira/browse/BEAM-593) | ok                                                           |
+|     7 | ok     | [BEAM-2112](https://issues.apache.org/jira/browse/BEAM-2112) | [BEAM-593](https://issues.apache.org/jira/browse/BEAM-593) | ok                                                           |
+|     8 | ok     | ok                                                           | [BEAM-593](https://issues.apache.org/jira/browse/BEAM-593) | ok                                                           |
+|     9 | ok     | ok                                                           | [BEAM-593](https://issues.apache.org/jira/browse/BEAM-593) | ok                                                           |
+|    10 | ok     | ok                                                           | [BEAM-593](https://issues.apache.org/jira/browse/BEAM-593) | ok                                                           |
+|    11 | ok     | ok                                                           | [BEAM-593](https://issues.apache.org/jira/browse/BEAM-593) | ok                                                           |
+|    12 | ok     | ok                                                           | [BEAM-593](https://issues.apache.org/jira/browse/BEAM-593) | ok                                                           |
 
 ## Batch / Synthetic / Cluster
 
@@ -219,26 +215,63 @@ TODO
 
 # Running Nexmark
 
-## Running on the DirectRunner (local)
+## Running SMOKE suite on the DirectRunner (local)
 
 Batch Mode
 
--Dexec.classpathScope="test"
+    mvn exec:java -Dexec.mainClass=org.apache.beam.integration.nexmark.Main -Pdirect-runner -Dexec.args="--runner=DirectRunner --suite=SMOKE --streaming=false --manageResources=false --monitorJobs=true --enforceEncodability=true --enforceImmutability=true"
+
+Streaming Mode
+
+    mvn exec:java -Dexec.mainClass=org.apache.beam.integration.nexmark.Main -Pdirect-runner -Dexec.args="--runner=DirectRunner --suite=SMOKE --streaming=true --manageResources=false --monitorJobs=true --enforceEncodability=true --enforceImmutability=true"
+
 
-    mvn exec:java -Dexec.mainClass=org.apache.beam.integration.nexmark.drivers.NexmarkDirectDriver -Dexec.args="--suite=SMOKE --streaming=false --numEventGenerators=4 --manageResources=false --monitorJobs=false --enforceEncodability=false --enforceImmutability=false"
+## Running SMOKE suite on the SparkRunner (local)
+
+Batch Mode
+
+    mvn exec:java -Dexec.mainClass=org.apache.beam.integration.nexmark.Main -Pspark-runner "-Dexec.args=--runner=SparkRunner --suite=SMOKE --streamTimeout=60 --streaming=false --manageResources=false --monitorJobs=true"
 
 Streaming Mode
 
-    mvn exec:java -Dexec.mainClass=org.apache.beam.integration.nexmark.drivers.NexmarkDirectDriver -Dexec.args="--suite=SMOKE --streaming=true --numEventGenerators=4 --manageResources=false --monitorJobs=false --enforceEncodability=false --enforceImmutability=false"
+    mvn exec:java -Dexec.mainClass=org.apache.beam.integration.nexmark.Main -Pspark-runner "-Dexec.args=--runner=SparkRunner --suite=SMOKE --streamTimeout=60 --streaming=true --manageResources=false --monitorJobs=true"
+
+
+## Running SMOKE suite on the FlinkRunner (local)
 
-## Running on Google Cloud Dataflow
+Batch Mode
+
+    mvn exec:java -Dexec.mainClass=org.apache.beam.integration.nexmark.Main -Pflink-runner "-Dexec.args=--runner=FlinkRunner --suite=SMOKE --streamTimeout=60 --streaming=false --manageResources=false --monitorJobs=true  --flinkMaster=local"
+
+Streaming Mode
+
+    mvn exec:java -Dexec.mainClass=org.apache.beam.integration.nexmark.Main -Pflink-runner "-Dexec.args=--runner=FlinkRunner --suite=SMOKE --streamTimeout=60 --streaming=true --manageResources=false --monitorJobs=true  --flinkMaster=local"
+
+
+## Running SMOKE suite on the ApexRunner (local)
+
+Batch Mode
+
+    mvn exec:java -Dexec.mainClass=org.apache.beam.integration.nexmark.Main -Papex-runner "-Dexec.args=--runner=ApexRunner --suite=SMOKE --streamTimeout=60 --streaming=false --manageResources=false --monitorJobs=false"
+
+Streaming Mode
+
+    mvn exec:java -Dexec.mainClass=org.apache.beam.integration.nexmark.Main -Papex-runner "-Dexec.args=--runner=ApexRunner --suite=SMOKE --streamTimeout=60 --streaming=true --manageResources=false --monitorJobs=false"
+
+
+## Running SMOKE suite on Google Cloud Dataflow
+
+Building package
+
+    mvn clean package -Pdataflow-runner
+
+Submit to Google Dataflow service
 
-An example invocation for **Query10** on the Google Cloud Dataflow
-service.
 
 ```
-java -cp integration/java/target/java-integration-all-bundled-0.2.0-incubating-SNAPSHOT.jar \
-  org.apache.beam.integration.nexmark.drivers.NexmarkGoogleDriver \
+java -cp integration/java/target/beam-integration-java-nexmark-bundled-2.1.0-SNAPSHOT.jar \
+  org.apache.beam.integration.nexmark.Main \
+  --runner=DataflowRunner
   --project=<your project> \
   --zone=<your zone> \
   --workerMachineType=n1-highmem-8 \
@@ -253,7 +286,7 @@ java -cp integration/java/target/java-integration-all-bundled-0.2.0-incubating-S
   --numEventGenerators=64 \
   --numWorkers=16 \
   --maxNumWorkers=16 \
-  --query=10 \
+  --suite=SMOKE \
   --firstEventRate=100000 \
   --nextEventRate=100000 \
   --ratePeriodSec=3600 \
@@ -270,8 +303,9 @@ java -cp integration/java/target/java-integration-all-bundled-0.2.0-incubating-S
 ```
 
 ```
-java -cp integration/java/target/java-integration-all-bundled-0.2.0-incubating-SNAPSHOT.jar \
-  org.apache.beam.integration.nexmark.drivers.NexmarkGoogleDriver \
+java -cp integration/java/target/beam-integration-java-nexmark-bundled-2.1.0-SNAPSHOT.jar \
+  org.apache.beam.integration.nexmark.Main \
+  --runner=DataflowRunner
   --project=<your project> \
   --zone=<your zone> \
   --workerMachineType=n1-highmem-8 \
@@ -285,7 +319,7 @@ java -cp integration/java/target/java-integration-all-bundled-0.2.0-incubating-S
   --monitorJobs=false \
   --numWorkers=64 \
   --maxNumWorkers=64 \
-  --query=10 \
+  --suite=SMOKE \
   --usePubsubPublishTime=true \
   --outputPath=<a gs path under which log files will be written> \
   --windowSizeSec=600 \
@@ -294,8 +328,13 @@ java -cp integration/java/target/java-integration-all-bundled-0.2.0-incubating-S
   --experiments=enable_custom_pubsub_source
 ```
 
-## Running on Flink
+## Running query 0 on a Spark cluster with yarn
+
+Building package
+
+    mvn clean package -Pspark-runner
+
+Submit to the cluster
+
+    spark-submit --master yarn-client --class org.apache.beam.integration.nexmark.Main --driver-memory 512m --executor-memory 512m --executor-cores 1 beam-integration-java-nexmark-bundled-2.1.0-SNAPSHOT.jar --runner=SparkRunner --query=0 --streamTimeout=60 --streaming=false --manageResources=false --monitorJobs=true
 
-See [BEAM_ON_FLINK_ON_GCP](./BEAM_ON_FLINK_ON_GCP.md) for instructions
-on running a NexMark pipeline using Flink hosted on a Google Compute
-Platform cluster.

http://git-wip-us.apache.org/repos/asf/beam/blob/dbd1b155/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkConfiguration.java
----------------------------------------------------------------------
diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkConfiguration.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkConfiguration.java
index 1da08b4..5a8cb71 100644
--- a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkConfiguration.java
+++ b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkConfiguration.java
@@ -140,15 +140,15 @@ public class NexmarkConfiguration implements Serializable {
 
   /** Ratio of bids to 'hot' auctions compared to all other auctions. */
   @JsonProperty
-  public int hotAuctionRatio = 1;
+  public int hotAuctionRatio = 2;
 
   /** Ratio of auctions for 'hot' sellers compared to all other people. */
   @JsonProperty
-  public int hotSellersRatio = 1;
+  public int hotSellersRatio = 4;
 
   /** Ratio of bids for 'hot' bidders compared to all other people. */
   @JsonProperty
-  public int hotBiddersRatio = 1;
+  public int hotBiddersRatio = 4;
 
   /** Window size, in seconds, for queries 3, 5, 7 and 8. */
   @JsonProperty
@@ -211,13 +211,13 @@ public class NexmarkConfiguration implements Serializable {
    * Length of occasional delay to impose on events (in seconds).
    */
   @JsonProperty
-  public long occasionalDelaySec = 0;
+  public long occasionalDelaySec = 3;
 
   /**
    * Probability that an event will be delayed by delayS.
    */
   @JsonProperty
-  public double probDelayedEvent = 0.0;
+  public double probDelayedEvent = 0.1;
 
   /**
    * Maximum size of each log file (in events). For Query10 only.


[29/55] [abbrv] beam git commit: Move WinningBids into the queries package

Posted by ie...@apache.org.
Move WinningBids into the queries package


Project: http://git-wip-us.apache.org/repos/asf/beam/repo
Commit: http://git-wip-us.apache.org/repos/asf/beam/commit/a39cb800
Tree: http://git-wip-us.apache.org/repos/asf/beam/tree/a39cb800
Diff: http://git-wip-us.apache.org/repos/asf/beam/diff/a39cb800

Branch: refs/heads/master
Commit: a39cb80009f569e1c8ba82ee9c67a7c5dbe3d16f
Parents: a6dbdfa
Author: Ismaël Mejía <ie...@apache.org>
Authored: Sun Apr 30 17:44:07 2017 +0200
Committer: Ismaël Mejía <ie...@gmail.com>
Committed: Wed Aug 23 19:07:28 2017 +0200

----------------------------------------------------------------------
 .../integration/nexmark/AbstractSimulator.java  | 210 ----------
 .../beam/integration/nexmark/NexmarkQuery.java  | 267 -------------
 .../integration/nexmark/NexmarkQueryModel.java  | 122 ------
 .../beam/integration/nexmark/NexmarkRunner.java |   2 +
 .../beam/integration/nexmark/WinningBids.java   | 377 ------------------
 .../nexmark/WinningBidsSimulator.java           | 205 ----------
 .../integration/nexmark/model/AuctionBid.java   |   3 +-
 .../nexmark/queries/AbstractSimulator.java      | 211 +++++++++++
 .../nexmark/queries/NexmarkQuery.java           | 270 +++++++++++++
 .../nexmark/queries/NexmarkQueryModel.java      | 123 ++++++
 .../integration/nexmark/queries/Query0.java     |   1 -
 .../nexmark/queries/Query0Model.java            |   4 +-
 .../integration/nexmark/queries/Query1.java     |   1 -
 .../integration/nexmark/queries/Query10.java    |   1 -
 .../integration/nexmark/queries/Query11.java    |   1 -
 .../integration/nexmark/queries/Query12.java    |   1 -
 .../nexmark/queries/Query1Model.java            |   2 -
 .../integration/nexmark/queries/Query2.java     |   1 -
 .../nexmark/queries/Query2Model.java            |   2 -
 .../integration/nexmark/queries/Query3.java     |   1 -
 .../nexmark/queries/Query3Model.java            |   2 -
 .../integration/nexmark/queries/Query4.java     |   2 -
 .../nexmark/queries/Query4Model.java            |   3 -
 .../integration/nexmark/queries/Query5.java     |   1 -
 .../nexmark/queries/Query5Model.java            |   2 -
 .../integration/nexmark/queries/Query6.java     |   2 -
 .../nexmark/queries/Query6Model.java            |   3 -
 .../integration/nexmark/queries/Query7.java     |   1 -
 .../nexmark/queries/Query7Model.java            |   2 -
 .../integration/nexmark/queries/Query8.java     |   1 -
 .../nexmark/queries/Query8Model.java            |   2 -
 .../integration/nexmark/queries/Query9.java     |   2 -
 .../nexmark/queries/Query9Model.java            |   3 -
 .../nexmark/queries/WinningBids.java            | 379 +++++++++++++++++++
 .../nexmark/queries/WinningBidsSimulator.java   | 207 ++++++++++
 .../integration/nexmark/queries/QueryTest.java  |   2 -
 36 files changed, 1194 insertions(+), 1225 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/beam/blob/a39cb800/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/AbstractSimulator.java
----------------------------------------------------------------------
diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/AbstractSimulator.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/AbstractSimulator.java
deleted file mode 100644
index b012842..0000000
--- a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/AbstractSimulator.java
+++ /dev/null
@@ -1,210 +0,0 @@
-/*
- * 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.beam.integration.nexmark;
-
-import java.util.ArrayList;
-import java.util.Iterator;
-import java.util.List;
-import javax.annotation.Nullable;
-
-import org.apache.beam.sdk.transforms.windowing.BoundedWindow;
-import org.apache.beam.sdk.values.TimestampedValue;
-import org.joda.time.Duration;
-import org.joda.time.Instant;
-
-/**
- * Abstract base class for simulator of a query.
- *
- * @param <InputT> Type of input elements.
- * @param <OutputT> Type of output elements.
- */
-public abstract class AbstractSimulator<InputT, OutputT> {
-  /** Window size for action bucket sampling. */
-  public static final Duration WINDOW_SIZE = Duration.standardMinutes(1);
-
-  /** Input event stream we should draw from. */
-  private final Iterator<TimestampedValue<InputT>> input;
-
-  /** Set to true when no more results. */
-  private boolean isDone;
-
-  /**
-   * Results which have not yet been returned by the {@link #results} iterator.
-   */
-  private final List<TimestampedValue<OutputT>> pendingResults;
-
-  /**
-   * Current window timestamp (ms since epoch).
-   */
-  private long currentWindow;
-
-  /**
-   * Number of (possibly intermediate) results for the current window.
-   */
-  private long currentCount;
-
-  /**
-   * Result counts per window which have not yet been returned by the {@link #resultsPerWindow}
-   * iterator.
-   */
-  private final List<Long> pendingCounts;
-
-  public AbstractSimulator(Iterator<TimestampedValue<InputT>> input) {
-    this.input = input;
-    isDone = false;
-    pendingResults = new ArrayList<>();
-    currentWindow = BoundedWindow.TIMESTAMP_MIN_VALUE.getMillis();
-    currentCount = 0;
-    pendingCounts = new ArrayList<>();
-  }
-
-  /** Called by implementors of {@link #run}: Fetch the next input element. */
-  @Nullable
-  protected TimestampedValue<InputT> nextInput() {
-    if (!input.hasNext()) {
-      return null;
-    }
-    TimestampedValue<InputT> timestampedInput = input.next();
-    NexmarkUtils.info("input: %s", timestampedInput);
-    return timestampedInput;
-  }
-
-  /**
-   * Called by implementors of {@link #run}:  Capture an intermediate result, for the purpose of
-   * recording the expected activity of the query over time.
-   */
-  protected void addIntermediateResult(TimestampedValue<OutputT> result) {
-    NexmarkUtils.info("intermediate result: %s", result);
-    updateCounts(result.getTimestamp());
-  }
-
-  /**
-   * Called by implementors of {@link #run}: Capture a final result, for the purpose of checking
-   * semantic correctness.
-   */
-  protected void addResult(TimestampedValue<OutputT> result) {
-    NexmarkUtils.info("result: %s", result);
-    pendingResults.add(result);
-    updateCounts(result.getTimestamp());
-  }
-
-  /**
-   * Update window and counts.
-   */
-  private void updateCounts(Instant timestamp) {
-    long window = timestamp.getMillis() - timestamp.getMillis() % WINDOW_SIZE.getMillis();
-    if (window > currentWindow) {
-      if (currentWindow > BoundedWindow.TIMESTAMP_MIN_VALUE.getMillis()) {
-        pendingCounts.add(currentCount);
-      }
-      currentCount = 0;
-      currentWindow = window;
-    }
-    currentCount++;
-  }
-
-  /** Called by implementors of {@link #run}: Record that no more results will be emitted. */
-  protected void allDone() {
-    isDone = true;
-  }
-
-  /**
-   * Overridden by derived classes to do the next increment of work. Each call should
-   * call one or more of {@link #nextInput}, {@link #addIntermediateResult}, {@link #addResult}
-   * or {@link #allDone}. It is ok for a single call to emit more than one result via
-   * {@link #addResult}. It is ok for a single call to run the entire simulation, though
-   * this will prevent the {@link #results} and {@link #resultsPerWindow} iterators to
-   * stall.
-   */
-  protected abstract void run();
-
-  /**
-   * Return iterator over all expected timestamped results. The underlying simulator state is
-   * changed. Only one of {@link #results} or {@link #resultsPerWindow} can be called.
-   */
-  public Iterator<TimestampedValue<OutputT>> results() {
-    return new Iterator<TimestampedValue<OutputT>>() {
-      @Override
-      public boolean hasNext() {
-        while (true) {
-          if (!pendingResults.isEmpty()) {
-            return true;
-          }
-          if (isDone) {
-            return false;
-          }
-          run();
-        }
-      }
-
-      @Override
-      public TimestampedValue<OutputT> next() {
-        TimestampedValue<OutputT> result = pendingResults.get(0);
-        pendingResults.remove(0);
-        return result;
-      }
-
-      @Override
-      public void remove() {
-        throw new UnsupportedOperationException();
-      }
-    };
-  }
-
-  /**
-   * Return an iterator over the number of results per {@link #WINDOW_SIZE} period. The underlying
-   * simulator state is changed.  Only one of {@link #results} or {@link #resultsPerWindow} can be
-   * called.
-   */
-  public Iterator<Long> resultsPerWindow() {
-    return new Iterator<Long>() {
-      @Override
-      public boolean hasNext() {
-        while (true) {
-          if (!pendingCounts.isEmpty()) {
-            return true;
-          }
-          if (isDone) {
-            if (currentCount > 0) {
-              pendingCounts.add(currentCount);
-              currentCount = 0;
-              currentWindow = BoundedWindow.TIMESTAMP_MAX_VALUE.getMillis();
-              return true;
-            } else {
-              return false;
-            }
-          }
-          run();
-        }
-      }
-
-      @Override
-      public Long next() {
-        Long result = pendingCounts.get(0);
-        pendingCounts.remove(0);
-        return result;
-      }
-
-      @Override
-      public void remove() {
-        throw new UnsupportedOperationException();
-      }
-    };
-  }
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/a39cb800/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkQuery.java
----------------------------------------------------------------------
diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkQuery.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkQuery.java
deleted file mode 100644
index ab1c305..0000000
--- a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkQuery.java
+++ /dev/null
@@ -1,267 +0,0 @@
-/*
- * 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.beam.integration.nexmark;
-
-import org.apache.beam.integration.nexmark.model.Auction;
-import org.apache.beam.integration.nexmark.model.Bid;
-import org.apache.beam.integration.nexmark.model.Event;
-import org.apache.beam.integration.nexmark.model.KnownSize;
-import org.apache.beam.integration.nexmark.model.Person;
-import org.apache.beam.sdk.metrics.Counter;
-import org.apache.beam.sdk.metrics.Metrics;
-import org.apache.beam.sdk.transforms.DoFn;
-import org.apache.beam.sdk.transforms.Filter;
-import org.apache.beam.sdk.transforms.PTransform;
-import org.apache.beam.sdk.transforms.ParDo;
-import org.apache.beam.sdk.transforms.SerializableFunction;
-import org.apache.beam.sdk.values.KV;
-import org.apache.beam.sdk.values.PCollection;
-import org.apache.beam.sdk.values.TimestampedValue;
-import org.apache.beam.sdk.values.TupleTag;
-import org.joda.time.Instant;
-
-/**
- * Base class for the eight 'NEXMark' queries. Supplies some fragments common to
- * multiple queries.
- */
-public abstract class NexmarkQuery
-    extends PTransform<PCollection<Event>, PCollection<TimestampedValue<KnownSize>>> {
-  protected static final TupleTag<Auction> AUCTION_TAG = new TupleTag<>("auctions");
-  protected static final TupleTag<Bid> BID_TAG = new TupleTag<>("bids");
-  protected static final TupleTag<Person> PERSON_TAG = new TupleTag<>("person");
-
-  /** Predicate to detect a new person event. */
-  protected static final SerializableFunction<Event, Boolean> IS_NEW_PERSON =
-      new SerializableFunction<Event, Boolean>() {
-        @Override
-        public Boolean apply(Event event) {
-          return event.newPerson != null;
-        }
-      };
-
-  /** DoFn to convert a new person event to a person. */
-  protected static final DoFn<Event, Person> AS_PERSON = new DoFn<Event, Person>() {
-    @ProcessElement
-    public void processElement(ProcessContext c) {
-      c.output(c.element().newPerson);
-    }
-  };
-
-  /** Predicate to detect a new auction event. */
-  protected static final SerializableFunction<Event, Boolean> IS_NEW_AUCTION =
-      new SerializableFunction<Event, Boolean>() {
-        @Override
-        public Boolean apply(Event event) {
-          return event.newAuction != null;
-        }
-      };
-
-  /** DoFn to convert a new auction event to an auction. */
-  protected static final DoFn<Event, Auction> AS_AUCTION = new DoFn<Event, Auction>() {
-    @ProcessElement
-    public void processElement(ProcessContext c) {
-      c.output(c.element().newAuction);
-    }
-  };
-
-  /** Predicate to detect a new bid event. */
-  protected static final SerializableFunction<Event, Boolean> IS_BID =
-      new SerializableFunction<Event, Boolean>() {
-        @Override
-        public Boolean apply(Event event) {
-          return event.bid != null;
-        }
-      };
-
-  /** DoFn to convert a bid event to a bid. */
-  protected static final DoFn<Event, Bid> AS_BID = new DoFn<Event, Bid>() {
-    @ProcessElement
-    public void processElement(ProcessContext c) {
-      c.output(c.element().bid);
-    }
-  };
-
-  /** Transform to key each person by their id. */
-  protected static final ParDo.SingleOutput<Person, KV<Long, Person>> PERSON_BY_ID =
-      ParDo.of(new DoFn<Person, KV<Long, Person>>() {
-             @ProcessElement
-             public void processElement(ProcessContext c) {
-               c.output(KV.of(c.element().id, c.element()));
-             }
-           });
-
-  /** Transform to key each auction by its id. */
-  protected static final ParDo.SingleOutput<Auction, KV<Long, Auction>> AUCTION_BY_ID =
-      ParDo.of(new DoFn<Auction, KV<Long, Auction>>() {
-             @ProcessElement
-             public void processElement(ProcessContext c) {
-               c.output(KV.of(c.element().id, c.element()));
-             }
-           });
-
-  /** Transform to key each auction by its seller id. */
-  protected static final ParDo.SingleOutput<Auction, KV<Long, Auction>> AUCTION_BY_SELLER =
-      ParDo.of(new DoFn<Auction, KV<Long, Auction>>() {
-             @ProcessElement
-             public void processElement(ProcessContext c) {
-               c.output(KV.of(c.element().seller, c.element()));
-             }
-           });
-
-  /** Transform to key each bid by it's auction id. */
-  protected static final ParDo.SingleOutput<Bid, KV<Long, Bid>> BID_BY_AUCTION =
-      ParDo.of(new DoFn<Bid, KV<Long, Bid>>() {
-             @ProcessElement
-             public void processElement(ProcessContext c) {
-               c.output(KV.of(c.element().auction, c.element()));
-             }
-           });
-
-  /** Transform to project the auction id from each bid. */
-  protected static final ParDo.SingleOutput<Bid, Long> BID_TO_AUCTION =
-      ParDo.of(new DoFn<Bid, Long>() {
-             @ProcessElement
-             public void processElement(ProcessContext c) {
-               c.output(c.element().auction);
-             }
-           });
-
-  /** Transform to project the price from each bid. */
-  protected static final ParDo.SingleOutput<Bid, Long> BID_TO_PRICE =
-      ParDo.of(new DoFn<Bid, Long>() {
-             @ProcessElement
-             public void processElement(ProcessContext c) {
-               c.output(c.element().price);
-             }
-           });
-
-  /** Transform to emit each event with the timestamp embedded within it. */
-  public static final ParDo.SingleOutput<Event, Event> EVENT_TIMESTAMP_FROM_DATA =
-      ParDo.of(new DoFn<Event, Event>() {
-             @ProcessElement
-             public void processElement(ProcessContext c) {
-               Event e = c.element();
-               if (e.bid != null) {
-                 c.outputWithTimestamp(e, new Instant(e.bid.dateTime));
-               } else if (e.newPerson != null) {
-                 c.outputWithTimestamp(e, new Instant(e.newPerson.dateTime));
-               } else if (e.newAuction != null) {
-                 c.outputWithTimestamp(e, new Instant(e.newAuction.dateTime));
-               }
-             }
-           });
-
-  /**
-   * Transform to filter for just the new auction events.
-   */
-  protected static final PTransform<PCollection<Event>, PCollection<Auction>> JUST_NEW_AUCTIONS =
-      new PTransform<PCollection<Event>, PCollection<Auction>>("justNewAuctions") {
-        @Override
-        public PCollection<Auction> expand(PCollection<Event> input) {
-          return input.apply("IsNewAuction", Filter.by(IS_NEW_AUCTION))
-                      .apply("AsAuction", ParDo.of(AS_AUCTION));
-        }
-      };
-
-  /**
-   * Transform to filter for just the new person events.
-   */
-  protected static final PTransform<PCollection<Event>, PCollection<Person>> JUST_NEW_PERSONS =
-      new PTransform<PCollection<Event>, PCollection<Person>>("justNewPersons") {
-        @Override
-        public PCollection<Person> expand(PCollection<Event> input) {
-          return input.apply("IsNewPerson", Filter.by(IS_NEW_PERSON))
-                      .apply("AsPerson", ParDo.of(AS_PERSON));
-        }
-      };
-
-  /**
-   * Transform to filter for just the bid events.
-   */
-  protected static final PTransform<PCollection<Event>, PCollection<Bid>> JUST_BIDS =
-      new PTransform<PCollection<Event>, PCollection<Bid>>("justBids") {
-        @Override
-        public PCollection<Bid> expand(PCollection<Event> input) {
-          return input.apply("IsBid", Filter.by(IS_BID))
-                      .apply("AsBid", ParDo.of(AS_BID));
-        }
-      };
-
-  protected final NexmarkConfiguration configuration;
-  public final Monitor<Event> eventMonitor;
-  public final Monitor<KnownSize> resultMonitor;
-  public final Monitor<Event> endOfStreamMonitor;
-  protected final Counter fatalCounter;
-
-  protected NexmarkQuery(NexmarkConfiguration configuration, String name) {
-    super(name);
-    this.configuration = configuration;
-    if (configuration.debug) {
-      eventMonitor = new Monitor<>(name + ".Events", "event");
-      resultMonitor = new Monitor<>(name + ".Results", "result");
-      endOfStreamMonitor = new Monitor<>(name + ".EndOfStream", "end");
-      fatalCounter = Metrics.counter(name , "fatal");
-    } else {
-      eventMonitor = null;
-      resultMonitor = null;
-      endOfStreamMonitor = null;
-      fatalCounter = null;
-    }
-  }
-
-  /**
-   * Implement the actual query. All we know about the result is it has a known encoded size.
-   */
-  protected abstract PCollection<KnownSize> applyPrim(PCollection<Event> events);
-
-  @Override
-  public PCollection<TimestampedValue<KnownSize>> expand(PCollection<Event> events) {
-
-    if (configuration.debug) {
-      events =
-          events
-              // Monitor events as they go by.
-              .apply(name + ".Monitor", eventMonitor.getTransform())
-              // Count each type of event.
-              .apply(name + ".Snoop", NexmarkUtils.snoop(name));
-    }
-
-    if (configuration.cpuDelayMs > 0) {
-      // Slow down by pegging one core at 100%.
-      events = events.apply(name + ".CpuDelay",
-              NexmarkUtils.<Event>cpuDelay(name, configuration.cpuDelayMs));
-    }
-
-    if (configuration.diskBusyBytes > 0) {
-      // Slow down by forcing bytes to durable store.
-      events = events.apply(name + ".DiskBusy",
-              NexmarkUtils.<Event>diskBusy(name, configuration.diskBusyBytes));
-    }
-
-    // Run the query.
-    PCollection<KnownSize> queryResults = applyPrim(events);
-
-    if (configuration.debug) {
-      // Monitor results as they go by.
-      queryResults = queryResults.apply(name + ".Debug", resultMonitor.getTransform());
-    }
-
-    // Timestamp the query results.
-    return queryResults.apply(name + ".Stamp", NexmarkUtils.<KnownSize>stamp(name));
-  }
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/a39cb800/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkQueryModel.java
----------------------------------------------------------------------
diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkQueryModel.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkQueryModel.java
deleted file mode 100644
index b2b1826..0000000
--- a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkQueryModel.java
+++ /dev/null
@@ -1,122 +0,0 @@
-/*
- * 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.beam.integration.nexmark;
-
-import java.io.Serializable;
-import java.util.ArrayList;
-import java.util.Collection;
-import java.util.HashSet;
-import java.util.Iterator;
-import java.util.List;
-import java.util.Set;
-
-import org.apache.beam.integration.nexmark.model.KnownSize;
-import org.apache.beam.sdk.transforms.SerializableFunction;
-import org.apache.beam.sdk.values.TimestampedValue;
-
-
-import org.hamcrest.core.IsEqual;
-import org.joda.time.Duration;
-import org.joda.time.Instant;
-import org.junit.Assert;
-
-/**
- * Base class for models of the eight NEXMark queries. Provides an assertion function which can be
- * applied against the actual query results to check their consistency with the model.
- */
-public abstract class NexmarkQueryModel implements Serializable {
-  protected final NexmarkConfiguration configuration;
-
-  public NexmarkQueryModel(NexmarkConfiguration configuration) {
-    this.configuration = configuration;
-  }
-
-  /**
-   * Return the start of the most recent window of {@code size} and {@code period} which ends
-   * strictly before {@code timestamp}.
-   */
-  public static Instant windowStart(Duration size, Duration period, Instant timestamp) {
-    long ts = timestamp.getMillis();
-    long p = period.getMillis();
-    long lim = ts - ts % p;
-    long s = size.getMillis();
-    return new Instant(lim - s);
-  }
-
-  /** Convert {@code itr} to strings capturing values, timestamps and order. */
-  protected static <T> List<String> toValueTimestampOrder(Iterator<TimestampedValue<T>> itr) {
-    List<String> strings = new ArrayList<>();
-    while (itr.hasNext()) {
-      strings.add(itr.next().toString());
-    }
-    return strings;
-  }
-
-  /** Convert {@code itr} to strings capturing values and order. */
-  protected static <T> List<String> toValueOrder(Iterator<TimestampedValue<T>> itr) {
-    List<String> strings = new ArrayList<>();
-    while (itr.hasNext()) {
-      strings.add(itr.next().getValue().toString());
-    }
-    return strings;
-  }
-
-  /** Convert {@code itr} to strings capturing values only. */
-  protected static <T> Set<String> toValue(Iterator<TimestampedValue<T>> itr) {
-    Set<String> strings = new HashSet<>();
-    while (itr.hasNext()) {
-      strings.add(itr.next().getValue().toString());
-    }
-    return strings;
-  }
-
-  /** Return simulator for query. */
-  protected abstract AbstractSimulator<?, ?> simulator();
-
-  /** Return sub-sequence of results which are significant for model. */
-  protected Iterable<TimestampedValue<KnownSize>> relevantResults(
-      Iterable<TimestampedValue<KnownSize>> results) {
-    return results;
-  }
-
-  /**
-   * Convert iterator of elements to collection of strings to use when testing coherence of model
-   * against actual query results.
-   */
-  protected abstract <T> Collection<String> toCollection(Iterator<TimestampedValue<T>> itr);
-
-  /** Return assertion to use on results of pipeline for this query. */
-  public SerializableFunction<Iterable<TimestampedValue<KnownSize>>, Void> assertionFor() {
-    final Collection<String> expectedStrings = toCollection(simulator().results());
-    final String[] expectedStringsArray =
-      expectedStrings.toArray(new String[expectedStrings.size()]);
-
-    return new SerializableFunction<Iterable<TimestampedValue<KnownSize>>, Void>() {
-      @Override
-      public Void apply(Iterable<TimestampedValue<KnownSize>> actual) {
-      Collection<String> actualStrings = toCollection(relevantResults(actual).iterator());
-        Assert.assertThat("wrong pipeline output", actualStrings,
-          IsEqual.equalTo(expectedStrings));
-//compare without order
-//      Assert.assertThat("wrong pipeline output", actualStrings,
-//        IsIterableContainingInAnyOrder.containsInAnyOrder(expectedStringsArray));
-        return null;
-      }
-    };
-  }
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/a39cb800/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkRunner.java
----------------------------------------------------------------------
diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkRunner.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkRunner.java
index ebfd196..a3c4d33 100644
--- a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkRunner.java
+++ b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkRunner.java
@@ -38,6 +38,8 @@ import org.apache.beam.integration.nexmark.model.Bid;
 import org.apache.beam.integration.nexmark.model.Event;
 import org.apache.beam.integration.nexmark.model.KnownSize;
 import org.apache.beam.integration.nexmark.model.Person;
+import org.apache.beam.integration.nexmark.queries.NexmarkQuery;
+import org.apache.beam.integration.nexmark.queries.NexmarkQueryModel;
 import org.apache.beam.integration.nexmark.queries.Query0;
 import org.apache.beam.integration.nexmark.queries.Query0Model;
 import org.apache.beam.integration.nexmark.queries.Query1;

http://git-wip-us.apache.org/repos/asf/beam/blob/a39cb800/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/WinningBids.java
----------------------------------------------------------------------
diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/WinningBids.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/WinningBids.java
deleted file mode 100644
index 3815b9d..0000000
--- a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/WinningBids.java
+++ /dev/null
@@ -1,377 +0,0 @@
-/*
- * 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.beam.integration.nexmark;
-
-import static com.google.common.base.Preconditions.checkState;
-
-import com.fasterxml.jackson.annotation.JsonCreator;
-import java.io.IOException;
-import java.io.InputStream;
-import java.io.OutputStream;
-import java.io.Serializable;
-import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.Collection;
-import java.util.List;
-import java.util.Map;
-import java.util.TreeMap;
-import org.apache.beam.integration.nexmark.model.Auction;
-import org.apache.beam.integration.nexmark.model.AuctionBid;
-import org.apache.beam.integration.nexmark.model.Bid;
-import org.apache.beam.integration.nexmark.model.Event;
-import org.apache.beam.integration.nexmark.sources.GeneratorConfig;
-import org.apache.beam.sdk.coders.Coder;
-import org.apache.beam.sdk.coders.CoderException;
-import org.apache.beam.sdk.coders.CustomCoder;
-import org.apache.beam.sdk.coders.VarIntCoder;
-import org.apache.beam.sdk.coders.VarLongCoder;
-import org.apache.beam.sdk.metrics.Counter;
-import org.apache.beam.sdk.metrics.Metrics;
-import org.apache.beam.sdk.transforms.DoFn;
-import org.apache.beam.sdk.transforms.PTransform;
-import org.apache.beam.sdk.transforms.ParDo;
-import org.apache.beam.sdk.transforms.join.CoGbkResult;
-import org.apache.beam.sdk.transforms.join.CoGroupByKey;
-import org.apache.beam.sdk.transforms.join.KeyedPCollectionTuple;
-import org.apache.beam.sdk.transforms.windowing.IntervalWindow;
-import org.apache.beam.sdk.transforms.windowing.Window;
-import org.apache.beam.sdk.transforms.windowing.WindowFn;
-import org.apache.beam.sdk.transforms.windowing.WindowMappingFn;
-import org.apache.beam.sdk.values.KV;
-import org.apache.beam.sdk.values.PCollection;
-import org.joda.time.Instant;
-
-/**
- * A transform to find the winning bid for each closed auction. In pseudo CQL syntax:
- *
- * <pre>{@code
- * SELECT Rstream(A.*, B.auction, B.bidder, MAX(B.price), B.dateTime)
- * FROM Auction A [ROWS UNBOUNDED], Bid B [ROWS UNBOUNDED]
- * WHERE A.id = B.auction AND B.datetime < A.expires AND A.expires < CURRENT_TIME
- * GROUP BY A.id
- * }</pre>
- *
- * <p>We will also check that the winning bid is above the auction reserve. Note that
- * we ignore the auction opening bid value since it has no impact on which bid eventually wins,
- * if any.
- *
- * <p>Our implementation will use a custom windowing function in order to bring bids and
- * auctions together without requiring global state.
- */
-public class WinningBids extends PTransform<PCollection<Event>, PCollection<AuctionBid>> {
-  /** Windows for open auctions and bids. */
-  private static class AuctionOrBidWindow extends IntervalWindow implements Serializable {
-    /** Id of auction this window is for. */
-    public final long auction;
-
-    /**
-     * True if this window represents an actual auction, and thus has a start/end
-     * time matching that of the auction. False if this window represents a bid, and
-     * thus has an unbounded start/end time.
-     */
-    public final boolean isAuctionWindow;
-
-    /** For avro only. */
-    private AuctionOrBidWindow() {
-      super(TIMESTAMP_MIN_VALUE, TIMESTAMP_MAX_VALUE);
-      auction = 0;
-      isAuctionWindow = false;
-    }
-
-    private AuctionOrBidWindow(
-        Instant start, Instant end, long auctionId, boolean isAuctionWindow) {
-      super(start, end);
-      this.auction = auctionId;
-      this.isAuctionWindow = isAuctionWindow;
-    }
-
-    /** Return an auction window for {@code auction}. */
-    public static AuctionOrBidWindow forAuction(Instant timestamp, Auction auction) {
-      AuctionOrBidWindow result =
-          new AuctionOrBidWindow(timestamp, new Instant(auction.expires), auction.id, true);
-      return result;
-    }
-
-    /**
-     * Return a bid window for {@code bid}. It should later be merged into
-     * the corresponding auction window. However, it is possible this bid is for an already
-     * expired auction, or for an auction which the system has not yet seen. So we
-     * give the bid a bit of wiggle room in its interval.
-     */
-    public static AuctionOrBidWindow forBid(
-        long expectedAuctionDurationMs, Instant timestamp, Bid bid) {
-      // At this point we don't know which auctions are still valid, and the bid may
-      // be for an auction which won't start until some unknown time in the future
-      // (due to Generator.AUCTION_ID_LEAD in Generator.nextBid).
-      // A real system would atomically reconcile bids and auctions by a separate mechanism.
-      // If we give bids an unbounded window it is possible a bid for an auction which
-      // has already expired would cause the system watermark to stall, since that window
-      // would never be retired.
-      // Instead, we will just give the bid a finite window which expires at
-      // the upper bound of auctions assuming the auction starts at the same time as the bid,
-      // and assuming the system is running at its lowest event rate (as per interEventDelayUs).
-      AuctionOrBidWindow result = new AuctionOrBidWindow(
-          timestamp, timestamp.plus(expectedAuctionDurationMs * 2), bid.auction, false);
-      return result;
-    }
-
-    /** Is this an auction window? */
-    public boolean isAuctionWindow() {
-      return isAuctionWindow;
-    }
-
-    @Override
-    public String toString() {
-      return String.format("AuctionOrBidWindow{start:%s; end:%s; auction:%d; isAuctionWindow:%s}",
-          start(), end(), auction, isAuctionWindow);
-    }
-  }
-
-  /**
-   * Encodes an {@link AuctionOrBidWindow} as an {@link IntervalWindow} and an auction id long.
-   */
-  private static class AuctionOrBidWindowCoder extends CustomCoder<AuctionOrBidWindow> {
-    private static final AuctionOrBidWindowCoder INSTANCE = new AuctionOrBidWindowCoder();
-    private static final Coder<IntervalWindow> SUPER_CODER = IntervalWindow.getCoder();
-    private static final Coder<Long> ID_CODER = VarLongCoder.of();
-    private static final Coder<Integer> INT_CODER = VarIntCoder.of();
-
-    @JsonCreator
-    public static AuctionOrBidWindowCoder of() {
-      return INSTANCE;
-    }
-
-    @Override
-    public void encode(AuctionOrBidWindow window, OutputStream outStream, Coder.Context context)
-        throws IOException, CoderException {
-      SUPER_CODER.encode(window, outStream, Coder.Context.NESTED);
-      ID_CODER.encode(window.auction, outStream, Coder.Context.NESTED);
-      INT_CODER.encode(window.isAuctionWindow ? 1 : 0, outStream, Coder.Context.NESTED);
-    }
-
-    @Override
-    public AuctionOrBidWindow decode(InputStream inStream, Coder.Context context)
-        throws IOException, CoderException {
-      IntervalWindow superWindow = SUPER_CODER.decode(inStream, Coder.Context.NESTED);
-      long auction = ID_CODER.decode(inStream, Coder.Context.NESTED);
-      boolean isAuctionWindow =
-          INT_CODER.decode(inStream, Coder.Context.NESTED) == 0 ? false : true;
-      return new AuctionOrBidWindow(
-          superWindow.start(), superWindow.end(), auction, isAuctionWindow);
-    }
-
-    @Override public void verifyDeterministic() throws NonDeterministicException {}
-  }
-
-  /** Assign events to auction windows and merges them intelligently. */
-  private static class AuctionOrBidWindowFn extends WindowFn<Event, AuctionOrBidWindow> {
-    /** Expected duration of auctions in ms. */
-    private final long expectedAuctionDurationMs;
-
-    public AuctionOrBidWindowFn(long expectedAuctionDurationMs) {
-      this.expectedAuctionDurationMs = expectedAuctionDurationMs;
-    }
-
-    @Override
-    public Collection<AuctionOrBidWindow> assignWindows(AssignContext c) {
-      Event event = c.element();
-      if (event.newAuction != null) {
-        // Assign auctions to an auction window which expires at the auction's close.
-        return Arrays.asList(AuctionOrBidWindow.forAuction(c.timestamp(), event.newAuction));
-      } else if (event.bid != null) {
-        // Assign bids to a temporary bid window which will later be merged into the appropriate
-        // auction window.
-        return Arrays.asList(
-            AuctionOrBidWindow.forBid(expectedAuctionDurationMs, c.timestamp(), event.bid));
-      } else {
-        // Don't assign people to any window. They will thus be dropped.
-        return Arrays.asList();
-      }
-    }
-
-    @Override
-    public void mergeWindows(MergeContext c) throws Exception {
-      // Split and index the auction and bid windows by auction id.
-      Map<Long, AuctionOrBidWindow> idToTrueAuctionWindow = new TreeMap<>();
-      Map<Long, List<AuctionOrBidWindow>> idToBidAuctionWindows = new TreeMap<>();
-      for (AuctionOrBidWindow window : c.windows()) {
-        if (window.isAuctionWindow()) {
-          idToTrueAuctionWindow.put(window.auction, window);
-        } else {
-          List<AuctionOrBidWindow> bidWindows = idToBidAuctionWindows.get(window.auction);
-          if (bidWindows == null) {
-            bidWindows = new ArrayList<>();
-            idToBidAuctionWindows.put(window.auction, bidWindows);
-          }
-          bidWindows.add(window);
-        }
-      }
-
-      // Merge all 'bid' windows into their corresponding 'auction' window, provided the
-      // auction has not expired.
-      for (long auction : idToTrueAuctionWindow.keySet()) {
-        AuctionOrBidWindow auctionWindow = idToTrueAuctionWindow.get(auction);
-        List<AuctionOrBidWindow> bidWindows = idToBidAuctionWindows.get(auction);
-        if (bidWindows != null) {
-          List<AuctionOrBidWindow> toBeMerged = new ArrayList<>();
-          for (AuctionOrBidWindow bidWindow : bidWindows) {
-            if (bidWindow.start().isBefore(auctionWindow.end())) {
-              toBeMerged.add(bidWindow);
-            }
-            // else: This bid window will remain until its expire time, at which point it
-            // will expire without ever contributing to an output.
-          }
-          if (!toBeMerged.isEmpty()) {
-            toBeMerged.add(auctionWindow);
-            c.merge(toBeMerged, auctionWindow);
-          }
-        }
-      }
-    }
-
-    @Override
-    public boolean isCompatible(WindowFn<?, ?> other) {
-      return other instanceof AuctionOrBidWindowFn;
-    }
-
-    @Override
-    public Coder<AuctionOrBidWindow> windowCoder() {
-      return AuctionOrBidWindowCoder.of();
-    }
-
-    @Override
-    public WindowMappingFn<AuctionOrBidWindow> getDefaultWindowMappingFn() {
-      throw new UnsupportedOperationException("AuctionWindowFn not supported for side inputs");
-    }
-
-    /**
-     * Below we will GBK auctions and bids on their auction ids. Then we will reduce those
-     * per id to emit {@code (auction, winning bid)} pairs for auctions which have expired with at
-     * least one valid bid. We would like those output pairs to have a timestamp of the auction's
-     * expiry (since that's the earliest we know for sure we have the correct winner). We would
-     * also like to make that winning results are available to following stages at the auction's
-     * expiry.
-     *
-     * <p>Each result of the GBK will have a timestamp of the min of the result of this object's
-     * assignOutputTime over all records which end up in one of its iterables. Thus we get the
-     * desired behavior if we ignore each record's timestamp and always return the auction window's
-     * 'maxTimestamp', which will correspond to the auction's expiry.
-     *
-     * <p>In contrast, if this object's assignOutputTime were to return 'inputTimestamp'
-     * (the usual implementation), then each GBK record will take as its timestamp the minimum of
-     * the timestamps of all bids and auctions within it, which will always be the auction's
-     * timestamp. An auction which expires well into the future would thus hold up the watermark
-     * of the GBK results until that auction expired. That in turn would hold up all winning pairs.
-     */
-    @Override
-    public Instant getOutputTime(
-        Instant inputTimestamp, AuctionOrBidWindow window) {
-      return window.maxTimestamp();
-    }
-  }
-
-  private final AuctionOrBidWindowFn auctionOrBidWindowFn;
-
-  public WinningBids(String name, NexmarkConfiguration configuration) {
-    super(name);
-    // What's the expected auction time (when the system is running at the lowest event rate).
-    long[] interEventDelayUs = configuration.rateShape.interEventDelayUs(
-        configuration.firstEventRate, configuration.nextEventRate,
-        configuration.rateUnit, configuration.numEventGenerators);
-    long longestDelayUs = 0;
-    for (int i = 0; i < interEventDelayUs.length; i++) {
-      longestDelayUs = Math.max(longestDelayUs, interEventDelayUs[i]);
-    }
-    // Adjust for proportion of auction events amongst all events.
-    longestDelayUs =
-        (longestDelayUs * GeneratorConfig.PROPORTION_DENOMINATOR)
-        / GeneratorConfig.AUCTION_PROPORTION;
-    // Adjust for number of in-flight auctions.
-    longestDelayUs = longestDelayUs * configuration.numInFlightAuctions;
-    long expectedAuctionDurationMs = (longestDelayUs + 999) / 1000;
-    NexmarkUtils.console("Expected auction duration is %d ms", expectedAuctionDurationMs);
-    auctionOrBidWindowFn = new AuctionOrBidWindowFn(expectedAuctionDurationMs);
-  }
-
-  @Override
-  public PCollection<AuctionBid> expand(PCollection<Event> events) {
-    // Window auctions and bids into custom auction windows. New people events will be discarded.
-    // This will allow us to bring bids and auctions together irrespective of how long
-    // each auction is open for.
-    events = events.apply("Window", Window.into(auctionOrBidWindowFn));
-
-    // Key auctions by their id.
-    PCollection<KV<Long, Auction>> auctionsById =
-        events.apply(NexmarkQuery.JUST_NEW_AUCTIONS)
-              .apply("AuctionById:", NexmarkQuery.AUCTION_BY_ID);
-
-    // Key bids by their auction id.
-    PCollection<KV<Long, Bid>> bidsByAuctionId =
-        events.apply(NexmarkQuery.JUST_BIDS).apply("BidByAuction", NexmarkQuery.BID_BY_AUCTION);
-
-    // Find the highest price valid bid for each closed auction.
-    return
-      // Join auctions and bids.
-      KeyedPCollectionTuple.of(NexmarkQuery.AUCTION_TAG, auctionsById)
-        .and(NexmarkQuery.BID_TAG, bidsByAuctionId)
-        .apply(CoGroupByKey.<Long>create())
-        // Filter and select.
-        .apply(name + ".Join",
-          ParDo.of(new DoFn<KV<Long, CoGbkResult>, AuctionBid>() {
-            private final Counter noAuctionCounter = Metrics.counter(name, "noAuction");
-            private final Counter underReserveCounter = Metrics.counter(name, "underReserve");
-            private final Counter noValidBidsCounter = Metrics.counter(name, "noValidBids");
-
-            @ProcessElement
-            public void processElement(ProcessContext c) {
-              Auction auction =
-                  c.element().getValue().getOnly(NexmarkQuery.AUCTION_TAG, null);
-              if (auction == null) {
-                // We have bids without a matching auction. Give up.
-                noAuctionCounter.inc();
-                return;
-              }
-              // Find the current winning bid for auction.
-              // The earliest bid with the maximum price above the reserve wins.
-              Bid bestBid = null;
-              for (Bid bid : c.element().getValue().getAll(NexmarkQuery.BID_TAG)) {
-                // Bids too late for their auction will have been
-                // filtered out by the window merge function.
-                checkState(bid.dateTime < auction.expires);
-                if (bid.price < auction.reserve) {
-                  // Bid price is below auction reserve.
-                  underReserveCounter.inc();
-                  continue;
-                }
-
-                if (bestBid == null
-                    || Bid.PRICE_THEN_DESCENDING_TIME.compare(bid, bestBid) > 0) {
-                  bestBid = bid;
-                }
-              }
-              if (bestBid == null) {
-                // We don't have any valid bids for auction.
-                noValidBidsCounter.inc();
-                return;
-              }
-              c.output(new AuctionBid(auction, bestBid));
-            }
-          }
-        ));
-  }
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/a39cb800/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/WinningBidsSimulator.java
----------------------------------------------------------------------
diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/WinningBidsSimulator.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/WinningBidsSimulator.java
deleted file mode 100644
index e7f51b7..0000000
--- a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/WinningBidsSimulator.java
+++ /dev/null
@@ -1,205 +0,0 @@
-/*
- * 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.beam.integration.nexmark;
-
-import java.util.ArrayList;
-import java.util.Iterator;
-import java.util.List;
-import java.util.Map;
-import java.util.Set;
-import java.util.TreeMap;
-import java.util.TreeSet;
-import javax.annotation.Nullable;
-
-import org.apache.beam.integration.nexmark.model.Auction;
-import org.apache.beam.integration.nexmark.model.AuctionBid;
-import org.apache.beam.integration.nexmark.model.Bid;
-import org.apache.beam.integration.nexmark.model.Event;
-import org.apache.beam.sdk.transforms.windowing.BoundedWindow;
-import org.apache.beam.sdk.values.TimestampedValue;
-import org.joda.time.Instant;
-
-/**
- * A simulator of the {@code WinningBids} query.
- */
-public class WinningBidsSimulator extends AbstractSimulator<Event, AuctionBid> {
-  /** Auctions currently still open, indexed by auction id. */
-  private final Map<Long, Auction> openAuctions;
-
-  /** The ids of auctions known to be closed. */
-  private final Set<Long> closedAuctions;
-
-  /** Current best valid bids for open auctions, indexed by auction id. */
-  private final Map<Long, Bid> bestBids;
-
-  /** Bids for auctions we havn't seen yet. */
-  private final List<Bid> bidsWithoutAuctions;
-
-  /**
-   * Timestamp of last new auction or bid event (ms since epoch).
-   */
-  private long lastTimestamp;
-
-  public WinningBidsSimulator(NexmarkConfiguration configuration) {
-    super(NexmarkUtils.standardEventIterator(configuration));
-    openAuctions = new TreeMap<>();
-    closedAuctions = new TreeSet<>();
-    bestBids = new TreeMap<>();
-    bidsWithoutAuctions = new ArrayList<>();
-    lastTimestamp = BoundedWindow.TIMESTAMP_MIN_VALUE.getMillis();
-  }
-
-  /**
-   * Try to account for {@code bid} in state. Return true if bid has now been
-   * accounted for by {@code bestBids}.
-   */
-  private boolean captureBestBid(Bid bid, boolean shouldLog) {
-    if (closedAuctions.contains(bid.auction)) {
-      // Ignore bids for known, closed auctions.
-      if (shouldLog) {
-        NexmarkUtils.info("closed auction: %s", bid);
-      }
-      return true;
-    }
-    Auction auction = openAuctions.get(bid.auction);
-    if (auction == null) {
-      // We don't have an auction for this bid yet, so can't determine if it is
-      // winning or not.
-      if (shouldLog) {
-        NexmarkUtils.info("pending auction: %s", bid);
-      }
-      return false;
-    }
-    if (bid.price < auction.reserve) {
-      // Bid price is too low.
-      if (shouldLog) {
-        NexmarkUtils.info("below reserve: %s", bid);
-      }
-      return true;
-    }
-    Bid existingBid = bestBids.get(bid.auction);
-    if (existingBid == null || Bid.PRICE_THEN_DESCENDING_TIME.compare(existingBid, bid) < 0) {
-      // We've found a (new) best bid for a known auction.
-      bestBids.put(bid.auction, bid);
-      if (shouldLog) {
-        NexmarkUtils.info("new winning bid: %s", bid);
-      }
-    } else {
-      if (shouldLog) {
-        NexmarkUtils.info("ignoring low bid: %s", bid);
-      }
-    }
-    return true;
-  }
-
-  /**
-   * Try to match bids without auctions to auctions.
-   */
-  private void flushBidsWithoutAuctions() {
-    Iterator<Bid> itr = bidsWithoutAuctions.iterator();
-    while (itr.hasNext()) {
-      Bid bid = itr.next();
-      if (captureBestBid(bid, false)) {
-        NexmarkUtils.info("bid now accounted for: %s", bid);
-        itr.remove();
-      }
-    }
-  }
-
-  /**
-   * Return the next winning bid for an expired auction relative to {@code timestamp}.
-   * Return null if no more winning bids, in which case all expired auctions will
-   * have been removed from our state. Retire auctions in order of expire time.
-   */
-  @Nullable
-  private TimestampedValue<AuctionBid> nextWinningBid(long timestamp) {
-    Map<Long, List<Long>> toBeRetired = new TreeMap<>();
-    for (Map.Entry<Long, Auction> entry : openAuctions.entrySet()) {
-      if (entry.getValue().expires <= timestamp) {
-        List<Long> idsAtTime = toBeRetired.get(entry.getValue().expires);
-        if (idsAtTime == null) {
-          idsAtTime = new ArrayList<>();
-          toBeRetired.put(entry.getValue().expires, idsAtTime);
-        }
-        idsAtTime.add(entry.getKey());
-      }
-    }
-    for (Map.Entry<Long, List<Long>> entry : toBeRetired.entrySet()) {
-      for (long id : entry.getValue()) {
-        Auction auction = openAuctions.get(id);
-        NexmarkUtils.info("retiring auction: %s", auction);
-        openAuctions.remove(id);
-        Bid bestBid = bestBids.get(id);
-        if (bestBid != null) {
-          TimestampedValue<AuctionBid> result =
-              TimestampedValue.of(new AuctionBid(auction, bestBid), new Instant(auction.expires));
-          NexmarkUtils.info("winning: %s", result);
-          return result;
-        }
-      }
-    }
-    return null;
-  }
-
-  @Override
-  protected void run() {
-    if (lastTimestamp > BoundedWindow.TIMESTAMP_MIN_VALUE.getMillis()) {
-      // We may have finally seen the auction a bid was intended for.
-      flushBidsWithoutAuctions();
-      TimestampedValue<AuctionBid> result = nextWinningBid(lastTimestamp);
-      if (result != null) {
-        addResult(result);
-        return;
-      }
-    }
-
-    TimestampedValue<Event> timestampedEvent = nextInput();
-    if (timestampedEvent == null) {
-      // No more events. Flush any still open auctions.
-      TimestampedValue<AuctionBid> result =
-          nextWinningBid(BoundedWindow.TIMESTAMP_MAX_VALUE.getMillis());
-      if (result == null) {
-        // We are done.
-        allDone();
-        return;
-      }
-      addResult(result);
-      //TODO test fails because offset of some hundreds of ms beween expect and actual
-      return;
-    }
-
-    Event event = timestampedEvent.getValue();
-    if (event.newPerson != null) {
-      // Ignore new person events.
-      return;
-    }
-
-    lastTimestamp = timestampedEvent.getTimestamp().getMillis();
-    if (event.newAuction != null) {
-      // Add this new open auction to our state.
-      openAuctions.put(event.newAuction.id, event.newAuction);
-    } else {
-      if (!captureBestBid(event.bid, true)) {
-        // We don't know what to do with this bid yet.
-        NexmarkUtils.info("bid not yet accounted for: %s", event.bid);
-        bidsWithoutAuctions.add(event.bid);
-      }
-    }
-    // Keep looking for winning bids.
-  }
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/a39cb800/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/model/AuctionBid.java
----------------------------------------------------------------------
diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/model/AuctionBid.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/model/AuctionBid.java
index 7f6b7c9..b1d9ec2 100644
--- a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/model/AuctionBid.java
+++ b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/model/AuctionBid.java
@@ -24,13 +24,12 @@ import java.io.InputStream;
 import java.io.OutputStream;
 import java.io.Serializable;
 import org.apache.beam.integration.nexmark.NexmarkUtils;
-import org.apache.beam.integration.nexmark.WinningBids;
 import org.apache.beam.sdk.coders.Coder;
 import org.apache.beam.sdk.coders.CoderException;
 import org.apache.beam.sdk.coders.CustomCoder;
 
 /**
- * Result of {@link WinningBids} transform.
+ * Result of {@link org.apache.beam.integration.nexmark.queries.WinningBids} transform.
  */
 public class AuctionBid implements KnownSize, Serializable {
   public static final Coder<AuctionBid> CODER = new CustomCoder<AuctionBid>() {

http://git-wip-us.apache.org/repos/asf/beam/blob/a39cb800/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/AbstractSimulator.java
----------------------------------------------------------------------
diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/AbstractSimulator.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/AbstractSimulator.java
new file mode 100644
index 0000000..270b5c3
--- /dev/null
+++ b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/AbstractSimulator.java
@@ -0,0 +1,211 @@
+/*
+ * 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.beam.integration.nexmark.queries;
+
+import java.util.ArrayList;
+import java.util.Iterator;
+import java.util.List;
+import javax.annotation.Nullable;
+
+import org.apache.beam.integration.nexmark.NexmarkUtils;
+import org.apache.beam.sdk.transforms.windowing.BoundedWindow;
+import org.apache.beam.sdk.values.TimestampedValue;
+import org.joda.time.Duration;
+import org.joda.time.Instant;
+
+/**
+ * Abstract base class for simulator of a query.
+ *
+ * @param <InputT> Type of input elements.
+ * @param <OutputT> Type of output elements.
+ */
+public abstract class AbstractSimulator<InputT, OutputT> {
+  /** Window size for action bucket sampling. */
+  public static final Duration WINDOW_SIZE = Duration.standardMinutes(1);
+
+  /** Input event stream we should draw from. */
+  private final Iterator<TimestampedValue<InputT>> input;
+
+  /** Set to true when no more results. */
+  private boolean isDone;
+
+  /**
+   * Results which have not yet been returned by the {@link #results} iterator.
+   */
+  private final List<TimestampedValue<OutputT>> pendingResults;
+
+  /**
+   * Current window timestamp (ms since epoch).
+   */
+  private long currentWindow;
+
+  /**
+   * Number of (possibly intermediate) results for the current window.
+   */
+  private long currentCount;
+
+  /**
+   * Result counts per window which have not yet been returned by the {@link #resultsPerWindow}
+   * iterator.
+   */
+  private final List<Long> pendingCounts;
+
+  public AbstractSimulator(Iterator<TimestampedValue<InputT>> input) {
+    this.input = input;
+    isDone = false;
+    pendingResults = new ArrayList<>();
+    currentWindow = BoundedWindow.TIMESTAMP_MIN_VALUE.getMillis();
+    currentCount = 0;
+    pendingCounts = new ArrayList<>();
+  }
+
+  /** Called by implementors of {@link #run}: Fetch the next input element. */
+  @Nullable
+  protected TimestampedValue<InputT> nextInput() {
+    if (!input.hasNext()) {
+      return null;
+    }
+    TimestampedValue<InputT> timestampedInput = input.next();
+    NexmarkUtils.info("input: %s", timestampedInput);
+    return timestampedInput;
+  }
+
+  /**
+   * Called by implementors of {@link #run}:  Capture an intermediate result, for the purpose of
+   * recording the expected activity of the query over time.
+   */
+  protected void addIntermediateResult(TimestampedValue<OutputT> result) {
+    NexmarkUtils.info("intermediate result: %s", result);
+    updateCounts(result.getTimestamp());
+  }
+
+  /**
+   * Called by implementors of {@link #run}: Capture a final result, for the purpose of checking
+   * semantic correctness.
+   */
+  protected void addResult(TimestampedValue<OutputT> result) {
+    NexmarkUtils.info("result: %s", result);
+    pendingResults.add(result);
+    updateCounts(result.getTimestamp());
+  }
+
+  /**
+   * Update window and counts.
+   */
+  private void updateCounts(Instant timestamp) {
+    long window = timestamp.getMillis() - timestamp.getMillis() % WINDOW_SIZE.getMillis();
+    if (window > currentWindow) {
+      if (currentWindow > BoundedWindow.TIMESTAMP_MIN_VALUE.getMillis()) {
+        pendingCounts.add(currentCount);
+      }
+      currentCount = 0;
+      currentWindow = window;
+    }
+    currentCount++;
+  }
+
+  /** Called by implementors of {@link #run}: Record that no more results will be emitted. */
+  protected void allDone() {
+    isDone = true;
+  }
+
+  /**
+   * Overridden by derived classes to do the next increment of work. Each call should
+   * call one or more of {@link #nextInput}, {@link #addIntermediateResult}, {@link #addResult}
+   * or {@link #allDone}. It is ok for a single call to emit more than one result via
+   * {@link #addResult}. It is ok for a single call to run the entire simulation, though
+   * this will prevent the {@link #results} and {@link #resultsPerWindow} iterators to
+   * stall.
+   */
+  protected abstract void run();
+
+  /**
+   * Return iterator over all expected timestamped results. The underlying simulator state is
+   * changed. Only one of {@link #results} or {@link #resultsPerWindow} can be called.
+   */
+  public Iterator<TimestampedValue<OutputT>> results() {
+    return new Iterator<TimestampedValue<OutputT>>() {
+      @Override
+      public boolean hasNext() {
+        while (true) {
+          if (!pendingResults.isEmpty()) {
+            return true;
+          }
+          if (isDone) {
+            return false;
+          }
+          run();
+        }
+      }
+
+      @Override
+      public TimestampedValue<OutputT> next() {
+        TimestampedValue<OutputT> result = pendingResults.get(0);
+        pendingResults.remove(0);
+        return result;
+      }
+
+      @Override
+      public void remove() {
+        throw new UnsupportedOperationException();
+      }
+    };
+  }
+
+  /**
+   * Return an iterator over the number of results per {@link #WINDOW_SIZE} period. The underlying
+   * simulator state is changed.  Only one of {@link #results} or {@link #resultsPerWindow} can be
+   * called.
+   */
+  public Iterator<Long> resultsPerWindow() {
+    return new Iterator<Long>() {
+      @Override
+      public boolean hasNext() {
+        while (true) {
+          if (!pendingCounts.isEmpty()) {
+            return true;
+          }
+          if (isDone) {
+            if (currentCount > 0) {
+              pendingCounts.add(currentCount);
+              currentCount = 0;
+              currentWindow = BoundedWindow.TIMESTAMP_MAX_VALUE.getMillis();
+              return true;
+            } else {
+              return false;
+            }
+          }
+          run();
+        }
+      }
+
+      @Override
+      public Long next() {
+        Long result = pendingCounts.get(0);
+        pendingCounts.remove(0);
+        return result;
+      }
+
+      @Override
+      public void remove() {
+        throw new UnsupportedOperationException();
+      }
+    };
+  }
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/a39cb800/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/NexmarkQuery.java
----------------------------------------------------------------------
diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/NexmarkQuery.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/NexmarkQuery.java
new file mode 100644
index 0000000..0796ce5
--- /dev/null
+++ b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/NexmarkQuery.java
@@ -0,0 +1,270 @@
+/*
+ * 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.beam.integration.nexmark.queries;
+
+import org.apache.beam.integration.nexmark.Monitor;
+import org.apache.beam.integration.nexmark.NexmarkConfiguration;
+import org.apache.beam.integration.nexmark.NexmarkUtils;
+import org.apache.beam.integration.nexmark.model.Auction;
+import org.apache.beam.integration.nexmark.model.Bid;
+import org.apache.beam.integration.nexmark.model.Event;
+import org.apache.beam.integration.nexmark.model.KnownSize;
+import org.apache.beam.integration.nexmark.model.Person;
+import org.apache.beam.sdk.metrics.Counter;
+import org.apache.beam.sdk.metrics.Metrics;
+import org.apache.beam.sdk.transforms.DoFn;
+import org.apache.beam.sdk.transforms.Filter;
+import org.apache.beam.sdk.transforms.PTransform;
+import org.apache.beam.sdk.transforms.ParDo;
+import org.apache.beam.sdk.transforms.SerializableFunction;
+import org.apache.beam.sdk.values.KV;
+import org.apache.beam.sdk.values.PCollection;
+import org.apache.beam.sdk.values.TimestampedValue;
+import org.apache.beam.sdk.values.TupleTag;
+import org.joda.time.Instant;
+
+/**
+ * Base class for the eight 'NEXMark' queries. Supplies some fragments common to
+ * multiple queries.
+ */
+public abstract class NexmarkQuery
+    extends PTransform<PCollection<Event>, PCollection<TimestampedValue<KnownSize>>> {
+  public static final TupleTag<Auction> AUCTION_TAG = new TupleTag<>("auctions");
+  public static final TupleTag<Bid> BID_TAG = new TupleTag<>("bids");
+  protected static final TupleTag<Person> PERSON_TAG = new TupleTag<>("person");
+
+  /** Predicate to detect a new person event. */
+  protected static final SerializableFunction<Event, Boolean> IS_NEW_PERSON =
+      new SerializableFunction<Event, Boolean>() {
+        @Override
+        public Boolean apply(Event event) {
+          return event.newPerson != null;
+        }
+      };
+
+  /** DoFn to convert a new person event to a person. */
+  protected static final DoFn<Event, Person> AS_PERSON = new DoFn<Event, Person>() {
+    @ProcessElement
+    public void processElement(ProcessContext c) {
+      c.output(c.element().newPerson);
+    }
+  };
+
+  /** Predicate to detect a new auction event. */
+  protected static final SerializableFunction<Event, Boolean> IS_NEW_AUCTION =
+      new SerializableFunction<Event, Boolean>() {
+        @Override
+        public Boolean apply(Event event) {
+          return event.newAuction != null;
+        }
+      };
+
+  /** DoFn to convert a new auction event to an auction. */
+  protected static final DoFn<Event, Auction> AS_AUCTION = new DoFn<Event, Auction>() {
+    @ProcessElement
+    public void processElement(ProcessContext c) {
+      c.output(c.element().newAuction);
+    }
+  };
+
+  /** Predicate to detect a new bid event. */
+  protected static final SerializableFunction<Event, Boolean> IS_BID =
+      new SerializableFunction<Event, Boolean>() {
+        @Override
+        public Boolean apply(Event event) {
+          return event.bid != null;
+        }
+      };
+
+  /** DoFn to convert a bid event to a bid. */
+  protected static final DoFn<Event, Bid> AS_BID = new DoFn<Event, Bid>() {
+    @ProcessElement
+    public void processElement(ProcessContext c) {
+      c.output(c.element().bid);
+    }
+  };
+
+  /** Transform to key each person by their id. */
+  protected static final ParDo.SingleOutput<Person, KV<Long, Person>> PERSON_BY_ID =
+      ParDo.of(new DoFn<Person, KV<Long, Person>>() {
+             @ProcessElement
+             public void processElement(ProcessContext c) {
+               c.output(KV.of(c.element().id, c.element()));
+             }
+           });
+
+  /** Transform to key each auction by its id. */
+  protected static final ParDo.SingleOutput<Auction, KV<Long, Auction>> AUCTION_BY_ID =
+      ParDo.of(new DoFn<Auction, KV<Long, Auction>>() {
+             @ProcessElement
+             public void processElement(ProcessContext c) {
+               c.output(KV.of(c.element().id, c.element()));
+             }
+           });
+
+  /** Transform to key each auction by its seller id. */
+  protected static final ParDo.SingleOutput<Auction, KV<Long, Auction>> AUCTION_BY_SELLER =
+      ParDo.of(new DoFn<Auction, KV<Long, Auction>>() {
+             @ProcessElement
+             public void processElement(ProcessContext c) {
+               c.output(KV.of(c.element().seller, c.element()));
+             }
+           });
+
+  /** Transform to key each bid by it's auction id. */
+  protected static final ParDo.SingleOutput<Bid, KV<Long, Bid>> BID_BY_AUCTION =
+      ParDo.of(new DoFn<Bid, KV<Long, Bid>>() {
+             @ProcessElement
+             public void processElement(ProcessContext c) {
+               c.output(KV.of(c.element().auction, c.element()));
+             }
+           });
+
+  /** Transform to project the auction id from each bid. */
+  protected static final ParDo.SingleOutput<Bid, Long> BID_TO_AUCTION =
+      ParDo.of(new DoFn<Bid, Long>() {
+             @ProcessElement
+             public void processElement(ProcessContext c) {
+               c.output(c.element().auction);
+             }
+           });
+
+  /** Transform to project the price from each bid. */
+  protected static final ParDo.SingleOutput<Bid, Long> BID_TO_PRICE =
+      ParDo.of(new DoFn<Bid, Long>() {
+             @ProcessElement
+             public void processElement(ProcessContext c) {
+               c.output(c.element().price);
+             }
+           });
+
+  /** Transform to emit each event with the timestamp embedded within it. */
+  public static final ParDo.SingleOutput<Event, Event> EVENT_TIMESTAMP_FROM_DATA =
+      ParDo.of(new DoFn<Event, Event>() {
+             @ProcessElement
+             public void processElement(ProcessContext c) {
+               Event e = c.element();
+               if (e.bid != null) {
+                 c.outputWithTimestamp(e, new Instant(e.bid.dateTime));
+               } else if (e.newPerson != null) {
+                 c.outputWithTimestamp(e, new Instant(e.newPerson.dateTime));
+               } else if (e.newAuction != null) {
+                 c.outputWithTimestamp(e, new Instant(e.newAuction.dateTime));
+               }
+             }
+           });
+
+  /**
+   * Transform to filter for just the new auction events.
+   */
+  public static final PTransform<PCollection<Event>, PCollection<Auction>> JUST_NEW_AUCTIONS =
+      new PTransform<PCollection<Event>, PCollection<Auction>>("justNewAuctions") {
+        @Override
+        public PCollection<Auction> expand(PCollection<Event> input) {
+          return input.apply("IsNewAuction", Filter.by(IS_NEW_AUCTION))
+                      .apply("AsAuction", ParDo.of(AS_AUCTION));
+        }
+      };
+
+  /**
+   * Transform to filter for just the new person events.
+   */
+  public static final PTransform<PCollection<Event>, PCollection<Person>> JUST_NEW_PERSONS =
+      new PTransform<PCollection<Event>, PCollection<Person>>("justNewPersons") {
+        @Override
+        public PCollection<Person> expand(PCollection<Event> input) {
+          return input.apply("IsNewPerson", Filter.by(IS_NEW_PERSON))
+                      .apply("AsPerson", ParDo.of(AS_PERSON));
+        }
+      };
+
+  /**
+   * Transform to filter for just the bid events.
+   */
+  public static final PTransform<PCollection<Event>, PCollection<Bid>> JUST_BIDS =
+      new PTransform<PCollection<Event>, PCollection<Bid>>("justBids") {
+        @Override
+        public PCollection<Bid> expand(PCollection<Event> input) {
+          return input.apply("IsBid", Filter.by(IS_BID))
+                      .apply("AsBid", ParDo.of(AS_BID));
+        }
+      };
+
+  protected final NexmarkConfiguration configuration;
+  public final Monitor<Event> eventMonitor;
+  public final Monitor<KnownSize> resultMonitor;
+  public final Monitor<Event> endOfStreamMonitor;
+  protected final Counter fatalCounter;
+
+  protected NexmarkQuery(NexmarkConfiguration configuration, String name) {
+    super(name);
+    this.configuration = configuration;
+    if (configuration.debug) {
+      eventMonitor = new Monitor<>(name + ".Events", "event");
+      resultMonitor = new Monitor<>(name + ".Results", "result");
+      endOfStreamMonitor = new Monitor<>(name + ".EndOfStream", "end");
+      fatalCounter = Metrics.counter(name , "fatal");
+    } else {
+      eventMonitor = null;
+      resultMonitor = null;
+      endOfStreamMonitor = null;
+      fatalCounter = null;
+    }
+  }
+
+  /**
+   * Implement the actual query. All we know about the result is it has a known encoded size.
+   */
+  protected abstract PCollection<KnownSize> applyPrim(PCollection<Event> events);
+
+  @Override
+  public PCollection<TimestampedValue<KnownSize>> expand(PCollection<Event> events) {
+
+    if (configuration.debug) {
+      events =
+          events
+              // Monitor events as they go by.
+              .apply(name + ".Monitor", eventMonitor.getTransform())
+              // Count each type of event.
+              .apply(name + ".Snoop", NexmarkUtils.snoop(name));
+    }
+
+    if (configuration.cpuDelayMs > 0) {
+      // Slow down by pegging one core at 100%.
+      events = events.apply(name + ".CpuDelay",
+              NexmarkUtils.<Event>cpuDelay(name, configuration.cpuDelayMs));
+    }
+
+    if (configuration.diskBusyBytes > 0) {
+      // Slow down by forcing bytes to durable store.
+      events = events.apply(name + ".DiskBusy",
+              NexmarkUtils.<Event>diskBusy(name, configuration.diskBusyBytes));
+    }
+
+    // Run the query.
+    PCollection<KnownSize> queryResults = applyPrim(events);
+
+    if (configuration.debug) {
+      // Monitor results as they go by.
+      queryResults = queryResults.apply(name + ".Debug", resultMonitor.getTransform());
+    }
+
+    // Timestamp the query results.
+    return queryResults.apply(name + ".Stamp", NexmarkUtils.<KnownSize>stamp(name));
+  }
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/a39cb800/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/NexmarkQueryModel.java
----------------------------------------------------------------------
diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/NexmarkQueryModel.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/NexmarkQueryModel.java
new file mode 100644
index 0000000..1ad9099
--- /dev/null
+++ b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/NexmarkQueryModel.java
@@ -0,0 +1,123 @@
+/*
+ * 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.beam.integration.nexmark.queries;
+
+import java.io.Serializable;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Set;
+
+import org.apache.beam.integration.nexmark.NexmarkConfiguration;
+import org.apache.beam.integration.nexmark.model.KnownSize;
+import org.apache.beam.sdk.transforms.SerializableFunction;
+import org.apache.beam.sdk.values.TimestampedValue;
+
+
+import org.hamcrest.core.IsEqual;
+import org.joda.time.Duration;
+import org.joda.time.Instant;
+import org.junit.Assert;
+
+/**
+ * Base class for models of the eight NEXMark queries. Provides an assertion function which can be
+ * applied against the actual query results to check their consistency with the model.
+ */
+public abstract class NexmarkQueryModel implements Serializable {
+  public final NexmarkConfiguration configuration;
+
+  public NexmarkQueryModel(NexmarkConfiguration configuration) {
+    this.configuration = configuration;
+  }
+
+  /**
+   * Return the start of the most recent window of {@code size} and {@code period} which ends
+   * strictly before {@code timestamp}.
+   */
+  public static Instant windowStart(Duration size, Duration period, Instant timestamp) {
+    long ts = timestamp.getMillis();
+    long p = period.getMillis();
+    long lim = ts - ts % p;
+    long s = size.getMillis();
+    return new Instant(lim - s);
+  }
+
+  /** Convert {@code itr} to strings capturing values, timestamps and order. */
+  protected static <T> List<String> toValueTimestampOrder(Iterator<TimestampedValue<T>> itr) {
+    List<String> strings = new ArrayList<>();
+    while (itr.hasNext()) {
+      strings.add(itr.next().toString());
+    }
+    return strings;
+  }
+
+  /** Convert {@code itr} to strings capturing values and order. */
+  protected static <T> List<String> toValueOrder(Iterator<TimestampedValue<T>> itr) {
+    List<String> strings = new ArrayList<>();
+    while (itr.hasNext()) {
+      strings.add(itr.next().getValue().toString());
+    }
+    return strings;
+  }
+
+  /** Convert {@code itr} to strings capturing values only. */
+  protected static <T> Set<String> toValue(Iterator<TimestampedValue<T>> itr) {
+    Set<String> strings = new HashSet<>();
+    while (itr.hasNext()) {
+      strings.add(itr.next().getValue().toString());
+    }
+    return strings;
+  }
+
+  /** Return simulator for query. */
+  public abstract AbstractSimulator<?, ?> simulator();
+
+  /** Return sub-sequence of results which are significant for model. */
+  protected Iterable<TimestampedValue<KnownSize>> relevantResults(
+      Iterable<TimestampedValue<KnownSize>> results) {
+    return results;
+  }
+
+  /**
+   * Convert iterator of elements to collection of strings to use when testing coherence of model
+   * against actual query results.
+   */
+  protected abstract <T> Collection<String> toCollection(Iterator<TimestampedValue<T>> itr);
+
+  /** Return assertion to use on results of pipeline for this query. */
+  public SerializableFunction<Iterable<TimestampedValue<KnownSize>>, Void> assertionFor() {
+    final Collection<String> expectedStrings = toCollection(simulator().results());
+    final String[] expectedStringsArray =
+      expectedStrings.toArray(new String[expectedStrings.size()]);
+
+    return new SerializableFunction<Iterable<TimestampedValue<KnownSize>>, Void>() {
+      @Override
+      public Void apply(Iterable<TimestampedValue<KnownSize>> actual) {
+      Collection<String> actualStrings = toCollection(relevantResults(actual).iterator());
+        Assert.assertThat("wrong pipeline output", actualStrings,
+          IsEqual.equalTo(expectedStrings));
+//compare without order
+//      Assert.assertThat("wrong pipeline output", actualStrings,
+//        IsIterableContainingInAnyOrder.containsInAnyOrder(expectedStringsArray));
+        return null;
+      }
+    };
+  }
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/a39cb800/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query0.java
----------------------------------------------------------------------
diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query0.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query0.java
index 84696c4..00a49a8 100644
--- a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query0.java
+++ b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query0.java
@@ -22,7 +22,6 @@ import java.io.ByteArrayOutputStream;
 import java.io.IOException;
 
 import org.apache.beam.integration.nexmark.NexmarkConfiguration;
-import org.apache.beam.integration.nexmark.NexmarkQuery;
 import org.apache.beam.integration.nexmark.NexmarkUtils;
 import org.apache.beam.integration.nexmark.model.Event;
 import org.apache.beam.integration.nexmark.model.KnownSize;

http://git-wip-us.apache.org/repos/asf/beam/blob/a39cb800/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query0Model.java
----------------------------------------------------------------------
diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query0Model.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query0Model.java
index 991b1d4..6fb6613 100644
--- a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query0Model.java
+++ b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query0Model.java
@@ -20,9 +20,7 @@ package org.apache.beam.integration.nexmark.queries;
 import java.util.Collection;
 import java.util.Iterator;
 
-import org.apache.beam.integration.nexmark.AbstractSimulator;
 import org.apache.beam.integration.nexmark.NexmarkConfiguration;
-import org.apache.beam.integration.nexmark.NexmarkQueryModel;
 import org.apache.beam.integration.nexmark.NexmarkUtils;
 import org.apache.beam.integration.nexmark.model.Event;
 import org.apache.beam.sdk.values.TimestampedValue;
@@ -56,7 +54,7 @@ public class Query0Model extends NexmarkQueryModel {
   }
 
   @Override
-  protected AbstractSimulator<?, ?> simulator() {
+  public AbstractSimulator<?, ?> simulator() {
     return new Simulator(configuration);
   }
 

http://git-wip-us.apache.org/repos/asf/beam/blob/a39cb800/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query1.java
----------------------------------------------------------------------
diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query1.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query1.java
index 0be77ce..8d90b70 100644
--- a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query1.java
+++ b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query1.java
@@ -18,7 +18,6 @@
 package org.apache.beam.integration.nexmark.queries;
 
 import org.apache.beam.integration.nexmark.NexmarkConfiguration;
-import org.apache.beam.integration.nexmark.NexmarkQuery;
 import org.apache.beam.integration.nexmark.NexmarkUtils;
 import org.apache.beam.integration.nexmark.model.Bid;
 import org.apache.beam.integration.nexmark.model.Event;

http://git-wip-us.apache.org/repos/asf/beam/blob/a39cb800/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query10.java
----------------------------------------------------------------------
diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query10.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query10.java
index d9b3557..c919691 100644
--- a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query10.java
+++ b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query10.java
@@ -28,7 +28,6 @@ import java.nio.channels.WritableByteChannel;
 import java.util.concurrent.ThreadLocalRandom;
 import javax.annotation.Nullable;
 import org.apache.beam.integration.nexmark.NexmarkConfiguration;
-import org.apache.beam.integration.nexmark.NexmarkQuery;
 import org.apache.beam.integration.nexmark.NexmarkUtils;
 import org.apache.beam.integration.nexmark.model.Done;
 import org.apache.beam.integration.nexmark.model.Event;

http://git-wip-us.apache.org/repos/asf/beam/blob/a39cb800/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query11.java
----------------------------------------------------------------------
diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query11.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query11.java
index a8a61ae..fd936a9 100644
--- a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query11.java
+++ b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query11.java
@@ -18,7 +18,6 @@
 package org.apache.beam.integration.nexmark.queries;
 
 import org.apache.beam.integration.nexmark.NexmarkConfiguration;
-import org.apache.beam.integration.nexmark.NexmarkQuery;
 import org.apache.beam.integration.nexmark.NexmarkUtils;
 import org.apache.beam.integration.nexmark.model.Bid;
 import org.apache.beam.integration.nexmark.model.BidsPerSession;

http://git-wip-us.apache.org/repos/asf/beam/blob/a39cb800/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query12.java
----------------------------------------------------------------------
diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query12.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query12.java
index a5db504..20f45fb 100644
--- a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query12.java
+++ b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query12.java
@@ -18,7 +18,6 @@
 package org.apache.beam.integration.nexmark.queries;
 
 import org.apache.beam.integration.nexmark.NexmarkConfiguration;
-import org.apache.beam.integration.nexmark.NexmarkQuery;
 import org.apache.beam.integration.nexmark.NexmarkUtils;
 import org.apache.beam.integration.nexmark.model.Bid;
 import org.apache.beam.integration.nexmark.model.BidsPerSession;

http://git-wip-us.apache.org/repos/asf/beam/blob/a39cb800/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query1Model.java
----------------------------------------------------------------------
diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query1Model.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query1Model.java
index 58037d3..0388687 100644
--- a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query1Model.java
+++ b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query1Model.java
@@ -21,9 +21,7 @@ import java.io.Serializable;
 import java.util.Collection;
 import java.util.Iterator;
 
-import org.apache.beam.integration.nexmark.AbstractSimulator;
 import org.apache.beam.integration.nexmark.NexmarkConfiguration;
-import org.apache.beam.integration.nexmark.NexmarkQueryModel;
 import org.apache.beam.integration.nexmark.NexmarkUtils;
 import org.apache.beam.integration.nexmark.model.Bid;
 import org.apache.beam.integration.nexmark.model.Event;

http://git-wip-us.apache.org/repos/asf/beam/blob/a39cb800/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query2.java
----------------------------------------------------------------------
diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query2.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query2.java
index 4c8f878..a365b97 100644
--- a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query2.java
+++ b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query2.java
@@ -18,7 +18,6 @@
 package org.apache.beam.integration.nexmark.queries;
 
 import org.apache.beam.integration.nexmark.NexmarkConfiguration;
-import org.apache.beam.integration.nexmark.NexmarkQuery;
 import org.apache.beam.integration.nexmark.NexmarkUtils;
 import org.apache.beam.integration.nexmark.model.AuctionPrice;
 import org.apache.beam.integration.nexmark.model.Bid;

http://git-wip-us.apache.org/repos/asf/beam/blob/a39cb800/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query2Model.java
----------------------------------------------------------------------
diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query2Model.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query2Model.java
index f578e4c..e00992f 100644
--- a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query2Model.java
+++ b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query2Model.java
@@ -21,9 +21,7 @@ import java.io.Serializable;
 import java.util.Collection;
 import java.util.Iterator;
 
-import org.apache.beam.integration.nexmark.AbstractSimulator;
 import org.apache.beam.integration.nexmark.NexmarkConfiguration;
-import org.apache.beam.integration.nexmark.NexmarkQueryModel;
 import org.apache.beam.integration.nexmark.NexmarkUtils;
 import org.apache.beam.integration.nexmark.model.AuctionPrice;
 import org.apache.beam.integration.nexmark.model.Bid;


[20/55] [abbrv] beam git commit: Fix Spark streaming termination via waitUntilFinish and timeout config

Posted by ie...@apache.org.
Fix Spark streaming termination via waitUntilFinish and timeout config

issue #39


Project: http://git-wip-us.apache.org/repos/asf/beam/repo
Commit: http://git-wip-us.apache.org/repos/asf/beam/commit/e10d5783
Tree: http://git-wip-us.apache.org/repos/asf/beam/tree/e10d5783
Diff: http://git-wip-us.apache.org/repos/asf/beam/diff/e10d5783

Branch: refs/heads/master
Commit: e10d5783d8c8ed32008e29d99d5a4b1dd3e408a6
Parents: 8098bb1
Author: Etienne Chauchot <ec...@gmail.com>
Authored: Fri Apr 14 17:13:59 2017 +0200
Committer: Ismaël Mejía <ie...@gmail.com>
Committed: Wed Aug 23 19:07:27 2017 +0200

----------------------------------------------------------------------
 .../nexmark/NexmarkConfiguration.java           | 61 +++++++++++++++++---
 .../integration/nexmark/NexmarkOptions.java     |  7 +++
 .../beam/integration/nexmark/NexmarkRunner.java |  2 +-
 3 files changed, 62 insertions(+), 8 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/beam/blob/e10d5783/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkConfiguration.java
----------------------------------------------------------------------
diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkConfiguration.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkConfiguration.java
index d6cd808..1da08b4 100644
--- a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkConfiguration.java
+++ b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkConfiguration.java
@@ -106,6 +106,12 @@ public class NexmarkConfiguration implements Serializable {
   public int preloadSeconds = 0;
 
   /**
+   * Timeout for stream pipelines to stop in seconds.
+   */
+  @JsonProperty
+  public int streamTimeout = 240;
+
+  /**
    * If true, and in streaming mode, generate events only when they are due according to their
    * timestamp.
    */
@@ -275,6 +281,9 @@ public class NexmarkConfiguration implements Serializable {
     if (options.getPreloadSeconds() != null) {
       preloadSeconds = options.getPreloadSeconds();
     }
+    if (options.getStreamTimeout() != null) {
+      streamTimeout = options.getStreamTimeout();
+    }
     if (options.getIsRateLimited() != null) {
       isRateLimited = options.getIsRateLimited();
     }
@@ -368,6 +377,7 @@ public class NexmarkConfiguration implements Serializable {
     result.rateUnit = rateUnit;
     result.ratePeriodSec = ratePeriodSec;
     result.preloadSeconds = preloadSeconds;
+    result.streamTimeout = streamTimeout;
     result.isRateLimited = isRateLimited;
     result.useWallclockEventTime = useWallclockEventTime;
     result.avgPersonByteSize = avgPersonByteSize;
@@ -436,6 +446,9 @@ public class NexmarkConfiguration implements Serializable {
     if (preloadSeconds != DEFAULT.preloadSeconds) {
       sb.append(String.format("; preloadSeconds:%d", preloadSeconds));
     }
+    if (streamTimeout != DEFAULT.streamTimeout) {
+      sb.append(String.format("; streamTimeout:%d", streamTimeout));
+    }
     if (isRateLimited != DEFAULT.isRateLimited) {
       sb.append(String.format("; isRateLimited:%s", isRateLimited));
     }
@@ -536,13 +549,44 @@ public class NexmarkConfiguration implements Serializable {
 
   @Override
   public int hashCode() {
-    return Objects.hash(debug, query, sourceType, sinkType, pubSubMode,
-        numEvents, numEventGenerators, rateShape, firstEventRate, nextEventRate, rateUnit,
-        ratePeriodSec, preloadSeconds, isRateLimited, useWallclockEventTime, avgPersonByteSize,
-        avgAuctionByteSize, avgBidByteSize, hotAuctionRatio, hotSellersRatio, hotBiddersRatio,
-        windowSizeSec, windowPeriodSec, watermarkHoldbackSec, numInFlightAuctions, numActivePeople,
-        coderStrategy, cpuDelayMs, diskBusyBytes, auctionSkip, fanout, maxAuctionsWaitingTime,
-        occasionalDelaySec, probDelayedEvent, maxLogEvents, usePubsubPublishTime,
+    return Objects.hash(
+        debug,
+        query,
+        sourceType,
+        sinkType,
+        pubSubMode,
+        numEvents,
+        numEventGenerators,
+        rateShape,
+        firstEventRate,
+        nextEventRate,
+        rateUnit,
+        ratePeriodSec,
+        preloadSeconds,
+        streamTimeout,
+        isRateLimited,
+        useWallclockEventTime,
+        avgPersonByteSize,
+        avgAuctionByteSize,
+        avgBidByteSize,
+        hotAuctionRatio,
+        hotSellersRatio,
+        hotBiddersRatio,
+        windowSizeSec,
+        windowPeriodSec,
+        watermarkHoldbackSec,
+        numInFlightAuctions,
+        numActivePeople,
+        coderStrategy,
+        cpuDelayMs,
+        diskBusyBytes,
+        auctionSkip,
+        fanout,
+        maxAuctionsWaitingTime,
+        occasionalDelaySec,
+        probDelayedEvent,
+        maxLogEvents,
+        usePubsubPublishTime,
         outOfOrderGroupSize);
   }
 
@@ -630,6 +674,9 @@ public class NexmarkConfiguration implements Serializable {
     if (preloadSeconds != other.preloadSeconds) {
       return false;
     }
+    if (streamTimeout != other.streamTimeout) {
+      return false;
+    }
     if (Double.doubleToLongBits(probDelayedEvent)
         != Double.doubleToLongBits(other.probDelayedEvent)) {
       return false;

http://git-wip-us.apache.org/repos/asf/beam/blob/e10d5783/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkOptions.java
----------------------------------------------------------------------
diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkOptions.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkOptions.java
index e39f0a4..5d093ae 100644
--- a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkOptions.java
+++ b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkOptions.java
@@ -110,6 +110,13 @@ public interface NexmarkOptions extends PubsubOptions {
 
   void setPreloadSeconds(Integer preloadSeconds);
 
+  @Description(
+      "Time in seconds to wait in pipelineResult.waitUntilFinish(), useful in streaming mode")
+  @Nullable
+  Integer getStreamTimeout();
+
+  void setStreamTimeout(Integer preloadSeconds);
+
   @Description("Number of unbounded sources to create events.")
   @Nullable
   Integer getNumEventGenerators();

http://git-wip-us.apache.org/repos/asf/beam/blob/e10d5783/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkRunner.java
----------------------------------------------------------------------
diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkRunner.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkRunner.java
index 3a0452f..ef5f0e2 100644
--- a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkRunner.java
+++ b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkRunner.java
@@ -1230,7 +1230,7 @@ public abstract class NexmarkRunner<OptionT extends NexmarkOptions> {
         waitForPublisherPreload();
       }
       mainResult = p.run();
-      mainResult.waitUntilFinish();
+      mainResult.waitUntilFinish(Duration.standardSeconds(configuration.streamTimeout));
       return monitor(query);
     } finally {
       //


[25/55] [abbrv] beam git commit: Change Nexmark pom structure to mirror other modules on Beam

Posted by ie...@apache.org.
Change Nexmark pom structure to mirror other modules on Beam

Fix compile after PubsubIO refactor


Project: http://git-wip-us.apache.org/repos/asf/beam/repo
Commit: http://git-wip-us.apache.org/repos/asf/beam/commit/8098bb1d
Tree: http://git-wip-us.apache.org/repos/asf/beam/tree/8098bb1d
Diff: http://git-wip-us.apache.org/repos/asf/beam/diff/8098bb1d

Branch: refs/heads/master
Commit: 8098bb1dbcc22153960d9b4483327e2977641148
Parents: 7ef49dc
Author: Ismaël Mejía <ie...@apache.org>
Authored: Thu Apr 13 10:47:54 2017 +0200
Committer: Ismaël Mejía <ie...@gmail.com>
Committed: Wed Aug 23 19:07:27 2017 +0200

----------------------------------------------------------------------
 integration/java/nexmark/pom.xml                |  48 +-
 .../beam/integration/nexmark/NexmarkRunner.java |   2 +-
 .../integration/nexmark/io/PubsubClient.java    | 543 +++++++++++++++++++
 .../integration/nexmark/io/PubsubHelper.java    |   2 -
 .../nexmark/io/PubsubJsonClient.java            | 318 +++++++++++
 .../nexmark/io/PubsubTestClient.java            | 436 +++++++++++++++
 integration/java/pom.xml                        |  37 ++
 integration/pom.xml                             |  37 ++
 pom.xml                                         |   2 +-
 9 files changed, 1401 insertions(+), 24 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/beam/blob/8098bb1d/integration/java/nexmark/pom.xml
----------------------------------------------------------------------
diff --git a/integration/java/nexmark/pom.xml b/integration/java/nexmark/pom.xml
index 7cd7d39..67d6117 100644
--- a/integration/java/nexmark/pom.xml
+++ b/integration/java/nexmark/pom.xml
@@ -22,19 +22,17 @@
 
   <parent>
     <groupId>org.apache.beam</groupId>
-    <artifactId>beam-parent</artifactId>
+    <artifactId>beam-integration-java-parent</artifactId>
     <version>0.7.0-SNAPSHOT</version>
-    <relativePath>../../../pom.xml</relativePath>
+    <relativePath>../pom.xml</relativePath>
   </parent>
 
-  <artifactId>beam-integration-java</artifactId>
+  <artifactId>beam-integration-java-nexmark</artifactId>
   <name>Apache Beam :: Integration Tests :: Java :: Nexmark</name>
 
   <packaging>jar</packaging>
 
   <properties>
-    <project.build.sourceEncoding>UTF-8</project.build.sourceEncoding>
-    <project.reporting.outputEncoding>UTF-8</project.reporting.outputEncoding>
     <flink.version>1.2.0</flink.version>
     <spark.version>1.6.3</spark.version>
     <apex.codehaus.jackson.version>1.9.3</apex.codehaus.jackson.version>
@@ -253,11 +251,36 @@
     </dependency>
 
     <dependency>
+      <groupId>com.google.apis</groupId>
+      <artifactId>google-api-services-pubsub</artifactId>
+    </dependency>
+
+    <dependency>
+      <groupId>com.google.auth</groupId>
+      <artifactId>google-auth-library-credentials</artifactId>
+    </dependency>
+
+    <dependency>
+      <groupId>com.google.auth</groupId>
+      <artifactId>google-auth-library-oauth2-http</artifactId>
+    </dependency>
+
+    <dependency>
       <groupId>com.google.cloud.bigdataoss</groupId>
       <artifactId>gcsio</artifactId>
     </dependency>
 
     <dependency>
+      <groupId>com.google.cloud.bigdataoss</groupId>
+      <artifactId>util</artifactId>
+    </dependency>
+
+    <dependency>
+      <groupId>com.google.http-client</groupId>
+      <artifactId>google-http-client</artifactId>
+    </dependency>
+
+    <dependency>
       <groupId>com.fasterxml.jackson.core</groupId>
       <artifactId>jackson-annotations</artifactId>
     </dependency>
@@ -288,13 +311,6 @@
       <scope>compile</scope>
     </dependency>
 
-    <!--<dependency>-->
-      <!--<groupId>org.slf4j</groupId>-->
-      <!--<artifactId>slf4j-jdk14</artifactId>-->
-      <!--&lt;!&ndash; When loaded at runtime this will wire up slf4j to the JUL backend &ndash;&gt;-->
-      <!--<scope>runtime</scope>-->
-    <!--</dependency>-->
-
     <dependency>
       <groupId>com.google.code.findbugs</groupId>
       <artifactId>jsr305</artifactId>
@@ -305,13 +321,5 @@
       <artifactId>junit</artifactId>
       <scope>compile</scope>
     </dependency>
-
-    <!--<dependency>-->
-      <!--<groupId>io.netty</groupId>-->
-      <!--<artifactId>netty-tcnative-boringssl-static</artifactId>-->
-      <!--<version>1.1.33.Fork13</version>-->
-      <!--<classifier>${os.detected.classifier}</classifier>-->
-      <!--<scope>runtime</scope>-->
-    <!--</dependency>-->
   </dependencies>
 </project>

http://git-wip-us.apache.org/repos/asf/beam/blob/8098bb1d/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkRunner.java
----------------------------------------------------------------------
diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkRunner.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkRunner.java
index df1000a..3a0452f 100644
--- a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkRunner.java
+++ b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkRunner.java
@@ -67,9 +67,9 @@ import org.apache.beam.sdk.AggregatorRetrievalException;
 import org.apache.beam.sdk.Pipeline;
 import org.apache.beam.sdk.PipelineResult;
 import org.apache.beam.sdk.io.AvroIO;
-import org.apache.beam.sdk.io.PubsubIO;
 import org.apache.beam.sdk.io.TextIO;
 import org.apache.beam.sdk.io.gcp.bigquery.BigQueryIO;
+import org.apache.beam.sdk.io.gcp.pubsub.PubsubIO;
 import org.apache.beam.sdk.testing.PAssert;
 import org.apache.beam.sdk.transforms.Aggregator;
 import org.apache.beam.sdk.transforms.DoFn;

http://git-wip-us.apache.org/repos/asf/beam/blob/8098bb1d/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/io/PubsubClient.java
----------------------------------------------------------------------
diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/io/PubsubClient.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/io/PubsubClient.java
new file mode 100644
index 0000000..687aa35
--- /dev/null
+++ b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/io/PubsubClient.java
@@ -0,0 +1,543 @@
+/*
+ * 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.beam.integration.nexmark.io;
+
+import static com.google.common.base.Preconditions.checkArgument;
+import static com.google.common.base.Preconditions.checkState;
+
+import com.google.api.client.util.DateTime;
+import com.google.common.base.Objects;
+import com.google.common.base.Strings;
+import java.io.Closeable;
+import java.io.IOException;
+import java.io.Serializable;
+import java.util.Arrays;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.ThreadLocalRandom;
+import javax.annotation.Nullable;
+import org.apache.beam.sdk.options.PubsubOptions;
+
+/**
+ * An (abstract) helper class for talking to Pubsub via an underlying transport.
+ */
+abstract class PubsubClient implements Closeable {
+  /**
+   * Factory for creating clients.
+   */
+  public interface PubsubClientFactory extends Serializable {
+    /**
+     * Construct a new Pubsub client. It should be closed via {@link #close} in order
+     * to ensure tidy cleanup of underlying netty resources (or use the try-with-resources
+     * construct). Uses {@code options} to derive pubsub endpoints and application credentials.
+     * If non-{@literal null}, use {@code timestampLabel} and {@code idLabel} to store custom
+     * timestamps/ids within message metadata.
+     */
+    PubsubClient newClient(@Nullable String timestampLabel,
+        @Nullable String idLabel, PubsubOptions options) throws IOException;
+
+    /**
+     * Return the display name for this factory. Eg "Json", "gRPC".
+     */
+    String getKind();
+  }
+
+  /**
+   * Return timestamp as ms-since-unix-epoch corresponding to {@code timestamp}.
+   * Return {@literal null} if no timestamp could be found. Throw {@link IllegalArgumentException}
+   * if timestamp cannot be recognized.
+   */
+  @Nullable
+  private static Long asMsSinceEpoch(@Nullable String timestamp) {
+    if (Strings.isNullOrEmpty(timestamp)) {
+      return null;
+    }
+    try {
+      // Try parsing as milliseconds since epoch. Note there is no way to parse a
+      // string in RFC 3339 format here.
+      // Expected IllegalArgumentException if parsing fails; we use that to fall back
+      // to RFC 3339.
+      return Long.parseLong(timestamp);
+    } catch (IllegalArgumentException e1) {
+      // Try parsing as RFC3339 string. DateTime.parseRfc3339 will throw an
+      // IllegalArgumentException if parsing fails, and the caller should handle.
+      return DateTime.parseRfc3339(timestamp).getValue();
+    }
+  }
+
+  /**
+   * Return the timestamp (in ms since unix epoch) to use for a Pubsub message with {@code
+   * attributes} and {@code pubsubTimestamp}.
+   *
+   * <p>If {@code timestampLabel} is non-{@literal null} then the message attributes must contain
+   * that label, and the value of that label will be taken as the timestamp.
+   * Otherwise the timestamp will be taken from the Pubsub publish timestamp {@code
+   * pubsubTimestamp}.
+   *
+   * @throws IllegalArgumentException if the timestamp cannot be recognized as a ms-since-unix-epoch
+   * or RFC3339 time.
+   */
+  protected static long extractTimestamp(
+      @Nullable String timestampLabel,
+      @Nullable String pubsubTimestamp,
+      @Nullable Map<String, String> attributes) {
+    Long timestampMsSinceEpoch;
+    if (Strings.isNullOrEmpty(timestampLabel)) {
+      timestampMsSinceEpoch = asMsSinceEpoch(pubsubTimestamp);
+      checkArgument(timestampMsSinceEpoch != null,
+                    "Cannot interpret PubSub publish timestamp: %s",
+                    pubsubTimestamp);
+    } else {
+      String value = attributes == null ? null : attributes.get(timestampLabel);
+      checkArgument(value != null,
+                    "PubSub message is missing a value for timestamp label %s",
+                    timestampLabel);
+      timestampMsSinceEpoch = asMsSinceEpoch(value);
+      checkArgument(timestampMsSinceEpoch != null,
+                    "Cannot interpret value of label %s as timestamp: %s",
+                    timestampLabel, value);
+    }
+    return timestampMsSinceEpoch;
+  }
+
+  /**
+   * Path representing a cloud project id.
+   */
+  static class ProjectPath implements Serializable {
+    private final String projectId;
+
+    /**
+     * Creates a {@link ProjectPath} from a {@link String} representation, which
+     * must be of the form {@code "projects/" + projectId}.
+     */
+    ProjectPath(String path) {
+      String[] splits = path.split("/");
+      checkArgument(
+          splits.length == 2 && splits[0].equals("projects"),
+          "Malformed project path \"%s\": must be of the form \"projects/\" + <project id>",
+          path);
+      this.projectId = splits[1];
+    }
+
+    public String getPath() {
+      return String.format("projects/%s", projectId);
+    }
+
+    public String getId() {
+      return projectId;
+    }
+
+    @Override
+    public boolean equals(Object o) {
+      if (this == o) {
+        return true;
+      }
+      if (o == null || getClass() != o.getClass()) {
+        return false;
+      }
+
+      ProjectPath that = (ProjectPath) o;
+
+      return projectId.equals(that.projectId);
+    }
+
+    @Override
+    public int hashCode() {
+      return projectId.hashCode();
+    }
+
+    @Override
+    public String toString() {
+      return getPath();
+    }
+  }
+
+  public static ProjectPath projectPathFromPath(String path) {
+    return new ProjectPath(path);
+  }
+
+  public static ProjectPath projectPathFromId(String projectId) {
+    return new ProjectPath(String.format("projects/%s", projectId));
+  }
+
+  /**
+   * Path representing a Pubsub subscription.
+   */
+  public static class SubscriptionPath implements Serializable {
+    private final String projectId;
+    private final String subscriptionName;
+
+    SubscriptionPath(String path) {
+      String[] splits = path.split("/");
+      checkState(
+          splits.length == 4 && splits[0].equals("projects") && splits[2].equals("subscriptions"),
+          "Malformed subscription path %s: "
+          + "must be of the form \"projects/\" + <project id> + \"subscriptions\"", path);
+      this.projectId = splits[1];
+      this.subscriptionName = splits[3];
+    }
+
+    public String getPath() {
+      return String.format("projects/%s/subscriptions/%s", projectId, subscriptionName);
+    }
+
+    public String getName() {
+      return subscriptionName;
+    }
+
+    public String getV1Beta1Path() {
+      return String.format("/subscriptions/%s/%s", projectId, subscriptionName);
+    }
+
+    @Override
+    public boolean equals(Object o) {
+      if (this == o) {
+        return true;
+      }
+      if (o == null || getClass() != o.getClass()) {
+        return false;
+      }
+      SubscriptionPath that = (SubscriptionPath) o;
+      return this.subscriptionName.equals(that.subscriptionName)
+          && this.projectId.equals(that.projectId);
+    }
+
+    @Override
+    public int hashCode() {
+      return Objects.hashCode(projectId, subscriptionName);
+    }
+
+    @Override
+    public String toString() {
+      return getPath();
+    }
+  }
+
+  public static SubscriptionPath subscriptionPathFromPath(String path) {
+    return new SubscriptionPath(path);
+  }
+
+  public static SubscriptionPath subscriptionPathFromName(
+      String projectId, String subscriptionName) {
+    return new SubscriptionPath(String.format("projects/%s/subscriptions/%s",
+                                              projectId, subscriptionName));
+  }
+
+  /**
+   * Path representing a Pubsub topic.
+   */
+  public static class TopicPath implements Serializable {
+    private final String path;
+
+    TopicPath(String path) {
+      this.path = path;
+    }
+
+    public String getPath() {
+      return path;
+    }
+
+    public String getName() {
+      String[] splits = path.split("/");
+      checkState(splits.length == 4, "Malformed topic path %s", path);
+      return splits[3];
+    }
+
+    public String getV1Beta1Path() {
+      String[] splits = path.split("/");
+      checkState(splits.length == 4, "Malformed topic path %s", path);
+      return String.format("/topics/%s/%s", splits[1], splits[3]);
+    }
+
+    @Override
+    public boolean equals(Object o) {
+      if (this == o) {
+        return true;
+      }
+      if (o == null || getClass() != o.getClass()) {
+        return false;
+      }
+      TopicPath topicPath = (TopicPath) o;
+      return path.equals(topicPath.path);
+    }
+
+    @Override
+    public int hashCode() {
+      return path.hashCode();
+    }
+
+    @Override
+    public String toString() {
+      return path;
+    }
+  }
+
+  public static TopicPath topicPathFromPath(String path) {
+    return new TopicPath(path);
+  }
+
+  public static TopicPath topicPathFromName(String projectId, String topicName) {
+    return new TopicPath(String.format("projects/%s/topics/%s", projectId, topicName));
+  }
+
+  /**
+   * A message to be sent to Pubsub.
+   *
+   * <p>NOTE: This class is {@link Serializable} only to support the {@link PubsubTestClient}.
+   * Java serialization is never used for non-test clients.
+   */
+  static class OutgoingMessage implements Serializable {
+    /**
+     * Underlying (encoded) element.
+     */
+    public final byte[] elementBytes;
+
+    public final Map<String, String> attributes;
+
+    /**
+     * Timestamp for element (ms since epoch).
+     */
+    public final long timestampMsSinceEpoch;
+
+    /**
+     * If using an id label, the record id to associate with this record's metadata so the receiver
+     * can reject duplicates. Otherwise {@literal null}.
+     */
+    @Nullable
+    public final String recordId;
+
+    public OutgoingMessage(byte[] elementBytes, Map<String, String> attributes,
+                           long timestampMsSinceEpoch, @Nullable String recordId) {
+      this.elementBytes = elementBytes;
+      this.attributes = attributes;
+      this.timestampMsSinceEpoch = timestampMsSinceEpoch;
+      this.recordId = recordId;
+    }
+
+    @Override
+    public String toString() {
+      return String.format("OutgoingMessage(%db, %dms)",
+                           elementBytes.length, timestampMsSinceEpoch);
+    }
+
+    @Override
+    public boolean equals(Object o) {
+      if (this == o) {
+        return true;
+      }
+      if (o == null || getClass() != o.getClass()) {
+        return false;
+      }
+
+      OutgoingMessage that = (OutgoingMessage) o;
+
+      return timestampMsSinceEpoch == that.timestampMsSinceEpoch
+              && Arrays.equals(elementBytes, that.elementBytes)
+              && Objects.equal(attributes, that.attributes)
+              && Objects.equal(recordId, that.recordId);
+    }
+
+    @Override
+    public int hashCode() {
+      return Objects.hashCode(Arrays.hashCode(elementBytes), attributes, timestampMsSinceEpoch,
+              recordId);
+    }
+  }
+
+  /**
+   * A message received from Pubsub.
+   *
+   * <p>NOTE: This class is {@link Serializable} only to support the {@link PubsubTestClient}.
+   * Java serialization is never used for non-test clients.
+   */
+  static class IncomingMessage implements Serializable {
+    /**
+     * Underlying (encoded) element.
+     */
+    public final byte[] elementBytes;
+
+    public Map<String, String> attributes;
+
+    /**
+     * Timestamp for element (ms since epoch). Either Pubsub's processing time,
+     * or the custom timestamp associated with the message.
+     */
+    public final long timestampMsSinceEpoch;
+
+    /**
+     * Timestamp (in system time) at which we requested the message (ms since epoch).
+     */
+    public final long requestTimeMsSinceEpoch;
+
+    /**
+     * Id to pass back to Pubsub to acknowledge receipt of this message.
+     */
+    public final String ackId;
+
+    /**
+     * Id to pass to the runner to distinguish this message from all others.
+     */
+    public final String recordId;
+
+    public IncomingMessage(
+        byte[] elementBytes,
+        Map<String, String> attributes,
+        long timestampMsSinceEpoch,
+        long requestTimeMsSinceEpoch,
+        String ackId,
+        String recordId) {
+      this.elementBytes = elementBytes;
+      this.attributes = attributes;
+      this.timestampMsSinceEpoch = timestampMsSinceEpoch;
+      this.requestTimeMsSinceEpoch = requestTimeMsSinceEpoch;
+      this.ackId = ackId;
+      this.recordId = recordId;
+    }
+
+    public IncomingMessage withRequestTime(long requestTimeMsSinceEpoch) {
+      return new IncomingMessage(elementBytes, attributes, timestampMsSinceEpoch,
+              requestTimeMsSinceEpoch, ackId, recordId);
+    }
+
+    @Override
+    public String toString() {
+      return String.format("IncomingMessage(%db, %dms)",
+                           elementBytes.length, timestampMsSinceEpoch);
+    }
+
+    @Override
+    public boolean equals(Object o) {
+      if (this == o) {
+        return true;
+      }
+      if (o == null || getClass() != o.getClass()) {
+        return false;
+      }
+
+      IncomingMessage that = (IncomingMessage) o;
+
+      return timestampMsSinceEpoch == that.timestampMsSinceEpoch
+             && requestTimeMsSinceEpoch == that.requestTimeMsSinceEpoch
+             && ackId.equals(that.ackId)
+             && recordId.equals(that.recordId)
+             && Arrays.equals(elementBytes, that.elementBytes)
+              && Objects.equal(attributes, that.attributes);
+    }
+
+    @Override
+    public int hashCode() {
+      return Objects.hashCode(Arrays.hashCode(elementBytes), attributes, timestampMsSinceEpoch,
+                              requestTimeMsSinceEpoch,
+                              ackId, recordId);
+    }
+  }
+
+  /**
+   * Publish {@code outgoingMessages} to Pubsub {@code topic}. Return number of messages
+   * published.
+   */
+  public abstract int publish(TopicPath topic, List<OutgoingMessage> outgoingMessages)
+      throws IOException;
+
+  /**
+   * Request the next batch of up to {@code batchSize} messages from {@code subscription}.
+   * Return the received messages, or empty collection if none were available. Does not
+   * wait for messages to arrive if {@code returnImmediately} is {@literal true}.
+   * Returned messages will record their request time as {@code requestTimeMsSinceEpoch}.
+   */
+  public abstract List<IncomingMessage> pull(
+      long requestTimeMsSinceEpoch,
+      SubscriptionPath subscription,
+      int batchSize,
+      boolean returnImmediately)
+      throws IOException;
+
+  /**
+   * Acknowldege messages from {@code subscription} with {@code ackIds}.
+   */
+  public abstract void acknowledge(SubscriptionPath subscription, List<String> ackIds)
+      throws IOException;
+
+  /**
+   * Modify the ack deadline for messages from {@code subscription} with {@code ackIds} to
+   * be {@code deadlineSeconds} from now.
+   */
+  public abstract void modifyAckDeadline(
+      SubscriptionPath subscription, List<String> ackIds,
+      int deadlineSeconds) throws IOException;
+
+  /**
+   * Create {@code topic}.
+   */
+  public abstract void createTopic(TopicPath topic) throws IOException;
+
+  /*
+   * Delete {@code topic}.
+   */
+  public abstract void deleteTopic(TopicPath topic) throws IOException;
+
+  /**
+   * Return a list of topics for {@code project}.
+   */
+  public abstract List<TopicPath> listTopics(ProjectPath project) throws IOException;
+
+  /**
+   * Create {@code subscription} to {@code topic}.
+   */
+  public abstract void createSubscription(
+      TopicPath topic, SubscriptionPath subscription, int ackDeadlineSeconds) throws IOException;
+
+  /**
+   * Create a random subscription for {@code topic}. Return the {@link SubscriptionPath}. It
+   * is the responsibility of the caller to later delete the subscription.
+   */
+  public SubscriptionPath createRandomSubscription(
+      ProjectPath project, TopicPath topic, int ackDeadlineSeconds) throws IOException {
+    // Create a randomized subscription derived from the topic name.
+    String subscriptionName = topic.getName() + "_beam_" + ThreadLocalRandom.current().nextLong();
+    SubscriptionPath subscription =
+        PubsubClient
+            .subscriptionPathFromName(project.getId(), subscriptionName);
+    createSubscription(topic, subscription, ackDeadlineSeconds);
+    return subscription;
+  }
+
+  /**
+   * Delete {@code subscription}.
+   */
+  public abstract void deleteSubscription(SubscriptionPath subscription) throws IOException;
+
+  /**
+   * Return a list of subscriptions for {@code topic} in {@code project}.
+   */
+  public abstract List<SubscriptionPath> listSubscriptions(ProjectPath project, TopicPath topic)
+      throws IOException;
+
+  /**
+   * Return the ack deadline, in seconds, for {@code subscription}.
+   */
+  public abstract int ackDeadlineSeconds(SubscriptionPath subscription) throws IOException;
+
+  /**
+   * Return {@literal true} if {@link #pull} will always return empty list. Actual clients
+   * will return {@literal false}. Test clients may return {@literal true} to signal that all
+   * expected messages have been pulled and the test may complete.
+   */
+  public abstract boolean isEOF();
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/8098bb1d/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/io/PubsubHelper.java
----------------------------------------------------------------------
diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/io/PubsubHelper.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/io/PubsubHelper.java
index f5cfc2b..15401b7 100644
--- a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/io/PubsubHelper.java
+++ b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/io/PubsubHelper.java
@@ -24,8 +24,6 @@ import java.util.List;
 
 import org.apache.beam.integration.nexmark.NexmarkUtils;
 import org.apache.beam.sdk.options.PubsubOptions;
-import org.apache.beam.sdk.util.PubsubClient;
-import org.apache.beam.sdk.util.PubsubJsonClient;
 
 /**
  * Helper for working with pubsub.

http://git-wip-us.apache.org/repos/asf/beam/blob/8098bb1d/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/io/PubsubJsonClient.java
----------------------------------------------------------------------
diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/io/PubsubJsonClient.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/io/PubsubJsonClient.java
new file mode 100644
index 0000000..b778a09
--- /dev/null
+++ b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/io/PubsubJsonClient.java
@@ -0,0 +1,318 @@
+/*
+ * 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.beam.integration.nexmark.io;
+
+import static com.google.common.base.Preconditions.checkState;
+
+import com.google.api.client.http.HttpRequestInitializer;
+import com.google.api.services.pubsub.Pubsub;
+import com.google.api.services.pubsub.Pubsub.Builder;
+import com.google.api.services.pubsub.model.AcknowledgeRequest;
+import com.google.api.services.pubsub.model.ListSubscriptionsResponse;
+import com.google.api.services.pubsub.model.ListTopicsResponse;
+import com.google.api.services.pubsub.model.ModifyAckDeadlineRequest;
+import com.google.api.services.pubsub.model.PublishRequest;
+import com.google.api.services.pubsub.model.PublishResponse;
+import com.google.api.services.pubsub.model.PubsubMessage;
+import com.google.api.services.pubsub.model.PullRequest;
+import com.google.api.services.pubsub.model.PullResponse;
+import com.google.api.services.pubsub.model.ReceivedMessage;
+import com.google.api.services.pubsub.model.Subscription;
+import com.google.api.services.pubsub.model.Topic;
+import com.google.auth.Credentials;
+import com.google.auth.http.HttpCredentialsAdapter;
+import com.google.cloud.hadoop.util.ChainingHttpRequestInitializer;
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Strings;
+import com.google.common.collect.ImmutableList;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Map;
+import java.util.TreeMap;
+import javax.annotation.Nullable;
+import org.apache.beam.sdk.options.PubsubOptions;
+import org.apache.beam.sdk.util.RetryHttpRequestInitializer;
+import org.apache.beam.sdk.util.Transport;
+
+/**
+ * A Pubsub client using JSON transport.
+ */
+class PubsubJsonClient extends PubsubClient {
+
+  private static class PubsubJsonClientFactory implements PubsubClientFactory {
+    private static HttpRequestInitializer chainHttpRequestInitializer(
+        Credentials credential, HttpRequestInitializer httpRequestInitializer) {
+      if (credential == null) {
+        return httpRequestInitializer;
+      } else {
+        return new ChainingHttpRequestInitializer(
+            new HttpCredentialsAdapter(credential),
+            httpRequestInitializer);
+      }
+    }
+
+    @Override
+    public PubsubClient newClient(
+        @Nullable String timestampLabel, @Nullable String idLabel, PubsubOptions options)
+        throws IOException {
+      Pubsub pubsub = new Builder(
+          Transport.getTransport(),
+          Transport.getJsonFactory(),
+          chainHttpRequestInitializer(
+              options.getGcpCredential(),
+              // Do not log 404. It clutters the output and is possibly even required by the caller.
+              new RetryHttpRequestInitializer(ImmutableList.of(404))))
+          .setRootUrl(options.getPubsubRootUrl())
+          .setApplicationName(options.getAppName())
+          .setGoogleClientRequestInitializer(options.getGoogleApiTrace())
+          .build();
+      return new PubsubJsonClient(timestampLabel, idLabel, pubsub);
+    }
+
+    @Override
+    public String getKind() {
+      return "Json";
+    }
+  }
+
+  /**
+   * Factory for creating Pubsub clients using Json transport.
+   */
+  public static final PubsubClientFactory FACTORY = new PubsubJsonClientFactory();
+
+  /**
+   * Label to use for custom timestamps, or {@literal null} if should use Pubsub publish time
+   * instead.
+   */
+  @Nullable
+  private final String timestampLabel;
+
+  /**
+   * Label to use for custom ids, or {@literal null} if should use Pubsub provided ids.
+   */
+  @Nullable
+  private final String idLabel;
+
+  /**
+   * Underlying JSON transport.
+   */
+  private Pubsub pubsub;
+
+  @VisibleForTesting PubsubJsonClient(
+      @Nullable String timestampLabel,
+      @Nullable String idLabel,
+      Pubsub pubsub) {
+    this.timestampLabel = timestampLabel;
+    this.idLabel = idLabel;
+    this.pubsub = pubsub;
+  }
+
+  @Override
+  public void close() {
+    // Nothing to close.
+  }
+
+  @Override
+  public int publish(TopicPath topic, List<OutgoingMessage> outgoingMessages)
+      throws IOException {
+    List<PubsubMessage> pubsubMessages = new ArrayList<>(outgoingMessages.size());
+    for (OutgoingMessage outgoingMessage : outgoingMessages) {
+      PubsubMessage pubsubMessage = new PubsubMessage().encodeData(outgoingMessage.elementBytes);
+
+      Map<String, String> attributes = outgoingMessage.attributes;
+      if ((timestampLabel != null || idLabel != null) && attributes == null) {
+        attributes = new TreeMap<>();
+      }
+      if (attributes != null) {
+        pubsubMessage.setAttributes(attributes);
+      }
+
+      if (timestampLabel != null) {
+        attributes.put(timestampLabel, String.valueOf(outgoingMessage.timestampMsSinceEpoch));
+      }
+
+      if (idLabel != null && !Strings.isNullOrEmpty(outgoingMessage.recordId)) {
+        attributes.put(idLabel, outgoingMessage.recordId);
+      }
+
+      pubsubMessages.add(pubsubMessage);
+    }
+    PublishRequest request = new PublishRequest().setMessages(pubsubMessages);
+    PublishResponse response = pubsub.projects()
+                                     .topics()
+                                     .publish(topic.getPath(), request)
+                                     .execute();
+    return response.getMessageIds().size();
+  }
+
+  @Override
+  public List<IncomingMessage> pull(
+      long requestTimeMsSinceEpoch,
+      SubscriptionPath subscription,
+      int batchSize,
+      boolean returnImmediately) throws IOException {
+    PullRequest request = new PullRequest()
+        .setReturnImmediately(returnImmediately)
+        .setMaxMessages(batchSize);
+    PullResponse response = pubsub.projects()
+                                  .subscriptions()
+                                  .pull(subscription.getPath(), request)
+                                  .execute();
+    if (response.getReceivedMessages() == null || response.getReceivedMessages().size() == 0) {
+      return ImmutableList.of();
+    }
+    List<IncomingMessage> incomingMessages = new ArrayList<>(response.getReceivedMessages().size());
+    for (ReceivedMessage message : response.getReceivedMessages()) {
+      PubsubMessage pubsubMessage = message.getMessage();
+      @Nullable Map<String, String> attributes = pubsubMessage.getAttributes();
+
+      // Payload.
+      byte[] elementBytes = pubsubMessage.decodeData();
+
+      // Timestamp.
+      long timestampMsSinceEpoch =
+          extractTimestamp(timestampLabel, message.getMessage().getPublishTime(), attributes);
+
+      // Ack id.
+      String ackId = message.getAckId();
+      checkState(!Strings.isNullOrEmpty(ackId));
+
+      // Record id, if any.
+      @Nullable String recordId = null;
+      if (idLabel != null && attributes != null) {
+        recordId = attributes.get(idLabel);
+      }
+      if (Strings.isNullOrEmpty(recordId)) {
+        // Fall back to the Pubsub provided message id.
+        recordId = pubsubMessage.getMessageId();
+      }
+
+      incomingMessages.add(new IncomingMessage(elementBytes, attributes, timestampMsSinceEpoch,
+                                               requestTimeMsSinceEpoch, ackId, recordId));
+    }
+
+    return incomingMessages;
+  }
+
+  @Override
+  public void acknowledge(SubscriptionPath subscription, List<String> ackIds) throws IOException {
+    AcknowledgeRequest request = new AcknowledgeRequest().setAckIds(ackIds);
+    pubsub.projects()
+          .subscriptions()
+          .acknowledge(subscription.getPath(), request)
+          .execute(); // ignore Empty result.
+  }
+
+  @Override
+  public void modifyAckDeadline(
+      SubscriptionPath subscription, List<String> ackIds, int deadlineSeconds)
+      throws IOException {
+    ModifyAckDeadlineRequest request =
+        new ModifyAckDeadlineRequest().setAckIds(ackIds)
+                                      .setAckDeadlineSeconds(deadlineSeconds);
+    pubsub.projects()
+          .subscriptions()
+          .modifyAckDeadline(subscription.getPath(), request)
+          .execute(); // ignore Empty result.
+  }
+
+  @Override
+  public void createTopic(TopicPath topic) throws IOException {
+    pubsub.projects()
+          .topics()
+          .create(topic.getPath(), new Topic())
+          .execute(); // ignore Topic result.
+  }
+
+  @Override
+  public void deleteTopic(TopicPath topic) throws IOException {
+    pubsub.projects()
+          .topics()
+          .delete(topic.getPath())
+          .execute(); // ignore Empty result.
+  }
+
+  @Override
+  public List<TopicPath> listTopics(ProjectPath project) throws IOException {
+    ListTopicsResponse response = pubsub.projects()
+                                        .topics()
+                                        .list(project.getPath())
+                                        .execute();
+    if (response.getTopics() == null || response.getTopics().isEmpty()) {
+      return ImmutableList.of();
+    }
+    List<TopicPath> topics = new ArrayList<>(response.getTopics().size());
+    for (Topic topic : response.getTopics()) {
+      topics.add(topicPathFromPath(topic.getName()));
+    }
+    return topics;
+  }
+
+  @Override
+  public void createSubscription(
+      TopicPath topic, SubscriptionPath subscription,
+      int ackDeadlineSeconds) throws IOException {
+    Subscription request = new Subscription()
+        .setTopic(topic.getPath())
+        .setAckDeadlineSeconds(ackDeadlineSeconds);
+    pubsub.projects()
+          .subscriptions()
+          .create(subscription.getPath(), request)
+          .execute(); // ignore Subscription result.
+  }
+
+  @Override
+  public void deleteSubscription(SubscriptionPath subscription) throws IOException {
+    pubsub.projects()
+          .subscriptions()
+          .delete(subscription.getPath())
+          .execute(); // ignore Empty result.
+  }
+
+  @Override
+  public List<SubscriptionPath> listSubscriptions(ProjectPath project, TopicPath topic)
+      throws IOException {
+    ListSubscriptionsResponse response = pubsub.projects()
+                                               .subscriptions()
+                                               .list(project.getPath())
+                                               .execute();
+    if (response.getSubscriptions() == null || response.getSubscriptions().isEmpty()) {
+      return ImmutableList.of();
+    }
+    List<SubscriptionPath> subscriptions = new ArrayList<>(response.getSubscriptions().size());
+    for (Subscription subscription : response.getSubscriptions()) {
+      if (subscription.getTopic().equals(topic.getPath())) {
+        subscriptions.add(subscriptionPathFromPath(subscription.getName()));
+      }
+    }
+    return subscriptions;
+  }
+
+  @Override
+  public int ackDeadlineSeconds(SubscriptionPath subscription) throws IOException {
+    Subscription response = pubsub.projects().subscriptions().get(subscription.getPath()).execute();
+    return response.getAckDeadlineSeconds();
+  }
+
+  @Override
+  public boolean isEOF() {
+    return false;
+  }
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/8098bb1d/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/io/PubsubTestClient.java
----------------------------------------------------------------------
diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/io/PubsubTestClient.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/io/PubsubTestClient.java
new file mode 100644
index 0000000..125a8d6
--- /dev/null
+++ b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/io/PubsubTestClient.java
@@ -0,0 +1,436 @@
+/*
+ * 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.beam.integration.nexmark.io;
+
+import static com.google.common.base.Preconditions.checkState;
+
+import com.google.api.client.util.Clock;
+import com.google.common.collect.Lists;
+import com.google.common.collect.Sets;
+import java.io.Closeable;
+import java.io.IOException;
+import java.io.Serializable;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import javax.annotation.Nullable;
+import org.apache.beam.sdk.options.PubsubOptions;
+
+/**
+ * A (partial) implementation of {@link PubsubClient} for use by unit tests. Only suitable for
+ * testing {@link #publish}, {@link #pull}, {@link #acknowledge} and {@link #modifyAckDeadline}
+ * methods. Relies on statics to mimic the Pubsub service, though we try to hide that.
+ */
+class PubsubTestClient extends PubsubClient implements Serializable {
+  /**
+   * Mimic the state of the simulated Pubsub 'service'.
+   *
+   * <p>Note that the {@link PubsubTestClientFactory} is serialized/deserialized even when running
+   * test pipelines. Meanwhile it is valid for multiple {@link PubsubTestClient}s to be created
+   * from the same client factory and run in parallel. Thus we can't enforce aliasing of the
+   * following data structures over all clients and must resort to a static.
+   */
+  private static class State {
+    /**
+     * True if has been primed for a test but not yet validated.
+     */
+    boolean isActive;
+
+    /**
+     * Publish mode only: Only publish calls for this topic are allowed.
+     */
+    @Nullable
+    TopicPath expectedTopic;
+
+    /**
+     * Publish mode only: Messages yet to seen in a {@link #publish} call.
+     */
+    @Nullable
+    Set<OutgoingMessage> remainingExpectedOutgoingMessages;
+
+    /**
+     * Publish mode only: Messages which should throw when first sent to simulate transient publish
+     * failure.
+     */
+    @Nullable
+    Set<OutgoingMessage> remainingFailingOutgoingMessages;
+
+    /**
+     * Pull mode only: Clock from which to get current time.
+     */
+    @Nullable
+    Clock clock;
+
+    /**
+     * Pull mode only: Only pull calls for this subscription are allowed.
+     */
+    @Nullable
+    SubscriptionPath expectedSubscription;
+
+    /**
+     * Pull mode only: Timeout to simulate.
+     */
+    int ackTimeoutSec;
+
+    /**
+     * Pull mode only: Messages waiting to be received by a {@link #pull} call.
+     */
+    @Nullable
+    List<IncomingMessage> remainingPendingIncomingMessages;
+
+    /**
+     * Pull mode only: Messages which have been returned from a {@link #pull} call and
+     * not yet ACKed by an {@link #acknowledge} call.
+     */
+    @Nullable
+    Map<String, IncomingMessage> pendingAckIncomingMessages;
+
+    /**
+     * Pull mode only: When above messages are due to have their ACK deadlines expire.
+     */
+    @Nullable
+    Map<String, Long> ackDeadline;
+  }
+
+  private static final State STATE = new State();
+
+  /** Closing the factory will validate all expected messages were processed. */
+  public interface PubsubTestClientFactory
+          extends PubsubClientFactory, Closeable, Serializable {
+  }
+
+  /**
+   * Return a factory for testing publishers. Only one factory may be in-flight at a time.
+   * The factory must be closed when the test is complete, at which point final validation will
+   * occur.
+   */
+  static PubsubTestClientFactory createFactoryForPublish(
+      final TopicPath expectedTopic,
+      final Iterable<OutgoingMessage> expectedOutgoingMessages,
+      final Iterable<OutgoingMessage> failingOutgoingMessages) {
+    synchronized (STATE) {
+      checkState(!STATE.isActive, "Test still in flight");
+      STATE.expectedTopic = expectedTopic;
+      STATE.remainingExpectedOutgoingMessages = Sets.newHashSet(expectedOutgoingMessages);
+      STATE.remainingFailingOutgoingMessages = Sets.newHashSet(failingOutgoingMessages);
+      STATE.isActive = true;
+    }
+    return new PubsubTestClientFactory() {
+      @Override
+      public PubsubClient newClient(
+          @Nullable String timestampLabel, @Nullable String idLabel, PubsubOptions options)
+          throws IOException {
+        return new PubsubTestClient();
+      }
+
+      @Override
+      public String getKind() {
+        return "PublishTest";
+      }
+
+      @Override
+      public void close() {
+        synchronized (STATE) {
+          checkState(STATE.isActive, "No test still in flight");
+          checkState(STATE.remainingExpectedOutgoingMessages.isEmpty(),
+                     "Still waiting for %s messages to be published",
+                     STATE.remainingExpectedOutgoingMessages.size());
+          STATE.isActive = false;
+          STATE.remainingExpectedOutgoingMessages = null;
+        }
+      }
+    };
+  }
+
+  /**
+   * Return a factory for testing subscribers. Only one factory may be in-flight at a time.
+   * The factory must be closed when the test in complete
+   */
+  public static PubsubTestClientFactory createFactoryForPull(
+      final Clock clock,
+      final SubscriptionPath expectedSubscription,
+      final int ackTimeoutSec,
+      final Iterable<IncomingMessage> expectedIncomingMessages) {
+    synchronized (STATE) {
+      checkState(!STATE.isActive, "Test still in flight");
+      STATE.clock = clock;
+      STATE.expectedSubscription = expectedSubscription;
+      STATE.ackTimeoutSec = ackTimeoutSec;
+      STATE.remainingPendingIncomingMessages = Lists.newArrayList(expectedIncomingMessages);
+      STATE.pendingAckIncomingMessages = new HashMap<>();
+      STATE.ackDeadline = new HashMap<>();
+      STATE.isActive = true;
+    }
+    return new PubsubTestClientFactory() {
+      @Override
+      public PubsubClient newClient(
+          @Nullable String timestampLabel, @Nullable String idLabel, PubsubOptions options)
+          throws IOException {
+        return new PubsubTestClient();
+      }
+
+      @Override
+      public String getKind() {
+        return "PullTest";
+      }
+
+      @Override
+      public void close() {
+        synchronized (STATE) {
+          checkState(STATE.isActive, "No test still in flight");
+          checkState(STATE.remainingPendingIncomingMessages.isEmpty(),
+                     "Still waiting for %s messages to be pulled",
+                     STATE.remainingPendingIncomingMessages.size());
+          checkState(STATE.pendingAckIncomingMessages.isEmpty(),
+                     "Still waiting for %s messages to be ACKed",
+                     STATE.pendingAckIncomingMessages.size());
+          checkState(STATE.ackDeadline.isEmpty(),
+                     "Still waiting for %s messages to be ACKed",
+                     STATE.ackDeadline.size());
+          STATE.isActive = false;
+          STATE.remainingPendingIncomingMessages = null;
+          STATE.pendingAckIncomingMessages = null;
+          STATE.ackDeadline = null;
+        }
+      }
+    };
+  }
+
+  public static PubsubTestClientFactory createFactoryForCreateSubscription() {
+    return new PubsubTestClientFactory() {
+      int numCalls = 0;
+
+      @Override
+      public void close() throws IOException {
+        checkState(
+            numCalls == 1, "Expected exactly one subscription to be created, got %s", numCalls);
+      }
+
+      @Override
+      public PubsubClient newClient(
+          @Nullable String timestampLabel, @Nullable String idLabel, PubsubOptions options)
+          throws IOException {
+        return new PubsubTestClient() {
+          @Override
+          public void createSubscription(
+              TopicPath topic, SubscriptionPath subscription, int ackDeadlineSeconds)
+              throws IOException {
+            checkState(numCalls == 0, "Expected at most one subscription to be created");
+            numCalls++;
+          }
+        };
+      }
+
+      @Override
+      public String getKind() {
+        return "CreateSubscriptionTest";
+      }
+    };
+  }
+
+  /**
+   * Return true if in pull mode.
+   */
+  private boolean inPullMode() {
+    checkState(STATE.isActive, "No test is active");
+    return STATE.expectedSubscription != null;
+  }
+
+  /**
+   * Return true if in publish mode.
+   */
+  private boolean inPublishMode() {
+    checkState(STATE.isActive, "No test is active");
+    return STATE.expectedTopic != null;
+  }
+
+  /**
+   * For subscription mode only:
+   * Track progression of time according to the {@link Clock} passed . This will simulate Pubsub
+   * expiring
+   * outstanding ACKs.
+   */
+  public void advance() {
+    synchronized (STATE) {
+      checkState(inPullMode(), "Can only advance in pull mode");
+      // Any messages who's ACKs timed out are available for re-pulling.
+      Iterator<Map.Entry<String, Long>> deadlineItr = STATE.ackDeadline.entrySet().iterator();
+      while (deadlineItr.hasNext()) {
+        Map.Entry<String, Long> entry = deadlineItr.next();
+        if (entry.getValue() <= STATE.clock.currentTimeMillis()) {
+          STATE.remainingPendingIncomingMessages.add(
+              STATE.pendingAckIncomingMessages.remove(entry.getKey()));
+          deadlineItr.remove();
+        }
+      }
+    }
+  }
+
+  @Override
+  public void close() {
+  }
+
+  @Override
+  public int publish(
+      TopicPath topic, List<OutgoingMessage> outgoingMessages) throws IOException {
+    synchronized (STATE) {
+      checkState(inPublishMode(), "Can only publish in publish mode");
+      checkState(topic.equals(STATE.expectedTopic), "Topic %s does not match expected %s", topic,
+                 STATE.expectedTopic);
+      for (OutgoingMessage outgoingMessage : outgoingMessages) {
+        if (STATE.remainingFailingOutgoingMessages.remove(outgoingMessage)) {
+          throw new RuntimeException("Simulating failure for " + outgoingMessage);
+        }
+        checkState(STATE.remainingExpectedOutgoingMessages.remove(outgoingMessage),
+                   "Unexpected outgoing message %s", outgoingMessage);
+      }
+      return outgoingMessages.size();
+    }
+  }
+
+  @Override
+  public List<IncomingMessage> pull(
+      long requestTimeMsSinceEpoch, SubscriptionPath subscription, int batchSize,
+      boolean returnImmediately) throws IOException {
+    synchronized (STATE) {
+      checkState(inPullMode(), "Can only pull in pull mode");
+      long now = STATE.clock.currentTimeMillis();
+      checkState(requestTimeMsSinceEpoch == now,
+                 "Simulated time %s does not match request time %s", now, requestTimeMsSinceEpoch);
+      checkState(subscription.equals(STATE.expectedSubscription),
+                 "Subscription %s does not match expected %s", subscription,
+                 STATE.expectedSubscription);
+      checkState(returnImmediately, "Pull only supported if returning immediately");
+
+      List<IncomingMessage> incomingMessages = new ArrayList<>();
+      Iterator<IncomingMessage> pendItr = STATE.remainingPendingIncomingMessages.iterator();
+      while (pendItr.hasNext()) {
+        IncomingMessage incomingMessage = pendItr.next();
+        pendItr.remove();
+        IncomingMessage incomingMessageWithRequestTime =
+            incomingMessage.withRequestTime(requestTimeMsSinceEpoch);
+        incomingMessages.add(incomingMessageWithRequestTime);
+        STATE.pendingAckIncomingMessages.put(incomingMessageWithRequestTime.ackId,
+                                             incomingMessageWithRequestTime);
+        STATE.ackDeadline.put(incomingMessageWithRequestTime.ackId,
+                              requestTimeMsSinceEpoch + STATE.ackTimeoutSec * 1000);
+        if (incomingMessages.size() >= batchSize) {
+          break;
+        }
+      }
+      return incomingMessages;
+    }
+  }
+
+  @Override
+  public void acknowledge(
+      SubscriptionPath subscription,
+      List<String> ackIds) throws IOException {
+    synchronized (STATE) {
+      checkState(inPullMode(), "Can only acknowledge in pull mode");
+      checkState(subscription.equals(STATE.expectedSubscription),
+                 "Subscription %s does not match expected %s", subscription,
+                 STATE.expectedSubscription);
+
+      for (String ackId : ackIds) {
+        checkState(STATE.ackDeadline.remove(ackId) != null,
+                   "No message with ACK id %s is waiting for an ACK", ackId);
+        checkState(STATE.pendingAckIncomingMessages.remove(ackId) != null,
+                   "No message with ACK id %s is waiting for an ACK", ackId);
+      }
+    }
+  }
+
+  @Override
+  public void modifyAckDeadline(
+      SubscriptionPath subscription, List<String> ackIds, int deadlineSeconds) throws IOException {
+    synchronized (STATE) {
+      checkState(inPullMode(), "Can only modify ack deadline in pull mode");
+      checkState(subscription.equals(STATE.expectedSubscription),
+                 "Subscription %s does not match expected %s", subscription,
+                 STATE.expectedSubscription);
+
+      for (String ackId : ackIds) {
+        if (deadlineSeconds > 0) {
+          checkState(STATE.ackDeadline.remove(ackId) != null,
+                     "No message with ACK id %s is waiting for an ACK", ackId);
+          checkState(STATE.pendingAckIncomingMessages.containsKey(ackId),
+                     "No message with ACK id %s is waiting for an ACK", ackId);
+          STATE.ackDeadline.put(ackId, STATE.clock.currentTimeMillis() + deadlineSeconds * 1000);
+        } else {
+          checkState(STATE.ackDeadline.remove(ackId) != null,
+                     "No message with ACK id %s is waiting for an ACK", ackId);
+          IncomingMessage message = STATE.pendingAckIncomingMessages.remove(ackId);
+          checkState(message != null, "No message with ACK id %s is waiting for an ACK", ackId);
+          STATE.remainingPendingIncomingMessages.add(message);
+        }
+      }
+    }
+  }
+
+  @Override
+  public void createTopic(TopicPath topic) throws IOException {
+    throw new UnsupportedOperationException();
+  }
+
+  @Override
+  public void deleteTopic(TopicPath topic) throws IOException {
+    throw new UnsupportedOperationException();
+  }
+
+  @Override
+  public List<TopicPath> listTopics(ProjectPath project) throws IOException {
+    throw new UnsupportedOperationException();
+  }
+
+  @Override
+  public void createSubscription(
+      TopicPath topic, SubscriptionPath subscription, int ackDeadlineSeconds) throws IOException {
+    throw new UnsupportedOperationException();
+  }
+
+  @Override
+  public void deleteSubscription(SubscriptionPath subscription) throws IOException {
+    throw new UnsupportedOperationException();
+  }
+
+  @Override
+  public List<SubscriptionPath> listSubscriptions(
+      ProjectPath project, TopicPath topic) throws IOException {
+    throw new UnsupportedOperationException();
+  }
+
+  @Override
+  public int ackDeadlineSeconds(SubscriptionPath subscription) throws IOException {
+    synchronized (STATE) {
+      return STATE.ackTimeoutSec;
+    }
+  }
+
+  @Override
+  public boolean isEOF() {
+    synchronized (STATE) {
+      checkState(inPullMode(), "Can only check EOF in pull mode");
+      return STATE.remainingPendingIncomingMessages.isEmpty();
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/8098bb1d/integration/java/pom.xml
----------------------------------------------------------------------
diff --git a/integration/java/pom.xml b/integration/java/pom.xml
new file mode 100644
index 0000000..dcad4c3
--- /dev/null
+++ b/integration/java/pom.xml
@@ -0,0 +1,37 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<!--
+    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.
+-->
+<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
+
+  <modelVersion>4.0.0</modelVersion>
+
+  <parent>
+    <groupId>org.apache.beam</groupId>
+    <artifactId>beam-integration-parent</artifactId>
+    <version>0.7.0-SNAPSHOT</version>
+    <relativePath>../pom.xml</relativePath>
+  </parent>
+
+  <artifactId>beam-integration-java-parent</artifactId>
+  <packaging>pom</packaging>
+  <name>Apache Beam :: Integration Tests :: Java</name>
+
+  <modules>
+    <module>nexmark</module>
+  </modules>
+
+</project>

http://git-wip-us.apache.org/repos/asf/beam/blob/8098bb1d/integration/pom.xml
----------------------------------------------------------------------
diff --git a/integration/pom.xml b/integration/pom.xml
new file mode 100644
index 0000000..4839da5
--- /dev/null
+++ b/integration/pom.xml
@@ -0,0 +1,37 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<!--
+    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.
+-->
+<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
+
+  <modelVersion>4.0.0</modelVersion>
+
+  <parent>
+    <groupId>org.apache.beam</groupId>
+    <artifactId>beam-parent</artifactId>
+    <version>0.7.0-SNAPSHOT</version>
+    <relativePath>../pom.xml</relativePath>
+  </parent>
+
+  <artifactId>beam-integration-parent</artifactId>
+  <packaging>pom</packaging>
+  <name>Apache Beam :: Integration Tests</name>
+
+  <modules>
+    <module>java</module>
+  </modules>
+
+</project>

http://git-wip-us.apache.org/repos/asf/beam/blob/8098bb1d/pom.xml
----------------------------------------------------------------------
diff --git a/pom.xml b/pom.xml
index c92d391..bddbf1f 100644
--- a/pom.xml
+++ b/pom.xml
@@ -187,7 +187,7 @@
     <module>sdks</module>
     <module>runners</module>
     <module>examples</module>
-    <module>integration/java/nexmark</module>
+    <module>integration</module>
     <!-- sdks/java/javadoc builds project-wide Javadoc. It has to run last. -->
     <module>sdks/java/javadoc</module>
   </modules>


[23/55] [abbrv] beam git commit: Improve query5, query10 and query11

Posted by ie...@apache.org.
Improve query5, query10 and query11

query5: Add comment on key lifting (issue #30)

query10: Add comment for strange groupByKey (issue #31)

query11: Replace Count.perKey by Count.perElement (issue #32)


Project: http://git-wip-us.apache.org/repos/asf/beam/repo
Commit: http://git-wip-us.apache.org/repos/asf/beam/commit/7bfc982c
Tree: http://git-wip-us.apache.org/repos/asf/beam/tree/7bfc982c
Diff: http://git-wip-us.apache.org/repos/asf/beam/diff/7bfc982c

Branch: refs/heads/master
Commit: 7bfc982c77de52f49ba1b304a81bb0d53de5f44a
Parents: a7f9f7d
Author: Etienne Chauchot <ec...@gmail.com>
Authored: Fri Mar 24 14:29:08 2017 +0100
Committer: Ismaël Mejía <ie...@gmail.com>
Committed: Wed Aug 23 19:07:27 2017 +0200

----------------------------------------------------------------------
 .../integration/nexmark/queries/Query10.java    |  3 +-
 .../integration/nexmark/queries/Query11.java    | 47 ++++++++++----------
 .../integration/nexmark/queries/Query5.java     |  2 +
 3 files changed, 27 insertions(+), 25 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/beam/blob/7bfc982c/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query10.java
----------------------------------------------------------------------
diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query10.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query10.java
index 6912ed1..5246427 100644
--- a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query10.java
+++ b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query10.java
@@ -322,8 +322,7 @@ public class Query10 extends NexmarkQuery {
             // We expect no late data here, but we'll assume the worst so we can detect any.
             .withAllowedLateness(Duration.standardDays(1))
             .discardingFiredPanes())
-      // TODO etienne: unnecessary groupByKey? because aggregators are shared in parallel
-      // and Pardo is also in parallel, why group all elements in memory of the same executor?
+      // this GroupByKey allows to have one file per window
       .apply(name + ".GroupByKey2", GroupByKey.<Void, OutputFile>create())
         .apply(name + ".Index",
             ParDo.of(new DoFn<KV<Void, Iterable<OutputFile>>, Done>() {

http://git-wip-us.apache.org/repos/asf/beam/blob/7bfc982c/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query11.java
----------------------------------------------------------------------
diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query11.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query11.java
index 4da99eb..a8a61ae 100644
--- a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query11.java
+++ b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query11.java
@@ -48,29 +48,30 @@ public class Query11 extends NexmarkQuery {
   }
 
   private PCollection<BidsPerSession> applyTyped(PCollection<Event> events) {
-    return events.apply(JUST_BIDS)
-        .apply(name + ".Rekey",
-          // TODO etienne: why not avoid this ParDo and do a Cont.perElement?
-          ParDo.of(new DoFn<Bid, KV<Long, Void>>() {
-                  @ProcessElement
-                  public void processElement(ProcessContext c) {
-                    Bid bid = c.element();
-                    c.output(KV.of(bid.bidder, (Void) null));
-                  }
-                }))
-        .apply(Window.<KV<Long, Void>>into(
-            Sessions.withGapDuration(Duration.standardSeconds(configuration.windowSizeSec)))
-        .triggering(Repeatedly.forever(AfterPane.elementCountAtLeast(configuration.maxLogEvents)))
-        .discardingFiredPanes()
-        .withAllowedLateness(Duration.standardSeconds(configuration.occasionalDelaySec / 2)))
-        .apply(Count.<Long, Void>perKey())
-        .apply(name + ".ToResult",
-            ParDo.of(new DoFn<KV<Long, Long>, BidsPerSession>() {
-                  @ProcessElement
-                  public void processElement(ProcessContext c) {
-                    c.output(new BidsPerSession(c.element().getKey(), c.element().getValue()));
-                  }
-                }));
+    PCollection<Long> bidders = events.apply(JUST_BIDS).apply(name + ".Rekey",
+        ParDo.of(new DoFn<Bid, Long>() {
+
+          @ProcessElement public void processElement(ProcessContext c) {
+            Bid bid = c.element();
+            c.output(bid.bidder);
+          }
+        }));
+
+    PCollection<Long> biddersWindowed = bidders.apply(
+        Window.<Long>into(
+          Sessions.withGapDuration(Duration.standardSeconds(configuration.windowSizeSec)))
+            .triggering(
+                Repeatedly.forever(AfterPane.elementCountAtLeast(configuration.maxLogEvents)))
+            .discardingFiredPanes()
+            .withAllowedLateness(Duration.standardSeconds(configuration.occasionalDelaySec / 2)));
+    PCollection<BidsPerSession> bidsPerSession = biddersWindowed.apply(Count.<Long>perElement())
+        .apply(name + ".ToResult", ParDo.of(new DoFn<KV<Long, Long>, BidsPerSession>() {
+
+          @ProcessElement public void processElement(ProcessContext c) {
+            c.output(new BidsPerSession(c.element().getKey(), c.element().getValue()));
+          }
+        }));
+    return bidsPerSession;
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/beam/blob/7bfc982c/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query5.java
----------------------------------------------------------------------
diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query5.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query5.java
index 9f02ddb..34b7b50 100644
--- a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query5.java
+++ b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query5.java
@@ -75,6 +75,8 @@ public class Query5 extends NexmarkQuery {
 
       // We'll want to keep all auctions with the maximal number of bids.
         // Start by lifting each into a singleton list.
+        // need to do so because bellow combine returns a list of auctions in the key in case of
+        // equal number of bids. Combine needs to have same input type and return type.
         .apply(name + ".ToSingletons",
             ParDo.of(new DoFn<KV<Long, Long>, KV<List<Long>, Long>>() {
                   @ProcessElement


[35/55] [abbrv] beam git commit: Rename NexmarkDriver to Main and NexmarkRunner to NexmarkLauncher

Posted by ie...@apache.org.
http://git-wip-us.apache.org/repos/asf/beam/blob/683680b1/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkRunner.java
----------------------------------------------------------------------
diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkRunner.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkRunner.java
deleted file mode 100644
index 935544e..0000000
--- a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkRunner.java
+++ /dev/null
@@ -1,1172 +0,0 @@
-/*
- * 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.beam.integration.nexmark;
-
-import static com.google.common.base.Preconditions.checkState;
-
-import com.google.api.services.bigquery.model.TableFieldSchema;
-import com.google.api.services.bigquery.model.TableRow;
-import com.google.api.services.bigquery.model.TableSchema;
-import com.google.common.base.Strings;
-import com.google.common.collect.ImmutableList;
-import com.google.common.collect.Lists;
-import java.io.IOException;
-import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.Collections;
-import java.util.HashMap;
-import java.util.List;
-import java.util.NoSuchElementException;
-import java.util.concurrent.ThreadLocalRandom;
-import javax.annotation.Nullable;
-import org.apache.beam.integration.nexmark.model.Auction;
-import org.apache.beam.integration.nexmark.model.Bid;
-import org.apache.beam.integration.nexmark.model.Event;
-import org.apache.beam.integration.nexmark.model.KnownSize;
-import org.apache.beam.integration.nexmark.model.Person;
-import org.apache.beam.integration.nexmark.queries.NexmarkQuery;
-import org.apache.beam.integration.nexmark.queries.NexmarkQueryModel;
-import org.apache.beam.integration.nexmark.queries.Query0;
-import org.apache.beam.integration.nexmark.queries.Query0Model;
-import org.apache.beam.integration.nexmark.queries.Query1;
-import org.apache.beam.integration.nexmark.queries.Query10;
-import org.apache.beam.integration.nexmark.queries.Query11;
-import org.apache.beam.integration.nexmark.queries.Query12;
-import org.apache.beam.integration.nexmark.queries.Query1Model;
-import org.apache.beam.integration.nexmark.queries.Query2;
-import org.apache.beam.integration.nexmark.queries.Query2Model;
-import org.apache.beam.integration.nexmark.queries.Query3;
-import org.apache.beam.integration.nexmark.queries.Query3Model;
-import org.apache.beam.integration.nexmark.queries.Query4;
-import org.apache.beam.integration.nexmark.queries.Query4Model;
-import org.apache.beam.integration.nexmark.queries.Query5;
-import org.apache.beam.integration.nexmark.queries.Query5Model;
-import org.apache.beam.integration.nexmark.queries.Query6;
-import org.apache.beam.integration.nexmark.queries.Query6Model;
-import org.apache.beam.integration.nexmark.queries.Query7;
-import org.apache.beam.integration.nexmark.queries.Query7Model;
-import org.apache.beam.integration.nexmark.queries.Query8;
-import org.apache.beam.integration.nexmark.queries.Query8Model;
-import org.apache.beam.integration.nexmark.queries.Query9;
-import org.apache.beam.integration.nexmark.queries.Query9Model;
-import org.apache.beam.sdk.Pipeline;
-import org.apache.beam.sdk.PipelineResult;
-import org.apache.beam.sdk.coders.CoderException;
-import org.apache.beam.sdk.io.AvroIO;
-import org.apache.beam.sdk.io.TextIO;
-import org.apache.beam.sdk.io.gcp.bigquery.BigQueryIO;
-import org.apache.beam.sdk.io.gcp.pubsub.PubsubIO;
-import org.apache.beam.sdk.io.gcp.pubsub.PubsubMessage;
-import org.apache.beam.sdk.metrics.DistributionResult;
-import org.apache.beam.sdk.metrics.MetricNameFilter;
-import org.apache.beam.sdk.metrics.MetricQueryResults;
-import org.apache.beam.sdk.metrics.MetricResult;
-import org.apache.beam.sdk.metrics.MetricsFilter;
-import org.apache.beam.sdk.testing.PAssert;
-import org.apache.beam.sdk.transforms.DoFn;
-import org.apache.beam.sdk.transforms.ParDo;
-import org.apache.beam.sdk.util.CoderUtils;
-import org.apache.beam.sdk.values.PCollection;
-import org.apache.beam.sdk.values.PCollectionTuple;
-import org.apache.beam.sdk.values.TimestampedValue;
-import org.apache.beam.sdk.values.TupleTag;
-import org.apache.beam.sdk.values.TupleTagList;
-import org.joda.time.Duration;
-
-/**
- * Run a single Nexmark query using a given configuration.
- */
-public class NexmarkRunner<OptionT extends NexmarkOptions> {
-  /**
-   * Minimum number of samples needed for 'stead-state' rate calculation.
-   */
-  private static final int MIN_SAMPLES = 9;
-  /**
-   * Minimum length of time over which to consider samples for 'steady-state' rate calculation.
-   */
-  private static final Duration MIN_WINDOW = Duration.standardMinutes(2);
-  /**
-   * Delay between perf samples.
-   */
-  private static final Duration PERF_DELAY = Duration.standardSeconds(15);
-  /**
-   * How long to let streaming pipeline run after all events have been generated and we've
-   * seen no activity.
-   */
-  private static final Duration DONE_DELAY = Duration.standardMinutes(1);
-  /**
-   * How long to allow no activity without warning.
-   */
-  private static final Duration STUCK_WARNING_DELAY = Duration.standardMinutes(10);
-  /**
-   * How long to let streaming pipeline run after we've
-   * seen no activity, even if all events have not been generated.
-   */
-  private static final Duration STUCK_TERMINATE_DELAY = Duration.standardDays(3);
-  /**
-   * NexmarkOptions shared by all runs.
-   */
-  private final OptionT options;
-
-  /**
-   * Which configuration we are running.
-   */
-  @Nullable
-  private NexmarkConfiguration configuration;
-
-  /**
-   * If in --pubsubMode=COMBINED, the event monitor for the publisher pipeline. Otherwise null.
-   */
-  @Nullable
-  private Monitor<Event> publisherMonitor;
-
-  /**
-   * If in --pubsubMode=COMBINED, the pipeline result for the publisher pipeline. Otherwise null.
-   */
-  @Nullable
-  private PipelineResult publisherResult;
-
-  /**
-   * Result for the main pipeline.
-   */
-  @Nullable
-  private PipelineResult mainResult;
-
-  /**
-   * Query name we are running.
-   */
-  @Nullable
-  private String queryName;
-
-  public NexmarkRunner(OptionT options) {
-    this.options = options;
-  }
-
-
-  /**
-   * Is this query running in streaming mode?
-   */
-  private boolean isStreaming() {
-    return options.isStreaming();
-  }
-
-  /**
-   * Return number of cores per worker.
-   */
-  protected int coresPerWorker() {
-    return 4;
-  }
-
-  /**
-   * Return maximum number of workers.
-   */
-  private int maxNumWorkers() {
-    return 5;
-  }
-
-  /**
-   * Return the current value for a long counter, or a default value if can't be retrieved.
-   * Note this uses only attempted metrics because some runners don't support committed metrics.
-   */
-  private long getCounterMetric(PipelineResult result, String namespace, String name,
-    long defaultValue) {
-    //TODO Ismael calc this only once
-    MetricQueryResults metrics = result.metrics().queryMetrics(
-        MetricsFilter.builder().addNameFilter(MetricNameFilter.named(namespace, name)).build());
-    Iterable<MetricResult<Long>> counters = metrics.counters();
-    try {
-      MetricResult<Long> metricResult = counters.iterator().next();
-      return metricResult.attempted();
-    } catch (NoSuchElementException e) {
-      //TODO Ismael
-    }
-    return defaultValue;
-  }
-
-  /**
-   * Return the current value for a long counter, or a default value if can't be retrieved.
-   * Note this uses only attempted metrics because some runners don't support committed metrics.
-   */
-  private long getDistributionMetric(PipelineResult result, String namespace, String name,
-      DistributionType distType, long defaultValue) {
-    MetricQueryResults metrics = result.metrics().queryMetrics(
-        MetricsFilter.builder().addNameFilter(MetricNameFilter.named(namespace, name)).build());
-    Iterable<MetricResult<DistributionResult>> distributions = metrics.distributions();
-    try {
-      MetricResult<DistributionResult> distributionResult = distributions.iterator().next();
-      if (distType.equals(DistributionType.MIN)) {
-        return distributionResult.attempted().min();
-      } else if (distType.equals(DistributionType.MAX)) {
-        return distributionResult.attempted().max();
-      } else {
-        //TODO Ismael
-      }
-    } catch (NoSuchElementException e) {
-      //TODO Ismael
-    }
-    return defaultValue;
-  }
-
-  private enum DistributionType {MIN, MAX}
-
-  /**
-   * Return the current value for a time counter, or -1 if can't be retrieved.
-   */
-  private long getTimestampMetric(long now, long value) {
-    //TODO Ismael improve doc
-    if (Math.abs(value - now) > Duration.standardDays(10000).getMillis()) {
-      return -1;
-    }
-    return value;
-  }
-
-  /**
-   * Find a 'steady state' events/sec from {@code snapshots} and
-   * store it in {@code perf} if found.
-   */
-  private void captureSteadyState(NexmarkPerf perf, List<NexmarkPerf.ProgressSnapshot> snapshots) {
-    if (!options.isStreaming()) {
-      return;
-    }
-
-    // Find the first sample with actual event and result counts.
-    int dataStart = 0;
-    for (; dataStart < snapshots.size(); dataStart++) {
-      if (snapshots.get(dataStart).numEvents >= 0 && snapshots.get(dataStart).numResults >= 0) {
-        break;
-      }
-    }
-
-    // Find the last sample which demonstrated progress.
-    int dataEnd = snapshots.size() - 1;
-    for (; dataEnd > dataStart; dataEnd--) {
-      if (snapshots.get(dataEnd).anyActivity(snapshots.get(dataEnd - 1))) {
-        break;
-      }
-    }
-
-    int numSamples = dataEnd - dataStart + 1;
-    if (numSamples < MIN_SAMPLES) {
-      // Not enough samples.
-      NexmarkUtils.console("%d samples not enough to calculate steady-state event rate",
-          numSamples);
-      return;
-    }
-
-    // We'll look at only the middle third samples.
-    int sampleStart = dataStart + numSamples / 3;
-    int sampleEnd = dataEnd - numSamples / 3;
-
-    double sampleSec =
-        snapshots.get(sampleEnd).secSinceStart - snapshots.get(sampleStart).secSinceStart;
-    if (sampleSec < MIN_WINDOW.getStandardSeconds()) {
-      // Not sampled over enough time.
-      NexmarkUtils.console(
-          "sample of %.1f sec not long enough to calculate steady-state event rate",
-          sampleSec);
-      return;
-    }
-
-    // Find rate with least squares error.
-    double sumxx = 0.0;
-    double sumxy = 0.0;
-    long prevNumEvents = -1;
-    for (int i = sampleStart; i <= sampleEnd; i++) {
-      if (prevNumEvents == snapshots.get(i).numEvents) {
-        // Skip samples with no change in number of events since they contribute no data.
-        continue;
-      }
-      // Use the effective runtime instead of wallclock time so we can
-      // insulate ourselves from delays and stutters in the query manager.
-      double x = snapshots.get(i).runtimeSec;
-      prevNumEvents = snapshots.get(i).numEvents;
-      double y = prevNumEvents;
-      sumxx += x * x;
-      sumxy += x * y;
-    }
-    double eventsPerSec = sumxy / sumxx;
-    NexmarkUtils.console("revising events/sec from %.1f to %.1f", perf.eventsPerSec, eventsPerSec);
-    perf.eventsPerSec = eventsPerSec;
-  }
-
-  /**
-   * Return the current performance given {@code eventMonitor} and {@code resultMonitor}.
-   */
-  private NexmarkPerf currentPerf(
-      long startMsSinceEpoch, long now, PipelineResult result,
-      List<NexmarkPerf.ProgressSnapshot> snapshots, Monitor<?> eventMonitor,
-      Monitor<?> resultMonitor) {
-    NexmarkPerf perf = new NexmarkPerf();
-
-    long numEvents =
-      getCounterMetric(result, eventMonitor.name, eventMonitor.prefix + ".elements", -1);
-    long numEventBytes =
-      getCounterMetric(result, eventMonitor.name, eventMonitor.prefix + ".bytes", -1);
-    long eventStart =
-      getTimestampMetric(now,
-        getDistributionMetric(result, eventMonitor.name, eventMonitor.prefix + ".startTime",
-          DistributionType.MIN, -1));
-    long eventEnd =
-      getTimestampMetric(now,
-        getDistributionMetric(result, eventMonitor.name, eventMonitor.prefix + ".endTime",
-          DistributionType.MAX, -1));
-
-    long numResults =
-      getCounterMetric(result, resultMonitor.name, resultMonitor.prefix + ".elements", -1);
-    long numResultBytes =
-      getCounterMetric(result, resultMonitor.name, resultMonitor.prefix + ".bytes", -1);
-    long resultStart =
-      getTimestampMetric(now,
-        getDistributionMetric(result, resultMonitor.name, resultMonitor.prefix + ".startTime",
-          DistributionType.MIN, -1));
-    long resultEnd =
-      getTimestampMetric(now,
-        getDistributionMetric(result, resultMonitor.name, resultMonitor.prefix + ".endTime",
-          DistributionType.MAX, -1));
-    long timestampStart =
-      getTimestampMetric(now,
-        getDistributionMetric(result,
-          resultMonitor.name, resultMonitor.prefix + ".startTimestamp",
-          DistributionType.MIN, -1));
-    long timestampEnd =
-      getTimestampMetric(now,
-        getDistributionMetric(result,
-          resultMonitor.name, resultMonitor.prefix + ".endTimestamp",
-          DistributionType.MAX, -1));
-
-    long effectiveEnd = -1;
-    if (eventEnd >= 0 && resultEnd >= 0) {
-      // It is possible for events to be generated after the last result was emitted.
-      // (Eg Query 2, which only yields results for a small prefix of the event stream.)
-      // So use the max of last event and last result times.
-      effectiveEnd = Math.max(eventEnd, resultEnd);
-    } else if (resultEnd >= 0) {
-      effectiveEnd = resultEnd;
-    } else if (eventEnd >= 0) {
-      // During startup we may have no result yet, but we would still like to track how
-      // long the pipeline has been running.
-      effectiveEnd = eventEnd;
-    }
-
-    if (effectiveEnd >= 0 && eventStart >= 0 && effectiveEnd >= eventStart) {
-      perf.runtimeSec = (effectiveEnd - eventStart) / 1000.0;
-    }
-
-    if (numEvents >= 0) {
-      perf.numEvents = numEvents;
-    }
-
-    if (numEvents >= 0 && perf.runtimeSec > 0.0) {
-      // For streaming we may later replace this with a 'steady-state' value calculated
-      // from the progress snapshots.
-      perf.eventsPerSec = numEvents / perf.runtimeSec;
-    }
-
-    if (numEventBytes >= 0 && perf.runtimeSec > 0.0) {
-      perf.eventBytesPerSec = numEventBytes / perf.runtimeSec;
-    }
-
-    if (numResults >= 0) {
-      perf.numResults = numResults;
-    }
-
-    if (numResults >= 0 && perf.runtimeSec > 0.0) {
-      perf.resultsPerSec = numResults / perf.runtimeSec;
-    }
-
-    if (numResultBytes >= 0 && perf.runtimeSec > 0.0) {
-      perf.resultBytesPerSec = numResultBytes / perf.runtimeSec;
-    }
-
-    if (eventStart >= 0) {
-      perf.startupDelaySec = (eventStart - startMsSinceEpoch) / 1000.0;
-    }
-
-    if (resultStart >= 0 && eventStart >= 0 && resultStart >= eventStart) {
-      perf.processingDelaySec = (resultStart - eventStart) / 1000.0;
-    }
-
-    if (timestampStart >= 0 && timestampEnd >= 0 && perf.runtimeSec > 0.0) {
-      double eventRuntimeSec = (timestampEnd - timestampStart) / 1000.0;
-      perf.timeDilation = eventRuntimeSec / perf.runtimeSec;
-    }
-
-    if (resultEnd >= 0) {
-      // Fill in the shutdown delay assuming the job has now finished.
-      perf.shutdownDelaySec = (now - resultEnd) / 1000.0;
-    }
-
-    // As soon as available, try to capture cumulative cost at this point too.
-
-    NexmarkPerf.ProgressSnapshot snapshot = new NexmarkPerf.ProgressSnapshot();
-    snapshot.secSinceStart = (now - startMsSinceEpoch) / 1000.0;
-    snapshot.runtimeSec = perf.runtimeSec;
-    snapshot.numEvents = numEvents;
-    snapshot.numResults = numResults;
-    snapshots.add(snapshot);
-
-    captureSteadyState(perf, snapshots);
-
-    return perf;
-  }
-
-  /**
-   * Build and run a pipeline using specified options.
-   */
-  interface PipelineBuilder<OptionT extends NexmarkOptions> {
-    void build(OptionT publishOnlyOptions);
-  }
-
-  /**
-   * Invoke the builder with options suitable for running a publish-only child pipeline.
-   */
-  private void invokeBuilderForPublishOnlyPipeline(PipelineBuilder<NexmarkOptions> builder) {
-    builder.build(options);
-//    throw new UnsupportedOperationException(
-//        "Cannot use --pubSubMode=COMBINED with DirectRunner");
-  }
-
-  /**
-   * If monitoring, wait until the publisher pipeline has run long enough to establish
-   * a backlog on the Pubsub topic. Otherwise, return immediately.
-   */
-  private void waitForPublisherPreload() {
-    throw new UnsupportedOperationException();
-  }
-
-  /**
-   * Monitor the performance and progress of a running job. Return final performance if
-   * it was measured.
-   */
-  @Nullable
-  private NexmarkPerf monitor(NexmarkQuery query) {
-    if (!options.getMonitorJobs()) {
-      return null;
-    }
-
-    if (configuration.debug) {
-      NexmarkUtils.console("Waiting for main pipeline to 'finish'");
-    } else {
-      NexmarkUtils.console("--debug=false, so job will not self-cancel");
-    }
-
-    PipelineResult job = mainResult;
-    PipelineResult publisherJob = publisherResult;
-    List<NexmarkPerf.ProgressSnapshot> snapshots = new ArrayList<>();
-    long startMsSinceEpoch = System.currentTimeMillis();
-    long endMsSinceEpoch = -1;
-    if (options.getRunningTimeMinutes() != null) {
-      endMsSinceEpoch = startMsSinceEpoch
-                        + Duration.standardMinutes(options.getRunningTimeMinutes()).getMillis()
-                        - Duration.standardSeconds(configuration.preloadSeconds).getMillis();
-    }
-    long lastActivityMsSinceEpoch = -1;
-    NexmarkPerf perf = null;
-    boolean waitingForShutdown = false;
-    boolean publisherCancelled = false;
-    List<String> errors = new ArrayList<>();
-
-    while (true) {
-      long now = System.currentTimeMillis();
-      if (endMsSinceEpoch >= 0 && now > endMsSinceEpoch && !waitingForShutdown) {
-        NexmarkUtils.console("Reached end of test, cancelling job");
-        try {
-          job.cancel();
-        } catch (IOException e) {
-          throw new RuntimeException("Unable to cancel main job: ", e);
-        }
-        if (publisherResult != null) {
-          try {
-            publisherJob.cancel();
-          } catch (IOException e) {
-            throw new RuntimeException("Unable to cancel publisher job: ", e);
-          }
-          publisherCancelled = true;
-        }
-        waitingForShutdown = true;
-      }
-
-      PipelineResult.State state = job.getState();
-      NexmarkUtils.console("%s %s%s", state, queryName,
-          waitingForShutdown ? " (waiting for shutdown)" : "");
-
-      NexmarkPerf currPerf;
-      if (configuration.debug) {
-        currPerf = currentPerf(startMsSinceEpoch, now, job, snapshots,
-                               query.eventMonitor, query.resultMonitor);
-      } else {
-        currPerf = null;
-      }
-
-      if (perf == null || perf.anyActivity(currPerf)) {
-        lastActivityMsSinceEpoch = now;
-      }
-
-      if (options.isStreaming() && !waitingForShutdown) {
-        Duration quietFor = new Duration(lastActivityMsSinceEpoch, now);
-        long fatalCount = getCounterMetric(job, query.getName(), "fatal", 0);
-        if (fatalCount > 0) {
-          NexmarkUtils.console("job has fatal errors, cancelling.");
-          errors.add(String.format("Pipeline reported %s fatal errors", fatalCount));
-          waitingForShutdown = true;
-        } else if (configuration.debug && configuration.numEvents > 0
-                   && currPerf.numEvents == configuration.numEvents
-                   && currPerf.numResults >= 0 && quietFor.isLongerThan(DONE_DELAY)) {
-          NexmarkUtils.console("streaming query appears to have finished, cancelling job.");
-          waitingForShutdown = true;
-        } else if (quietFor.isLongerThan(STUCK_TERMINATE_DELAY)) {
-          NexmarkUtils.console("streaming query appears to have gotten stuck, cancelling job.");
-          errors.add("Streaming job was cancelled since appeared stuck");
-          waitingForShutdown = true;
-        } else if (quietFor.isLongerThan(STUCK_WARNING_DELAY)) {
-          NexmarkUtils.console("WARNING: streaming query appears to have been stuck for %d min.",
-              quietFor.getStandardMinutes());
-          errors.add(
-              String.format("Streaming query was stuck for %d min", quietFor.getStandardMinutes()));
-        }
-
-        if (waitingForShutdown) {
-          try {
-            job.cancel();
-          } catch (IOException e) {
-            throw new RuntimeException("Unable to cancel main job: ", e);
-          }
-        }
-      }
-
-      perf = currPerf;
-
-      boolean running = true;
-      switch (state) {
-        case UNKNOWN:
-        case STOPPED:
-        case RUNNING:
-          // Keep going.
-          break;
-        case DONE:
-          // All done.
-          running = false;
-          break;
-        case CANCELLED:
-          running = false;
-          if (!waitingForShutdown) {
-            errors.add("Job was unexpectedly cancelled");
-          }
-          break;
-        case FAILED:
-        case UPDATED:
-          // Abnormal termination.
-          running = false;
-          errors.add("Job was unexpectedly updated");
-          break;
-      }
-
-      if (!running) {
-        break;
-      }
-
-      if (lastActivityMsSinceEpoch == now) {
-        NexmarkUtils.console("new perf %s", perf);
-      } else {
-        NexmarkUtils.console("no activity");
-      }
-
-      try {
-        Thread.sleep(PERF_DELAY.getMillis());
-      } catch (InterruptedException e) {
-        Thread.interrupted();
-        NexmarkUtils.console("Interrupted: pipeline is still running");
-      }
-    }
-
-    perf.errors = errors;
-    perf.snapshots = snapshots;
-
-    if (publisherResult != null) {
-      NexmarkUtils.console("Shutting down publisher pipeline.");
-      try {
-        if (!publisherCancelled) {
-          publisherJob.cancel();
-        }
-        publisherJob.waitUntilFinish(Duration.standardMinutes(5));
-      } catch (IOException e) {
-        throw new RuntimeException("Unable to cancel publisher job: ", e);
-      } //TODO Ismael
-//      catch (InterruptedException e) {
-//        Thread.interrupted();
-//        throw new RuntimeException("Interrupted: publish job still running.", e);
-//      }
-    }
-
-    return perf;
-  }
-
-  // ================================================================================
-  // Basic sources and sinks
-  // ================================================================================
-
-  /**
-   * Return a topic name.
-   */
-  private String shortTopic(long now) {
-    String baseTopic = options.getPubsubTopic();
-    if (Strings.isNullOrEmpty(baseTopic)) {
-      throw new RuntimeException("Missing --pubsubTopic");
-    }
-    switch (options.getResourceNameMode()) {
-      case VERBATIM:
-        return baseTopic;
-      case QUERY:
-        return String.format("%s_%s_source", baseTopic, queryName);
-      case QUERY_AND_SALT:
-        return String.format("%s_%s_%d_source", baseTopic, queryName, now);
-    }
-    throw new RuntimeException("Unrecognized enum " + options.getResourceNameMode());
-  }
-
-  /**
-   * Return a subscription name.
-   */
-  private String shortSubscription(long now) {
-    String baseSubscription = options.getPubsubSubscription();
-    if (Strings.isNullOrEmpty(baseSubscription)) {
-      throw new RuntimeException("Missing --pubsubSubscription");
-    }
-    switch (options.getResourceNameMode()) {
-      case VERBATIM:
-        return baseSubscription;
-      case QUERY:
-        return String.format("%s_%s_source", baseSubscription, queryName);
-      case QUERY_AND_SALT:
-        return String.format("%s_%s_%d_source", baseSubscription, queryName, now);
-    }
-    throw new RuntimeException("Unrecognized enum " + options.getResourceNameMode());
-  }
-
-  /**
-   * Return a file name for plain text.
-   */
-  private String textFilename(long now) {
-    String baseFilename = options.getOutputPath();
-    if (Strings.isNullOrEmpty(baseFilename)) {
-      throw new RuntimeException("Missing --outputPath");
-    }
-    switch (options.getResourceNameMode()) {
-      case VERBATIM:
-        return baseFilename;
-      case QUERY:
-        return String.format("%s/nexmark_%s.txt", baseFilename, queryName);
-      case QUERY_AND_SALT:
-        return String.format("%s/nexmark_%s_%d.txt", baseFilename, queryName, now);
-    }
-    throw new RuntimeException("Unrecognized enum " + options.getResourceNameMode());
-  }
-
-  /**
-   * Return a BigQuery table spec.
-   */
-  private String tableSpec(long now, String version) {
-    String baseTableName = options.getBigQueryTable();
-    if (Strings.isNullOrEmpty(baseTableName)) {
-      throw new RuntimeException("Missing --bigQueryTable");
-    }
-    switch (options.getResourceNameMode()) {
-      case VERBATIM:
-        return String.format("%s:nexmark.%s_%s",
-                             options.getProject(), baseTableName, version);
-      case QUERY:
-        return String.format("%s:nexmark.%s_%s_%s",
-                             options.getProject(), baseTableName, queryName, version);
-      case QUERY_AND_SALT:
-        return String.format("%s:nexmark.%s_%s_%s_%d",
-                             options.getProject(), baseTableName, queryName, version, now);
-    }
-    throw new RuntimeException("Unrecognized enum " + options.getResourceNameMode());
-  }
-
-  /**
-   * Return a directory for logs.
-   */
-  private String logsDir(long now) {
-    String baseFilename = options.getOutputPath();
-    if (Strings.isNullOrEmpty(baseFilename)) {
-      throw new RuntimeException("Missing --outputPath");
-    }
-    switch (options.getResourceNameMode()) {
-      case VERBATIM:
-        return baseFilename;
-      case QUERY:
-        return String.format("%s/logs_%s", baseFilename, queryName);
-      case QUERY_AND_SALT:
-        return String.format("%s/logs_%s_%d", baseFilename, queryName, now);
-    }
-    throw new RuntimeException("Unrecognized enum " + options.getResourceNameMode());
-  }
-
-  /**
-   * Return a source of synthetic events.
-   */
-  private PCollection<Event> sourceEventsFromSynthetic(Pipeline p) {
-    if (isStreaming()) {
-      NexmarkUtils.console("Generating %d events in streaming mode", configuration.numEvents);
-      return p.apply(queryName + ".ReadUnbounded", NexmarkUtils.streamEventsSource(configuration));
-    } else {
-      NexmarkUtils.console("Generating %d events in batch mode", configuration.numEvents);
-      return p.apply(queryName + ".ReadBounded", NexmarkUtils.batchEventsSource(configuration));
-    }
-  }
-
-  /**
-   * Return source of events from Pubsub.
-   */
-  private PCollection<Event> sourceEventsFromPubsub(Pipeline p, long now) {
-    String shortSubscription = shortSubscription(now);
-    NexmarkUtils.console("Reading events from Pubsub %s", shortSubscription);
-
-    PubsubIO.Read<PubsubMessage> io =
-        PubsubIO.readPubsubMessagesWithAttributes().fromSubscription(shortSubscription)
-            .withIdAttribute(NexmarkUtils.PUBSUB_ID);
-    if (!configuration.usePubsubPublishTime) {
-      io = io.withTimestampAttribute(NexmarkUtils.PUBSUB_TIMESTAMP);
-    }
-
-    return p
-      .apply(queryName + ".ReadPubsubEvents", io)
-      .apply(queryName + ".PubsubMessageToEvent", ParDo.of(new DoFn<PubsubMessage, Event>() {
-        @ProcessElement
-        public void processElement(ProcessContext c) {
-          byte[] payload = c.element().getPayload();
-          try {
-            Event event = CoderUtils.decodeFromByteArray(Event.CODER, payload);
-            c.output(event);
-          } catch (CoderException e) {
-            // TODO Log decoding Event error
-          }
-        }
-      }));
-  }
-
-  /**
-   * Return Avro source of events from {@code options.getInputFilePrefix}.
-   */
-  private PCollection<Event> sourceEventsFromAvro(Pipeline p) {
-    String filename = options.getInputPath();
-    if (Strings.isNullOrEmpty(filename)) {
-      throw new RuntimeException("Missing --inputPath");
-    }
-    NexmarkUtils.console("Reading events from Avro files at %s", filename);
-    return p
-        .apply(queryName + ".ReadAvroEvents", AvroIO.read(Event.class)
-                          .from(filename + "*.avro"))
-        .apply("OutputWithTimestamp", NexmarkQuery.EVENT_TIMESTAMP_FROM_DATA);
-  }
-
-  /**
-   * Send {@code events} to Pubsub.
-   */
-  private void sinkEventsToPubsub(PCollection<Event> events, long now) {
-    String shortTopic = shortTopic(now);
-    NexmarkUtils.console("Writing events to Pubsub %s", shortTopic);
-
-    PubsubIO.Write<PubsubMessage> io =
-        PubsubIO.writePubsubMessages().to(shortTopic)
-            .withIdAttribute(NexmarkUtils.PUBSUB_ID);
-    if (!configuration.usePubsubPublishTime) {
-      io = io.withTimestampAttribute(NexmarkUtils.PUBSUB_TIMESTAMP);
-    }
-
-    events.apply(queryName + ".EventToPubsubMessage",
-            ParDo.of(new DoFn<Event, PubsubMessage>() {
-              @ProcessElement
-              public void processElement(ProcessContext c) {
-                try {
-                  byte[] payload = CoderUtils.encodeToByteArray(Event.CODER, c.element());
-                  c.output(new PubsubMessage(payload, new HashMap<String, String>()));
-                } catch (CoderException e1) {
-                  // TODO Log encoding Event error
-                }
-              }
-            })
-        )
-        .apply(queryName + ".WritePubsubEvents", io);
-  }
-
-  /**
-   * Send {@code formattedResults} to Pubsub.
-   */
-  private void sinkResultsToPubsub(PCollection<String> formattedResults, long now) {
-    String shortTopic = shortTopic(now);
-    NexmarkUtils.console("Writing results to Pubsub %s", shortTopic);
-    PubsubIO.Write<String> io =
-        PubsubIO.writeStrings().to(shortTopic)
-            .withIdAttribute(NexmarkUtils.PUBSUB_ID);
-    if (!configuration.usePubsubPublishTime) {
-      io = io.withTimestampAttribute(NexmarkUtils.PUBSUB_TIMESTAMP);
-    }
-    formattedResults.apply(queryName + ".WritePubsubResults", io);
-  }
-
-  /**
-   * Sink all raw Events in {@code source} to {@code options.getOutputPath}.
-   * This will configure the job to write the following files:
-   * <ul>
-   * <li>{@code $outputPath/event*.avro} All Event entities.
-   * <li>{@code $outputPath/auction*.avro} Auction entities.
-   * <li>{@code $outputPath/bid*.avro} Bid entities.
-   * <li>{@code $outputPath/person*.avro} Person entities.
-   * </ul>
-   *
-   * @param source A PCollection of events.
-   */
-  private void sinkEventsToAvro(PCollection<Event> source) {
-    String filename = options.getOutputPath();
-    if (Strings.isNullOrEmpty(filename)) {
-      throw new RuntimeException("Missing --outputPath");
-    }
-    NexmarkUtils.console("Writing events to Avro files at %s", filename);
-    source.apply(queryName + ".WriteAvroEvents",
-            AvroIO.write(Event.class).to(filename + "/event").withSuffix(".avro"));
-    source.apply(NexmarkQuery.JUST_BIDS)
-          .apply(queryName + ".WriteAvroBids",
-            AvroIO.write(Bid.class).to(filename + "/bid").withSuffix(".avro"));
-    source.apply(NexmarkQuery.JUST_NEW_AUCTIONS)
-          .apply(queryName + ".WriteAvroAuctions",
-            AvroIO.write(Auction.class).to(filename + "/auction").withSuffix(".avro"));
-    source.apply(NexmarkQuery.JUST_NEW_PERSONS)
-          .apply(queryName + ".WriteAvroPeople",
-            AvroIO.write(Person.class).to(filename + "/person").withSuffix(".avro"));
-  }
-
-  /**
-   * Send {@code formattedResults} to text files.
-   */
-  private void sinkResultsToText(PCollection<String> formattedResults, long now) {
-    String filename = textFilename(now);
-    NexmarkUtils.console("Writing results to text files at %s", filename);
-    formattedResults.apply(queryName + ".WriteTextResults",
-        TextIO.write().to(filename));
-  }
-
-  private static class StringToTableRow extends DoFn<String, TableRow> {
-    @ProcessElement
-    public void processElement(ProcessContext c) {
-      int n = ThreadLocalRandom.current().nextInt(10);
-      List<TableRow> records = new ArrayList<>(n);
-      for (int i = 0; i < n; i++) {
-        records.add(new TableRow().set("index", i).set("value", Integer.toString(i)));
-      }
-      c.output(new TableRow().set("result", c.element()).set("records", records));
-    }
-  }
-
-  /**
-   * Send {@code formattedResults} to BigQuery.
-   */
-  private void sinkResultsToBigQuery(
-      PCollection<String> formattedResults, long now,
-      String version) {
-    String tableSpec = tableSpec(now, version);
-    TableSchema tableSchema =
-        new TableSchema().setFields(ImmutableList.of(
-            new TableFieldSchema().setName("result").setType("STRING"),
-            new TableFieldSchema().setName("records").setMode("REPEATED").setType("RECORD")
-                                  .setFields(ImmutableList.of(
-                                      new TableFieldSchema().setName("index").setType("INTEGER"),
-                                      new TableFieldSchema().setName("value").setType("STRING")))));
-    NexmarkUtils.console("Writing results to BigQuery table %s", tableSpec);
-    BigQueryIO.Write io =
-        BigQueryIO.write().to(tableSpec)
-                        .withSchema(tableSchema)
-                        .withCreateDisposition(BigQueryIO.Write.CreateDisposition.CREATE_IF_NEEDED)
-                        .withWriteDisposition(BigQueryIO.Write.WriteDisposition.WRITE_APPEND);
-    formattedResults
-        .apply(queryName + ".StringToTableRow", ParDo.of(new StringToTableRow()))
-        .apply(queryName + ".WriteBigQueryResults", io);
-  }
-
-  // ================================================================================
-  // Construct overall pipeline
-  // ================================================================================
-
-  /**
-   * Return source of events for this run, or null if we are simply publishing events
-   * to Pubsub.
-   */
-  private PCollection<Event> createSource(Pipeline p, final long now) {
-    PCollection<Event> source = null;
-    switch (configuration.sourceType) {
-      case DIRECT:
-        source = sourceEventsFromSynthetic(p);
-        break;
-      case AVRO:
-        source = sourceEventsFromAvro(p);
-        break;
-      case PUBSUB:
-        // Setup the sink for the publisher.
-        switch (configuration.pubSubMode) {
-          case SUBSCRIBE_ONLY:
-            // Nothing to publish.
-            break;
-          case PUBLISH_ONLY:
-            // Send synthesized events to Pubsub in this job.
-            sinkEventsToPubsub(sourceEventsFromSynthetic(p).apply(queryName + ".Snoop",
-                    NexmarkUtils.snoop(queryName)), now);
-            break;
-          case COMBINED:
-            // Send synthesized events to Pubsub in separate publisher job.
-            // We won't start the main pipeline until the publisher has sent the pre-load events.
-            // We'll shutdown the publisher job when we notice the main job has finished.
-            invokeBuilderForPublishOnlyPipeline(new PipelineBuilder<NexmarkOptions>() {
-              @Override
-              public void build(NexmarkOptions publishOnlyOptions) {
-                Pipeline sp = Pipeline.create(options);
-                NexmarkUtils.setupPipeline(configuration.coderStrategy, sp);
-                publisherMonitor = new Monitor<>(queryName, "publisher");
-                sinkEventsToPubsub(
-                    sourceEventsFromSynthetic(sp)
-                            .apply(queryName + ".Monitor", publisherMonitor.getTransform()),
-                    now);
-                publisherResult = sp.run();
-              }
-            });
-            break;
-        }
-
-        // Setup the source for the consumer.
-        switch (configuration.pubSubMode) {
-          case PUBLISH_ONLY:
-            // Nothing to consume. Leave source null.
-            break;
-          case SUBSCRIBE_ONLY:
-          case COMBINED:
-            // Read events from pubsub.
-            source = sourceEventsFromPubsub(p, now);
-            break;
-        }
-        break;
-    }
-    return source;
-  }
-
-  private static final TupleTag<String> MAIN = new TupleTag<String>(){};
-  private static final TupleTag<String> SIDE = new TupleTag<String>(){};
-
-  private static class PartitionDoFn extends DoFn<String, String> {
-    @ProcessElement
-    public void processElement(ProcessContext c) {
-      if (c.element().hashCode() % 2 == 0) {
-        c.output(c.element());
-      } else {
-        c.output(SIDE, c.element());
-      }
-    }
-  }
-
-  /**
-   * Consume {@code results}.
-   */
-  private void sink(PCollection<TimestampedValue<KnownSize>> results, long now) {
-    if (configuration.sinkType == NexmarkUtils.SinkType.COUNT_ONLY) {
-      // Avoid the cost of formatting the results.
-      results.apply(queryName + ".DevNull", NexmarkUtils.devNull(queryName));
-      return;
-    }
-
-    PCollection<String> formattedResults =
-      results.apply(queryName + ".Format", NexmarkUtils.format(queryName));
-    if (options.getLogResults()) {
-      formattedResults = formattedResults.apply(queryName + ".Results.Log",
-              NexmarkUtils.<String>log(queryName + ".Results"));
-    }
-
-    switch (configuration.sinkType) {
-      case DEVNULL:
-        // Discard all results
-        formattedResults.apply(queryName + ".DevNull", NexmarkUtils.devNull(queryName));
-        break;
-      case PUBSUB:
-        sinkResultsToPubsub(formattedResults, now);
-        break;
-      case TEXT:
-        sinkResultsToText(formattedResults, now);
-        break;
-      case AVRO:
-        NexmarkUtils.console(
-            "WARNING: with --sinkType=AVRO, actual query results will be discarded.");
-        break;
-      case BIGQUERY:
-        // Multiple BigQuery backends to mimic what most customers do.
-        PCollectionTuple res = formattedResults.apply(queryName + ".Partition",
-            ParDo.of(new PartitionDoFn()).withOutputTags(MAIN, TupleTagList.of(SIDE)));
-        sinkResultsToBigQuery(res.get(MAIN), now, "main");
-        sinkResultsToBigQuery(res.get(SIDE), now, "side");
-        sinkResultsToBigQuery(formattedResults, now, "copy");
-        break;
-      case COUNT_ONLY:
-        // Short-circuited above.
-        throw new RuntimeException();
-    }
-  }
-
-  // ================================================================================
-  // Entry point
-  // ================================================================================
-
-  /**
-   * Calculate the distribution of the expected rate of results per minute (in event time, not
-   * wallclock time).
-   */
-  private void modelResultRates(NexmarkQueryModel model) {
-    List<Long> counts = Lists.newArrayList(model.simulator().resultsPerWindow());
-    Collections.sort(counts);
-    int n = counts.size();
-    if (n < 5) {
-      NexmarkUtils.console("Query%d: only %d samples", model.configuration.query, n);
-    } else {
-      NexmarkUtils.console("Query%d: N:%d; min:%d; 1st%%:%d; mean:%d; 3rd%%:%d; max:%d",
-                           model.configuration.query, n, counts.get(0), counts.get(n / 4),
-                           counts.get(n / 2),
-                           counts.get(n - 1 - n / 4), counts.get(n - 1));
-    }
-  }
-
-  /**
-   * Run {@code configuration} and return its performance if possible.
-   */
-  @Nullable
-  public NexmarkPerf run(NexmarkConfiguration runConfiguration) {
-    if (options.getManageResources() && !options.getMonitorJobs()) {
-      throw new RuntimeException("If using --manageResources then must also use --monitorJobs.");
-    }
-
-    //
-    // Setup per-run state.
-    //
-    checkState(configuration == null);
-    checkState(queryName == null);
-    configuration = runConfiguration;
-
-    try {
-      NexmarkUtils.console("Running %s", configuration.toShortString());
-
-      if (configuration.numEvents < 0) {
-        NexmarkUtils.console("skipping since configuration is disabled");
-        return null;
-      }
-
-      List<NexmarkQuery> queries = Arrays.asList(new Query0(configuration),
-                                                 new Query1(configuration),
-                                                 new Query2(configuration),
-                                                 new Query3(configuration),
-                                                 new Query4(configuration),
-                                                 new Query5(configuration),
-                                                 new Query6(configuration),
-                                                 new Query7(configuration),
-                                                 new Query8(configuration),
-                                                 new Query9(configuration),
-                                                 new Query10(configuration),
-                                                 new Query11(configuration),
-                                                 new Query12(configuration));
-      NexmarkQuery query = queries.get(configuration.query);
-      queryName = query.getName();
-
-      List<NexmarkQueryModel> models = Arrays.asList(
-          new Query0Model(configuration),
-          new Query1Model(configuration),
-          new Query2Model(configuration),
-          new Query3Model(configuration),
-          new Query4Model(configuration),
-          new Query5Model(configuration),
-          new Query6Model(configuration),
-          new Query7Model(configuration),
-          new Query8Model(configuration),
-          new Query9Model(configuration),
-          null,
-          null,
-          null);
-      NexmarkQueryModel model = models.get(configuration.query);
-
-      if (options.getJustModelResultRate()) {
-        if (model == null) {
-          throw new RuntimeException(String.format("No model for %s", queryName));
-        }
-        modelResultRates(model);
-        return null;
-      }
-
-      long now = System.currentTimeMillis();
-      Pipeline p = Pipeline.create(options);
-      NexmarkUtils.setupPipeline(configuration.coderStrategy, p);
-
-      // Generate events.
-      PCollection<Event> source = createSource(p, now);
-
-      if (options.getLogEvents()) {
-        source = source.apply(queryName + ".Events.Log",
-                NexmarkUtils.<Event>log(queryName + ".Events"));
-      }
-
-      // Source will be null if source type is PUBSUB and mode is PUBLISH_ONLY.
-      // In that case there's nothing more to add to pipeline.
-      if (source != null) {
-        // Optionally sink events in Avro format.
-        // (Query results are ignored).
-        if (configuration.sinkType == NexmarkUtils.SinkType.AVRO) {
-          sinkEventsToAvro(source);
-        }
-
-        // Special hacks for Query 10 (big logger).
-        if (configuration.query == 10) {
-          String path = null;
-          if (options.getOutputPath() != null && !options.getOutputPath().isEmpty()) {
-            path = logsDir(now);
-          }
-          ((Query10) query).setOutputPath(path);
-          ((Query10) query).setMaxNumWorkers(maxNumWorkers());
-        }
-
-        // Apply query.
-        PCollection<TimestampedValue<KnownSize>> results = source.apply(query);
-
-        if (options.getAssertCorrectness()) {
-          if (model == null) {
-            throw new RuntimeException(String.format("No model for %s", queryName));
-          }
-          // We know all our streams have a finite number of elements.
-          results.setIsBoundedInternal(PCollection.IsBounded.BOUNDED);
-          // If we have a finite number of events then assert our pipeline's
-          // results match those of a model using the same sequence of events.
-          PAssert.that(results).satisfies(model.assertionFor());
-        }
-
-        // Output results.
-        sink(results, now);
-      }
-
-      if (publisherResult != null) {
-        waitForPublisherPreload();
-      }
-      mainResult = p.run();
-      mainResult.waitUntilFinish(Duration.standardSeconds(configuration.streamTimeout));
-      return monitor(query);
-    } finally {
-      configuration = null;
-      queryName = null;
-    }
-  }
-}


[40/55] [abbrv] beam git commit: Update to Beam 2.2.0-SNAPSHOT and final touches before merge into master

Posted by ie...@apache.org.
Update to Beam 2.2.0-SNAPSHOT and final touches before merge into master

Clean pom, exclude nexmark packages from aggregated javadoc, put spark logs in WARN

Update execution matrix in README: Flink termination of streaming pipelines is now ok as far as Nexmark is concerned

Remove how to run Nexmark on Flink on Google Cloud Platform from README

Update command lines in README after moving nexmark from integration to sdks module


Project: http://git-wip-us.apache.org/repos/asf/beam/repo
Commit: http://git-wip-us.apache.org/repos/asf/beam/commit/0acd7201
Tree: http://git-wip-us.apache.org/repos/asf/beam/tree/0acd7201
Diff: http://git-wip-us.apache.org/repos/asf/beam/diff/0acd7201

Branch: refs/heads/master
Commit: 0acd72016b046640dd753d60a569403488f595a5
Parents: f4333df
Author: Etienne Chauchot <ec...@gmail.com>
Authored: Fri Aug 11 16:41:00 2017 +0200
Committer: Ismaël Mejía <ie...@gmail.com>
Committed: Wed Aug 23 19:07:29 2017 +0200

----------------------------------------------------------------------
 .../java/nexmark/BEAM_ON_FLINK_ON_GCP.md        | 282 -------------------
 sdks/java/javadoc/ant.xml                       |   1 +
 sdks/java/nexmark/README.md                     |  56 ++--
 sdks/java/nexmark/pom.xml                       |  35 +--
 .../apache/beam/sdk/nexmark/package-info.java   |   2 +-
 .../nexmark/src/main/resources/log4j.properties |   4 +-
 6 files changed, 34 insertions(+), 346 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/beam/blob/0acd7201/integration/java/nexmark/BEAM_ON_FLINK_ON_GCP.md
----------------------------------------------------------------------
diff --git a/integration/java/nexmark/BEAM_ON_FLINK_ON_GCP.md b/integration/java/nexmark/BEAM_ON_FLINK_ON_GCP.md
deleted file mode 100644
index 6a7fd34..0000000
--- a/integration/java/nexmark/BEAM_ON_FLINK_ON_GCP.md
+++ /dev/null
@@ -1,282 +0,0 @@
-<!--
-    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.
--->
-
-# Running NexMark on Beam on Flink on Google Compute Platform
-
-Here's how to create a cluster of VMs on Google Compute Platform, deploy
-Flink to them, and invoke a NexMark pipeline using the Beam-on-Flink
-runner.
-
-These instructions are somewhat baroque and I hope they can be
-simplified over time.
-
-## Prerequisites
-
-You'll need:
-
-* the Google Cloud SDK
-* a clone of the Beam repository
-* a Flink binary distribution
-* a project on Google Compute Platform.
-
-## Establish the shell environment
-
-```
-# Beam root
-BEAM=<path to Beam source directory>
-# Flink root
-FLINK_VER=flink-1.0.3
-FLINK=<path to Flink distribution directory>
-# Google Cloud project
-PROJECT=<your project id>
-# Google Cloud zone
-ZONE=<your project zone>
-# Cloud commands
-GCLOUD=<path to gcloud command>
-GSUTIL=<path to gsutil command>
-```
-
-## Establish VM names for Flink master and workers
-
-```
-MASTER=flink-m
-NUM_WORKERS=5
-WORKERS=""
-for (( i = 0; i < $NUM_WORKERS; i++ )); do
-  WORKERS="$WORKERS flink-w-$i"
-done
-ALL="$MASTER $WORKERS"
-```
-
-## Build Beam
-
-```
-( cd $BEAM && mvn clean install )
-```
-
-## Bring up the cluster
-
-Establish project defaults and authenticate:
-```
-$GCLOUD init
-$GCLOUD auth login
-```
-
-Build Google Cloud Dataproc cluster:
-```
-$GCLOUD beta dataproc clusters create \
-  --project=$PROJECT \
-  --zone=$ZONE \
-  --bucket=nexmark \
-  --scopes=cloud-platform \
-  --num-workers=$NUM_WORKERS \
-  --image-version=preview \
-  flink
-```
-
-Force google_compute_engine ssh keys to be generated locally:
-```
-$GCLOUD compute ssh \
-  --project=$PROJECT \
-  --zone=$ZONE \
-  $MASTER \
-  --command 'exit'
-```
-
-Open ports on the VMs:
-```
-$GCLOUD compute firewall-rules create allow-monitoring --allow tcp:8080-8081
-$GCLOUD compute firewall-rules create allow-debug --allow tcp:5555
-```
-
-Establish keys on master and workers
-**CAUTION:** This will leave your private key on your master VM.
-Better would be to create a key just for inter-worker ssh.
-```
-for m in $ALL; do
-  echo "*** $m ***"
-  $GCLOUD beta compute scp \
-    --project=$PROJECT \
-    --zone=$ZONE \
-    ~/.ssh/google_compute_engine.pub $m:~/.ssh/
-done
-$GCLOUD beta compute scp \
-  --project=$PROJECT \
-  --zone=$ZONE \
-  ~/.ssh/google_compute_engine $MASTER:~/.ssh/
-```
-
-Collect IP addresses for workers:
-```
-MASTER_EXT_IP=$($GCLOUD compute instances describe \
- --project=$PROJECT \
-  --zone=$ZONE \
-  $MASTER | grep natIP: | sed 's/[ ]*natIP:[ ]*//')
-MASTER_IP=$($GCLOUD compute instances describe \
- --project=$PROJECT \
-  --zone=$ZONE \
-  $MASTER | grep networkIP: | sed 's/[ ]*networkIP:[ ]*//')
-WORKER_IPS=""
-for m in $WORKERS; do
-  echo "*** $m ***"
-  WORKER_IP=$($GCLOUD compute instances describe \
-    --project=$PROJECT \
-    --zone=$ZONE \
-    $m | grep networkIP: | sed 's/[ ]*networkIP:[ ]*//')
-  WORKER_IPS="$WORKER_IPS $WORKER_IP"
-done
-```
-
-Configure Flink:
-```
-cat $FLINK/conf/flink-conf.yaml \
-  | sed "s|.*\(jobmanager.rpc.address\):.*|\1: $MASTER_IP|g" \
-  | sed "s|.*\(jobmanager.heap.mb\):.*|\1: 4096|g" \
-  | sed "s|.*\(taskmanager.heap.mb\):.*|\1: 8192|g" \
-  | sed "s|.*\(parallelism.default\):.*|\1: $(($NUM_WORKERS * 4))|g" \
-  | sed "s|.*\(fs.hdfs.hadoopconf\):.*|\1: /etc/hadoop/conf|g" \
-  | sed "s|.*\(taskmanager.numberOfTaskSlots\):.*|\1: 4|g" \
-  | sed "s|.*\(jobmanager.web.submit.enable\):.*|\1: false|g" \
-  | sed "s|.*\(env.ssh.opts\):.*||g" \
-  > ~/flink-conf.yaml
-cat $FLINK/conf/log4j.properties \
-  | sed "s|.*\(log4j.rootLogger\)=.*|\1=ERROR, file|g" \
-  > ~/log4j.properties
-echo "env.ssh.opts: -i /home/$USER/.ssh/google_compute_engine -o StrictHostKeyChecking=no" >> ~/flink-conf.yaml
-echo "$MASTER_IP:8081" > ~/masters
-echo -n > ~/slaves
-for ip in $WORKER_IPS; do
-  echo $ip >> ~/slaves
-done
-cp -f \
-  ~/flink-conf.yaml \
-  ~/masters ~/slaves \
-  ~/log4j.properties \
-  $FLINK/conf/
-```
-
-Package configured Flink for distribution to workers:
-```
-( cd ~/ && tar -cvzf ~/flink.tgz $FLINK/* )
-```
-
-Distribute:
-```
-$GSUTIL cp ~/flink.tgz gs://nexmark
-for m in $ALL; do
-  echo "*** $m ***"
-  $GCLOUD compute ssh \
-    --project=$PROJECT \
-    --zone=$ZONE \
-    $m \
-    --command 'gsutil cp gs://nexmark/flink.tgz ~/ && tar -xvzf ~/flink.tgz'
-done
-```
-
-Start the Flink cluster:
-```
-$GCLOUD compute ssh \
-  --project=$PROJECT \
-  --zone=$ZONE \
-  $MASTER \
-  --command "~/$FLINK_VER/bin/start-cluster.sh"
-```
-
-Bring up the Flink monitoring UI:
-```
-/usr/bin/google-chrome $MASTER_EXT_IP:8081 &
-```
-
-## Run NexMark
-
-Distribute the Beam + NexMark jar to all workers:
-```
-$GSUTIL cp $BEAM/integration/java/target/java-integration-all-bundled-0.2.0-incubating-SNAPSHOT.jar gs://nexmark
-for m in $ALL; do
-  echo "*** $m ***"
-  $GCLOUD compute ssh \
-    --project=$PROJECT \
-    --zone=$ZONE \
-    $m \
-    --command "gsutil cp gs://nexmark/java-integration-all-bundled-0.2.0-incubating-SNAPSHOT.jar ~/$FLINK_VER/lib/"
-done
-```
-
-Create a Pubsub topic and subscription for testing:
-```
-$GCLOUD alpha pubsub \
-  --project=$PROJECT \
-  topics create flink_test
-
-$GCLOUD alpha pubsub \
-  --project=$PROJECT \
-  subscriptions create flink_test \
-  --topic flink_test \
-  --ack-deadline=60 \
-  --topic-project=$PROJECT
-```
-
-Launch!
-**NOTE:** As of flink-1.0.3 this will throw a `NullPointerException`
-in `org.apache.beam.sdk.io.PubsubUnboundedSink$WriterFn.startBundle`.
-See Jira issue [BEAM-196](https://issues.apache.org/jira/browse/BEAM-196).
-
-```
-$GCLOUD compute ssh \
-  --project=$PROJECT \
-  --zone=$ZONE \
-  $MASTER \
-  --command "~/$FLINK_VER/bin/flink run \
-  -c org.apache.beam.integration.nexmark.drivers.NexmarkFlinkDriver \
-  ~/$FLINK_VER/lib/java-integration-all-bundled-0.2.0-incubating-SNAPSHOT.jar \
-  --project=$PROJECT \
-  --streaming=true \
-  --query=0 \
-  --sourceType=PUBSUB \
-  --pubSubMode=COMBINED \
-  --pubsubTopic=flink_test \
-  --resourceNameMode=VERBATIM \
-  --manageResources=false \
-  --monitorJobs=false \
-  --numEventGenerators=5 \
-  --firstEventRate=1000 \
-  --nextEventRate=1000 \
-  --isRateLimited=true \
-  --numEvents=0 \
-  --useWallclockEventTime=true \
-  --usePubsubPublishTime=true"
-```
-
-## Teardown the cluster
-
-Stop the Flink cluster:
-```
-$GCLOUD compute ssh \
-  --project=$PROJECT \
-  --zone=$ZONE \
-  $MASTER \
-  --command "~/$FLINK_VER/bin/stop-cluster.sh"
-```
-
-Teardown the Dataproc cluster:
-```
-$GCLOUD beta dataproc clusters delete \
-  --project=$PROJECT \
-  flink
-```

http://git-wip-us.apache.org/repos/asf/beam/blob/0acd7201/sdks/java/javadoc/ant.xml
----------------------------------------------------------------------
diff --git a/sdks/java/javadoc/ant.xml b/sdks/java/javadoc/ant.xml
index 9a1880a..152b9dd 100644
--- a/sdks/java/javadoc/ant.xml
+++ b/sdks/java/javadoc/ant.xml
@@ -30,6 +30,7 @@
       <fileset dir="..">
         <include name="**/src/main/java/**/*.java"/>
         <exclude name="**/maven-archetypes/**"/>
+        <exclude name="**/nexmark/**"/>
       </fileset>
       <!-- For each pathname, turn X/src/main/java/Y to Y. This
            results in one Java source tree. -->

http://git-wip-us.apache.org/repos/asf/beam/blob/0acd7201/sdks/java/nexmark/README.md
----------------------------------------------------------------------
diff --git a/sdks/java/nexmark/README.md b/sdks/java/nexmark/README.md
index a9acd63..f252943 100644
--- a/sdks/java/nexmark/README.md
+++ b/sdks/java/nexmark/README.md
@@ -17,7 +17,7 @@
     under the License.
 -->
 
-# NEXMark integration suite
+# NEXMark test suite
 
 This is a suite of pipelines inspired by the 'continuous data stream'
 queries in [http://datalab.cs.pdx.edu/niagaraST/NEXMark/]
@@ -139,7 +139,7 @@ Available suites are:
     --runner=DataflowRunner \
     --tempLocation=gs://talend-imejia/nexmark/temp/ \
     --stagingLocation=gs://talend-imejia/nexmark/temp/staging/ \
-    --filesToStage=target/beam-integration-java-0.7.0-SNAPSHOT.jar
+    --filesToStage=target/beam-sdks-java-nexmark-2.1.0-SNAPSHOT.jar
 
 ## Direct specific configuration
 
@@ -183,19 +183,19 @@ Open issues are tracked [here](https://github.com../../../../../issues):
 
 | Query | Direct | Spark                                                        | Flink                                                      | Apex                                                         |
 | ----: | ------ | ------------------------------------------------------------ | ---------------------------------------------------------- | ------------------------------------------------------------ |
-|     0 | ok     | ok                                                           | [BEAM-593](https://issues.apache.org/jira/browse/BEAM-593) | ok                                                           |
-|     1 | ok     | ok                                                           | [BEAM-593](https://issues.apache.org/jira/browse/BEAM-593) | ok                                                           |
-|     2 | ok     | ok                                                           | [BEAM-593](https://issues.apache.org/jira/browse/BEAM-593) | ok                                                           |
-|     3 | ok     | [BEAM-1035](https://issues.apache.org/jira/browse/BEAM-1035) | [BEAM-593](https://issues.apache.org/jira/browse/BEAM-593) | [BEAM-1037](https://issues.apache.org/jira/browse/BEAM-1037) |
-|     4 | ok     | ok                                                           | [BEAM-593](https://issues.apache.org/jira/browse/BEAM-593) | ok                                                           |
-|     5 | ok     | ok                                                           | [BEAM-593](https://issues.apache.org/jira/browse/BEAM-593) | ok                                                           |
-|     6 | ok     | ok                                                           | [BEAM-593](https://issues.apache.org/jira/browse/BEAM-593) | ok                                                           |
-|     7 | ok     | [BEAM-2112](https://issues.apache.org/jira/browse/BEAM-2112) | [BEAM-593](https://issues.apache.org/jira/browse/BEAM-593) | ok                                                           |
-|     8 | ok     | ok                                                           | [BEAM-593](https://issues.apache.org/jira/browse/BEAM-593) | ok                                                           |
-|     9 | ok     | ok                                                           | [BEAM-593](https://issues.apache.org/jira/browse/BEAM-593) | ok                                                           |
-|    10 | ok     | ok                                                           | [BEAM-593](https://issues.apache.org/jira/browse/BEAM-593) | ok                                                           |
-|    11 | ok     | ok                                                           | [BEAM-593](https://issues.apache.org/jira/browse/BEAM-593) | ok                                                           |
-|    12 | ok     | ok                                                           | [BEAM-593](https://issues.apache.org/jira/browse/BEAM-593) | ok                                                           |
+|     0 | ok     | ok                                                           | ok                                                         | ok                                                           |
+|     1 | ok     | ok                                                           | ok                                                         | ok                                                           |
+|     2 | ok     | ok                                                           | ok                                                         | ok                                                           |
+|     3 | ok     | [BEAM-1035](https://issues.apache.org/jira/browse/BEAM-1035) | ok                                                         | [BEAM-1037](https://issues.apache.org/jira/browse/BEAM-1037) |
+|     4 | ok     | ok                                                           | ok                                                         | ok                                                           |
+|     5 | ok     | ok                                                           | ok                                                         | ok                                                           |
+|     6 | ok     | ok                                                           | ok                                                         | ok                                                           |
+|     7 | ok     | [BEAM-2112](https://issues.apache.org/jira/browse/BEAM-2112) | ok                                                         | ok                                                           |
+|     8 | ok     | ok                                                           | ok                                                         | ok                                                           |
+|     9 | ok     | ok                                                           | ok                                                         | ok                                                           |
+|    10 | ok     | ok                                                           | ok                                                         | ok                                                           |
+|    11 | ok     | ok                                                           | ok                                                         | ok                                                           |
+|    12 | ok     | ok                                                           | ok                                                         | ok                                                           |
 
 ## Batch / Synthetic / Cluster
 
@@ -219,44 +219,44 @@ TODO
 
 Batch Mode
 
-    mvn exec:java -Dexec.mainClass=org.apache.beam.integration.nexmark.Main -Pdirect-runner -Dexec.args="--runner=DirectRunner --suite=SMOKE --streaming=false --manageResources=false --monitorJobs=true --enforceEncodability=true --enforceImmutability=true"
+    mvn exec:java -Dexec.mainClass=org.apache.beam.sdk.nexmark.Main -Pdirect-runner -Dexec.args="--runner=DirectRunner --suite=SMOKE --streaming=false --manageResources=false --monitorJobs=true --enforceEncodability=true --enforceImmutability=true"
 
 Streaming Mode
 
-    mvn exec:java -Dexec.mainClass=org.apache.beam.integration.nexmark.Main -Pdirect-runner -Dexec.args="--runner=DirectRunner --suite=SMOKE --streaming=true --manageResources=false --monitorJobs=true --enforceEncodability=true --enforceImmutability=true"
+    mvn exec:java -Dexec.mainClass=org.apache.beam.sdk.nexmark.Main -Pdirect-runner -Dexec.args="--runner=DirectRunner --suite=SMOKE --streaming=true --manageResources=false --monitorJobs=true --enforceEncodability=true --enforceImmutability=true"
 
 
 ## Running SMOKE suite on the SparkRunner (local)
 
 Batch Mode
 
-    mvn exec:java -Dexec.mainClass=org.apache.beam.integration.nexmark.Main -Pspark-runner "-Dexec.args=--runner=SparkRunner --suite=SMOKE --streamTimeout=60 --streaming=false --manageResources=false --monitorJobs=true"
+    mvn exec:java -Dexec.mainClass=org.apache.beam.sdk.nexmark.Main -Pspark-runner "-Dexec.args=--runner=SparkRunner --suite=SMOKE --streamTimeout=60 --streaming=false --manageResources=false --monitorJobs=true"
 
 Streaming Mode
 
-    mvn exec:java -Dexec.mainClass=org.apache.beam.integration.nexmark.Main -Pspark-runner "-Dexec.args=--runner=SparkRunner --suite=SMOKE --streamTimeout=60 --streaming=true --manageResources=false --monitorJobs=true"
+    mvn exec:java -Dexec.mainClass=org.apache.beam.sdk.nexmark.Main -Pspark-runner "-Dexec.args=--runner=SparkRunner --suite=SMOKE --streamTimeout=60 --streaming=true --manageResources=false --monitorJobs=true"
 
 
 ## Running SMOKE suite on the FlinkRunner (local)
 
 Batch Mode
 
-    mvn exec:java -Dexec.mainClass=org.apache.beam.integration.nexmark.Main -Pflink-runner "-Dexec.args=--runner=FlinkRunner --suite=SMOKE --streamTimeout=60 --streaming=false --manageResources=false --monitorJobs=true  --flinkMaster=local"
+    mvn exec:java -Dexec.mainClass=org.apache.beam.sdk.nexmark.Main -Pflink-runner "-Dexec.args=--runner=FlinkRunner --suite=SMOKE --streamTimeout=60 --streaming=false --manageResources=false --monitorJobs=true  --flinkMaster=local"
 
 Streaming Mode
 
-    mvn exec:java -Dexec.mainClass=org.apache.beam.integration.nexmark.Main -Pflink-runner "-Dexec.args=--runner=FlinkRunner --suite=SMOKE --streamTimeout=60 --streaming=true --manageResources=false --monitorJobs=true  --flinkMaster=local"
+    mvn exec:java -Dexec.mainClass=org.apache.beam.sdk.nexmark.Main -Pflink-runner "-Dexec.args=--runner=FlinkRunner --suite=SMOKE --streamTimeout=60 --streaming=true --manageResources=false --monitorJobs=true  --flinkMaster=local"
 
 
 ## Running SMOKE suite on the ApexRunner (local)
 
 Batch Mode
 
-    mvn exec:java -Dexec.mainClass=org.apache.beam.integration.nexmark.Main -Papex-runner "-Dexec.args=--runner=ApexRunner --suite=SMOKE --streamTimeout=60 --streaming=false --manageResources=false --monitorJobs=false"
+    mvn exec:java -Dexec.mainClass=org.apache.beam.sdk.nexmark.Main -Papex-runner "-Dexec.args=--runner=ApexRunner --suite=SMOKE --streamTimeout=60 --streaming=false --manageResources=false --monitorJobs=false"
 
 Streaming Mode
 
-    mvn exec:java -Dexec.mainClass=org.apache.beam.integration.nexmark.Main -Papex-runner "-Dexec.args=--runner=ApexRunner --suite=SMOKE --streamTimeout=60 --streaming=true --manageResources=false --monitorJobs=false"
+    mvn exec:java -Dexec.mainClass=org.apache.beam.sdk.nexmark.Main -Papex-runner "-Dexec.args=--runner=ApexRunner --suite=SMOKE --streamTimeout=60 --streaming=true --manageResources=false --monitorJobs=false"
 
 
 ## Running SMOKE suite on Google Cloud Dataflow
@@ -269,8 +269,8 @@ Submit to Google Dataflow service
 
 
 ```
-java -cp integration/java/target/beam-integration-java-nexmark-bundled-2.1.0-SNAPSHOT.jar \
-  org.apache.beam.integration.nexmark.Main \
+java -cp sdks/java/nexmark/target/beam-sdks-java-nexmark-bundled-2.1.0-SNAPSHOT.jar \
+  org.apache.beam.sdk.nexmark.Main \
   --runner=DataflowRunner
   --project=<your project> \
   --zone=<your zone> \
@@ -303,8 +303,8 @@ java -cp integration/java/target/beam-integration-java-nexmark-bundled-2.1.0-SNA
 ```
 
 ```
-java -cp integration/java/target/beam-integration-java-nexmark-bundled-2.1.0-SNAPSHOT.jar \
-  org.apache.beam.integration.nexmark.Main \
+java -cp sdks/java/nexmark/target/beam-sdks-java-nexmark-bundled-2.1.0-SNAPSHOT.jar \
+  org.apache.beam.sdk.nexmark.Main \
   --runner=DataflowRunner
   --project=<your project> \
   --zone=<your zone> \
@@ -336,5 +336,5 @@ Building package
 
 Submit to the cluster
 
-    spark-submit --master yarn-client --class org.apache.beam.integration.nexmark.Main --driver-memory 512m --executor-memory 512m --executor-cores 1 beam-integration-java-nexmark-bundled-2.1.0-SNAPSHOT.jar --runner=SparkRunner --query=0 --streamTimeout=60 --streaming=false --manageResources=false --monitorJobs=true
+    spark-submit --master yarn-client --class org.apache.beam.sdk.nexmark.Main --driver-memory 512m --executor-memory 512m --executor-cores 1 beam-sdks-java-nexmark-bundled-2.1.0-SNAPSHOT.jar --runner=SparkRunner --query=0 --streamTimeout=60 --streaming=false --manageResources=false --monitorJobs=true
 

http://git-wip-us.apache.org/repos/asf/beam/blob/0acd7201/sdks/java/nexmark/pom.xml
----------------------------------------------------------------------
diff --git a/sdks/java/nexmark/pom.xml b/sdks/java/nexmark/pom.xml
index c1b6025..67628a1 100644
--- a/sdks/java/nexmark/pom.xml
+++ b/sdks/java/nexmark/pom.xml
@@ -23,7 +23,7 @@
   <parent>
     <groupId>org.apache.beam</groupId>
     <artifactId>beam-sdks-java-parent</artifactId>
-    <version>2.1.0-SNAPSHOT</version>
+    <version>2.2.0-SNAPSHOT</version>
     <relativePath>../pom.xml</relativePath>
   </parent>
 
@@ -35,7 +35,7 @@
 
     <!--
       The direct runner is available by default.
-      You can also include it on the classpath explicitly with -P direct-runner
+      You can also include it on the classpath explicitly using the profile below
     -->
     <profile>
       <id>direct-runner</id>
@@ -51,7 +51,6 @@
       </dependencies>
     </profile>
 
-    <!-- Include the Apache Apex runner with -P apex-runner -->
     <profile>
       <id>apex-runner</id>
       <dependencies>
@@ -60,27 +59,9 @@
           <artifactId>beam-runners-apex</artifactId>
           <scope>runtime</scope>
         </dependency>
-        <!--
-          Apex depends on httpclient version 4.3.5, project has a transitive dependency to httpclient 4.0.1 from
-          google-http-client. Apex dependency version being specified explicitly so that it gets picked up. This
-          can be removed when the project no longer has a dependency on a different httpclient version.
-        -->
-        <dependency>
-          <groupId>org.apache.httpcomponents</groupId>
-          <artifactId>httpclient</artifactId>
-          <version>4.3.5</version>
-          <scope>runtime</scope>
-          <exclusions>
-            <exclusion>
-              <groupId>commons-codec</groupId>
-              <artifactId>commons-codec</artifactId>
-            </exclusion>
-          </exclusions>
-        </dependency>
       </dependencies>
     </profile>
 
-    <!-- Include the Apache Flink runner with -P flink-runner -->
     <profile>
       <id>flink-runner</id>
       <dependencies>
@@ -92,7 +73,6 @@
       </dependencies>
     </profile>
 
-    <!-- Include the Apache Spark runner -P spark-runner -->
     <profile>
       <id>spark-runner</id>
       <dependencies>
@@ -122,7 +102,6 @@
       </dependencies>
     </profile>
 
-    <!-- Include the Google Cloud Dataflow runner -P dataflow-runner -->
     <profile>
       <id>dataflow-runner</id>
       <dependencies>
@@ -186,23 +165,15 @@
           </execution>
         </executions>
       </plugin>
-
-      <!-- Coverage analysis for unit tests. -->
-      <plugin>
-        <groupId>org.jacoco</groupId>
-        <artifactId>jacoco-maven-plugin</artifactId>
-      </plugin>
     </plugins>
   </build>
 
   <dependencies>
-    <!-- Java SDK -->
     <dependency>
       <groupId>org.apache.beam</groupId>
       <artifactId>beam-sdks-java-core</artifactId>
     </dependency>
 
-    <!-- IOs -->
     <dependency>
       <groupId>org.apache.beam</groupId>
       <artifactId>beam-sdks-java-io-google-cloud-platform</artifactId>
@@ -218,7 +189,6 @@
       <artifactId>google-api-services-bigquery</artifactId>
     </dependency>
 
-    <!-- Extra libraries -->
     <dependency>
       <groupId>com.fasterxml.jackson.core</groupId>
       <artifactId>jackson-core</artifactId>
@@ -282,7 +252,6 @@
       <artifactId>hamcrest-all</artifactId>
     </dependency>
 
-    <!-- Test -->
     <dependency>
       <groupId>org.apache.beam</groupId>
       <artifactId>beam-runners-direct-java</artifactId>

http://git-wip-us.apache.org/repos/asf/beam/blob/0acd7201/sdks/java/nexmark/src/main/java/org/apache/beam/sdk/nexmark/package-info.java
----------------------------------------------------------------------
diff --git a/sdks/java/nexmark/src/main/java/org/apache/beam/sdk/nexmark/package-info.java b/sdks/java/nexmark/src/main/java/org/apache/beam/sdk/nexmark/package-info.java
index 7500a24..62218a4 100644
--- a/sdks/java/nexmark/src/main/java/org/apache/beam/sdk/nexmark/package-info.java
+++ b/sdks/java/nexmark/src/main/java/org/apache/beam/sdk/nexmark/package-info.java
@@ -16,6 +16,6 @@
  * limitations under the License.
  */
 /**
- * Nexmark.
+ * Nexmark test suite.
  */
 package org.apache.beam.sdk.nexmark;

http://git-wip-us.apache.org/repos/asf/beam/blob/0acd7201/sdks/java/nexmark/src/main/resources/log4j.properties
----------------------------------------------------------------------
diff --git a/sdks/java/nexmark/src/main/resources/log4j.properties b/sdks/java/nexmark/src/main/resources/log4j.properties
index 7dd57b5..14f8acd 100644
--- a/sdks/java/nexmark/src/main/resources/log4j.properties
+++ b/sdks/java/nexmark/src/main/resources/log4j.properties
@@ -27,14 +27,14 @@ log4j.logger.org.apache.beam.runners.direct=WARN
 log4j.logger.org.apache.beam.sdk=WARN
 
 # Nexmark specific
-log4j.logger.org.apache.beam.integration.nexmark=WARN
+log4j.logger.org.apache.beam.sdk.nexmark=WARN
 
 # Settings to quiet third party logs that are too verbose
 log4j.logger.org.spark_project.jetty=WARN
 log4j.logger.org.spark_project.jetty.util.component.AbstractLifeCycle=ERROR
 
 # Setting to quiet spark logs, Beam logs should standout
-log4j.logger.org.apache.beam.runners.spark=INFO
+log4j.logger.org.apache.beam.runners.spark=WARN
 log4j.logger.org.apache.spark=WARN
 log4j.logger.org.spark-project=WARN
 log4j.logger.io.netty=INFO