From 4d04939da132303108b07c4e6287400065c4c326 Mon Sep 17 00:00:00 2001 From: Mark Shields Date: Mon, 28 Mar 2016 16:25:29 -0700 Subject: [PATCH 1/4] NexMark --- integration/java/pom.xml | 255 ++++++ .../nexmark/AbstractSimulator.java | 212 +++++ .../beam/integration/nexmark/Auction.java | 190 +++++ .../beam/integration/nexmark/AuctionBid.java | 87 ++ .../integration/nexmark/AuctionCount.java | 90 +++ .../integration/nexmark/AuctionPrice.java | 91 +++ .../nexmark/BEAM_ON_FLINK_ON_GCP.md | 266 +++++++ .../apache/beam/integration/nexmark/Bid.java | 178 +++++ .../integration/nexmark/BidsPerSession.java | 89 +++ .../nexmark/BoundedEventSource.java | 197 +++++ .../integration/nexmark/CategoryPrice.java | 100 +++ .../apache/beam/integration/nexmark/Done.java | 83 ++ .../beam/integration/nexmark/Event.java | 181 +++++ .../beam/integration/nexmark/Generator.java | 590 ++++++++++++++ .../integration/nexmark/GeneratorConfig.java | 295 +++++++ .../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 ++++++++++++++++ .../integration/nexmark/NexmarkDriver.java | 297 +++++++ .../nexmark/NexmarkFlinkDriver.java | 49 ++ .../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 +++++ .../integration/nexmark/NexmarkQuery.java | 276 +++++++ .../nexmark/NexmarkQueryModel.java | 123 +++ .../integration/nexmark/NexmarkRunner.java | 746 ++++++++++++++++++ .../integration/nexmark/NexmarkSuite.java | 112 +++ .../integration/nexmark/NexmarkUtils.java | 681 ++++++++++++++++ .../beam/integration/nexmark/Options.java | 360 +++++++++ .../beam/integration/nexmark/Person.java | 166 ++++ .../integration/nexmark/PubsubHelper.java | 217 +++++ .../beam/integration/nexmark/Query0.java | 72 ++ .../beam/integration/nexmark/Query0Model.java | 62 ++ .../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 ++ .../beam/integration/nexmark/Query2.java | 75 ++ .../beam/integration/nexmark/Query2Model.java | 76 ++ .../beam/integration/nexmark/Query3.java | 248 ++++++ .../beam/integration/nexmark/Query3Model.java | 119 +++ .../beam/integration/nexmark/Query4.java | 110 +++ .../beam/integration/nexmark/Query4Model.java | 181 +++++ .../beam/integration/nexmark/Query5.java | 127 +++ .../beam/integration/nexmark/Query5Model.java | 174 ++++ .../beam/integration/nexmark/Query6.java | 154 ++++ .../beam/integration/nexmark/Query6Model.java | 128 +++ .../beam/integration/nexmark/Query7.java | 87 ++ .../beam/integration/nexmark/Query7Model.java | 128 +++ .../beam/integration/nexmark/Query8.java | 92 +++ .../beam/integration/nexmark/Query8Model.java | 145 ++++ .../beam/integration/nexmark/Query9.java | 40 + .../beam/integration/nexmark/Query9Model.java | 44 ++ .../apache/beam/integration/nexmark/README.md | 147 ++++ .../beam/integration/nexmark/SellerPrice.java | 91 +++ .../nexmark/UnboundedEventSource.java | 329 ++++++++ .../beam/integration/nexmark/WinningBids.java | 378 +++++++++ .../nexmark/WinningBidsSimulator.java | 203 +++++ pom.xml | 1 + 65 files changed, 11963 insertions(+) create mode 100644 integration/java/pom.xml create mode 100644 integration/java/src/main/java/org/apache/beam/integration/nexmark/AbstractSimulator.java create mode 100644 integration/java/src/main/java/org/apache/beam/integration/nexmark/Auction.java create mode 100644 integration/java/src/main/java/org/apache/beam/integration/nexmark/AuctionBid.java create mode 100644 integration/java/src/main/java/org/apache/beam/integration/nexmark/AuctionCount.java create mode 100644 integration/java/src/main/java/org/apache/beam/integration/nexmark/AuctionPrice.java create mode 100644 integration/java/src/main/java/org/apache/beam/integration/nexmark/BEAM_ON_FLINK_ON_GCP.md create mode 100644 integration/java/src/main/java/org/apache/beam/integration/nexmark/Bid.java create mode 100644 integration/java/src/main/java/org/apache/beam/integration/nexmark/BidsPerSession.java create mode 100644 integration/java/src/main/java/org/apache/beam/integration/nexmark/BoundedEventSource.java create mode 100644 integration/java/src/main/java/org/apache/beam/integration/nexmark/CategoryPrice.java create mode 100644 integration/java/src/main/java/org/apache/beam/integration/nexmark/Done.java create mode 100644 integration/java/src/main/java/org/apache/beam/integration/nexmark/Event.java create mode 100644 integration/java/src/main/java/org/apache/beam/integration/nexmark/Generator.java create mode 100644 integration/java/src/main/java/org/apache/beam/integration/nexmark/GeneratorConfig.java create mode 100644 integration/java/src/main/java/org/apache/beam/integration/nexmark/IdNameReserve.java create mode 100644 integration/java/src/main/java/org/apache/beam/integration/nexmark/KnownSize.java create mode 100644 integration/java/src/main/java/org/apache/beam/integration/nexmark/Monitor.java create mode 100644 integration/java/src/main/java/org/apache/beam/integration/nexmark/NameCityStateId.java create mode 100644 integration/java/src/main/java/org/apache/beam/integration/nexmark/NexmarkConfiguration.java create mode 100644 integration/java/src/main/java/org/apache/beam/integration/nexmark/NexmarkDriver.java create mode 100644 integration/java/src/main/java/org/apache/beam/integration/nexmark/NexmarkFlinkDriver.java create mode 100644 integration/java/src/main/java/org/apache/beam/integration/nexmark/NexmarkFlinkRunner.java create mode 100644 integration/java/src/main/java/org/apache/beam/integration/nexmark/NexmarkGoogleDriver.java create mode 100644 integration/java/src/main/java/org/apache/beam/integration/nexmark/NexmarkGoogleRunner.java create mode 100644 integration/java/src/main/java/org/apache/beam/integration/nexmark/NexmarkInProcessDriver.java create mode 100644 integration/java/src/main/java/org/apache/beam/integration/nexmark/NexmarkInProcessRunner.java create mode 100644 integration/java/src/main/java/org/apache/beam/integration/nexmark/NexmarkPerf.java create mode 100644 integration/java/src/main/java/org/apache/beam/integration/nexmark/NexmarkQuery.java create mode 100644 integration/java/src/main/java/org/apache/beam/integration/nexmark/NexmarkQueryModel.java create mode 100644 integration/java/src/main/java/org/apache/beam/integration/nexmark/NexmarkRunner.java create mode 100644 integration/java/src/main/java/org/apache/beam/integration/nexmark/NexmarkSuite.java create mode 100644 integration/java/src/main/java/org/apache/beam/integration/nexmark/NexmarkUtils.java create mode 100644 integration/java/src/main/java/org/apache/beam/integration/nexmark/Options.java create mode 100644 integration/java/src/main/java/org/apache/beam/integration/nexmark/Person.java create mode 100644 integration/java/src/main/java/org/apache/beam/integration/nexmark/PubsubHelper.java create mode 100644 integration/java/src/main/java/org/apache/beam/integration/nexmark/Query0.java create mode 100644 integration/java/src/main/java/org/apache/beam/integration/nexmark/Query0Model.java create mode 100644 integration/java/src/main/java/org/apache/beam/integration/nexmark/Query1.java create mode 100644 integration/java/src/main/java/org/apache/beam/integration/nexmark/Query10.java create mode 100644 integration/java/src/main/java/org/apache/beam/integration/nexmark/Query11.java create mode 100644 integration/java/src/main/java/org/apache/beam/integration/nexmark/Query12.java create mode 100644 integration/java/src/main/java/org/apache/beam/integration/nexmark/Query1Model.java create mode 100644 integration/java/src/main/java/org/apache/beam/integration/nexmark/Query2.java create mode 100644 integration/java/src/main/java/org/apache/beam/integration/nexmark/Query2Model.java create mode 100644 integration/java/src/main/java/org/apache/beam/integration/nexmark/Query3.java create mode 100644 integration/java/src/main/java/org/apache/beam/integration/nexmark/Query3Model.java create mode 100644 integration/java/src/main/java/org/apache/beam/integration/nexmark/Query4.java create mode 100644 integration/java/src/main/java/org/apache/beam/integration/nexmark/Query4Model.java create mode 100644 integration/java/src/main/java/org/apache/beam/integration/nexmark/Query5.java create mode 100644 integration/java/src/main/java/org/apache/beam/integration/nexmark/Query5Model.java create mode 100644 integration/java/src/main/java/org/apache/beam/integration/nexmark/Query6.java create mode 100644 integration/java/src/main/java/org/apache/beam/integration/nexmark/Query6Model.java create mode 100644 integration/java/src/main/java/org/apache/beam/integration/nexmark/Query7.java create mode 100644 integration/java/src/main/java/org/apache/beam/integration/nexmark/Query7Model.java create mode 100644 integration/java/src/main/java/org/apache/beam/integration/nexmark/Query8.java create mode 100644 integration/java/src/main/java/org/apache/beam/integration/nexmark/Query8Model.java create mode 100644 integration/java/src/main/java/org/apache/beam/integration/nexmark/Query9.java create mode 100644 integration/java/src/main/java/org/apache/beam/integration/nexmark/Query9Model.java create mode 100644 integration/java/src/main/java/org/apache/beam/integration/nexmark/README.md create mode 100644 integration/java/src/main/java/org/apache/beam/integration/nexmark/SellerPrice.java create mode 100644 integration/java/src/main/java/org/apache/beam/integration/nexmark/UnboundedEventSource.java create mode 100644 integration/java/src/main/java/org/apache/beam/integration/nexmark/WinningBids.java create mode 100644 integration/java/src/main/java/org/apache/beam/integration/nexmark/WinningBidsSimulator.java diff --git a/integration/java/pom.xml b/integration/java/pom.xml new file mode 100644 index 000000000000..b158ff4c555c --- /dev/null +++ b/integration/java/pom.xml @@ -0,0 +1,255 @@ + + + + 4.0.0 + + + org.apache.beam + parent + 0.1.0-incubating-SNAPSHOT + ../../pom.xml + + + java-integration-all + Apache Beam :: Integration Tests :: Java All + Beam SDK Java All provides a simple, Java-based + interface for processing virtually any size data. This + artifact includes some Java SDK integration tests. + + jar + + + + + + kr.motd.maven + os-maven-plugin + 1.4.0.Final + + + + + + maven-compiler-plugin + 3.1 + + 1.7 + 1.7 + + + + + org.apache.maven.plugins + maven-dependency-plugin + + + + org.apache.maven.plugins + maven-checkstyle-plugin + + + + org.apache.maven.plugins + maven-shade-plugin + 2.4.1 + + + package + + shade + + + + + *:* + + + + + *:* + + META-INF/*.SF + META-INF/*.DSA + META-INF/*.RSA + + + + + + + + + + + org.apache.avro + avro-maven-plugin + ${avro.version} + + + schemas + generate-sources + + schema + + + ${project.basedir}/src/main/ + ${project.build.directory}/generated-sources/java + + + + + + + org.apache.maven.plugins + maven-jar-plugin + + + default-jar + + jar + + + + default-test-jar + + test-jar + + + + + + + + org.jacoco + jacoco-maven-plugin + + + + + + + + org.apache.beam + java-sdk-all + + + + + org.apache.beam + google-cloud-dataflow-java-runner + ${project.version} + + + + org.apache.beam + runners-core + ${project.version} + + + + + org.apache.beam.runners + direct + ${project.version} + + + + + org.apache.beam + flink-runner_2.10 + ${project.version} + + + + + org.apache.avro + avro + ${avro.version} + + + + com.google.guava + guava + ${guava.version} + + + + org.hamcrest + hamcrest-all + ${hamcrest.version} + + + + org.slf4j + slf4j-api + ${slf4j.version} + + + + org.slf4j + slf4j-jdk14 + ${slf4j.version} + + runtime + + + + com.google.code.findbugs + jsr305 + ${jsr305.version} + + + + commons-logging + commons-logging + 1.2 + + + + log4j + log4j + 1.2.17 + + + + org.apache.logging.log4j + log4j-api + 2.4 + + + + junit + junit + provided + + + + io.netty + netty-tcnative-boringssl-static + 1.1.33.Fork13 + ${os.detected.classifier} + + + org.apache.beam.runners + direct + 0.1.0-incubating-SNAPSHOT + + + 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 000000000000..6473c35c3996 --- /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 Type of input elements. + * @param Type of output elements. + */ +abstract class AbstractSimulator { + /** 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> 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> 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 pendingCounts; + + public AbstractSimulator(Iterator> 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 nextInput() { + if (!input.hasNext()) { + return null; + } + TimestampedValue 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 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 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> results() { + return new Iterator>() { + @Override + public boolean hasNext() { + while (true) { + if (!pendingResults.isEmpty()) { + return true; + } + if (isDone) { + return false; + } + run(); + } + } + + @Override + public TimestampedValue next() { + TimestampedValue 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 resultsPerWindow() { + return new Iterator() { + @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(); + } + }; + } +} 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 000000000000..94f2647e28d7 --- /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_CODER = VarLongCoder.of(); + private static final Coder STRING_CODER = StringUtf8Coder.of(); + + public static final Coder CODER = new AtomicCoder() { + @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); + } + } +} 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 000000000000..8c3697a3780e --- /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 CODER = new AtomicCoder() { + @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); + } + } +} 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 000000000000..a0fbebc36e2d --- /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_CODER = VarLongCoder.of(); + + public static final Coder CODER = new AtomicCoder() { + @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); + } + } +} 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 000000000000..4f25a9b64de8 --- /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_CODER = VarLongCoder.of(); + + public static final Coder CODER = new AtomicCoder() { + @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); + } + } +} 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 000000000000..7b6131e69cab --- /dev/null +++ b/integration/java/src/main/java/org/apache/beam/integration/nexmark/BEAM_ON_FLINK_ON_GCP.md @@ -0,0 +1,266 @@ +# 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= +# Flink root +FLINK_VER=flink-1.0.3 +FLINK= +# Google Cloud project +PROJECT= +# Google Cloud zone +ZONE= +# Cloud commands +GCLOUD= +GSUTIL= +``` + +## 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/ +cp -f \ + $BEAM/integration/java/target/java-integration-all-0.1.0-incubating-SNAPSHOT.jar \ + $FLINK/lib/ +``` + +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-0.1.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-0.1.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-0.1.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 +``` 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 000000000000..ce2184b22156 --- /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_CODER = VarLongCoder.of(); + private static final Coder STRING_CODER = StringUtf8Coder.of(); + + public static final Coder CODER = new AtomicCoder() { + @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 PRICE_THEN_DESCENDING_TIME = new Comparator() { + @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 ASCENDING_TIME_THEN_PRICE = new Comparator() { + @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); + } + } +} 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 000000000000..cfdd170431bb --- /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_CODER = VarLongCoder.of(); + + public static final Coder CODER = new AtomicCoder() { + @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); + } + } +} 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 000000000000..f6cc16aef16f --- /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 { + /** 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 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 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 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 splitIntoBundles( + long desiredBundleSizeBytes, PipelineOptions options) { + NexmarkUtils.info("slitting bounded source %s into %d sub-sources", config, numEventGenerators); + List 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 getDefaultOutputCoder() { + return Event.CODER; + } +} 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 000000000000..ab5d92d264d4 --- /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_CODER = VarLongCoder.of(); + private static final Coder INT_CODER = VarIntCoder.of(); + + public static final Coder CODER = new AtomicCoder() { + @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); + } + } +} 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 000000000000..659da441ed11 --- /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_CODER = StringUtf8Coder.of(); + + public static final Coder CODER = new AtomicCoder() { + @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); + } + } +} 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 000000000000..a382b8ebed43 --- /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 INT_CODER = VarIntCoder.of(); + + public static final Coder CODER = new AtomicCoder() { + @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"); + } + } +} 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 000000000000..98f4f0042b99 --- /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. + * + *

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. + * + *

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>, 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 US_STATES = Arrays.asList(("AZ,CA,ID,OR,WA,WY").split(",")); + + private static final List US_CITIES = + Arrays.asList( + ("Phoenix,Los Angeles,San Francisco,Boise,Portland,Bend,Redmond,Seattle,Kent,Cheyenne") + .split(",")); + + private static final List FIRST_NAMES = + Arrays.asList(("Peter,Paul,Luke,John,Saul,Vicky,Kate,Julie,Sarah,Deiter,Walter").split(",")); + + private static final List 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_CODER = VarLongCoder.of(); + + /** Coder for this class. */ + public static final Coder CODER_INSTANCE = + new AtomicCoder() { + @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 { + /** 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 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); + } +} 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 000000000000..59aaf492be69 --- /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 split(int n) { + List 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 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(); + } +} 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 000000000000..c72b76aa63f7 --- /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_CODER = VarLongCoder.of(); + private static final Coder STRING_CODER = StringUtf8Coder.of(); + + public static final Coder CODER = new AtomicCoder() { + @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); + } + } +} 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 000000000000..394b6db43eed --- /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(); +} + 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 000000000000..687457893708 --- /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 Type of element we are monitoring. + */ +public class Monitor implements Serializable { + private class MonitorDoFn extends DoFn { + public final Aggregator elementCounter = + createAggregator(counterNamePrefix + "_elements", new SumLongFn()); + public final Aggregator bytesCounter = + createAggregator(counterNamePrefix + "_bytes", new SumLongFn()); + public final Aggregator startTime = + createAggregator(counterNamePrefix + "_startTime", new MinLongFn()); + public final Aggregator endTime = + createAggregator(counterNamePrefix + "_endTime", new MaxLongFn()); + public final Aggregator startTimestamp = + createAggregator("startTimestamp", new MinLongFn()); + public final Aggregator 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> 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> getTransform() { + return transform; + } + + public Aggregator getElementCounter() { + return doFn.elementCounter; + } + + public Aggregator getBytesCounter() { + return doFn.bytesCounter; + } + + public Aggregator getStartTime() { + return doFn.startTime; + } + + public Aggregator getEndTime() { + return doFn.endTime; + } + + public Aggregator getStartTimestamp() { + return doFn.startTimestamp; + } + + public Aggregator getEndTimestamp() { + return doFn.endTimestamp; + } +} 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 000000000000..2753d2ecf935 --- /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_CODER = VarLongCoder.of(); + private static final Coder STRING_CODER = StringUtf8Coder.of(); + + public static final Coder CODER = new AtomicCoder() { + @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); + } + } +} 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 000000000000..2292ba527ac7 --- /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; + } +} 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 000000000000..dbc1ce27a504 --- /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: + *

    + *
  • {@link Person} represents a person submitting an item for auction and/or making a bid + * on an auction. + *
  • {@link Auction} represents an item under auction. + *
  • {@link Bid} represents a bid for an item under auction. + *
+ * The queries exercise many aspects of streaming dataflow. + *

+ *

We synthesize the creation of people, auctions and bids in real-time. The data is not + * particularly sensible. + *

+ *

See + * + * http://datalab.cs.pdx.edu/niagaraST/NEXMark/ + */ +public class NexmarkDriver { + + /** + * Entry point. + */ + public void runAll(OptionT options, NexmarkRunner runner) { + Instant start = Instant.now(); + Map baseline = loadBaseline(options.getBaselineFilename()); + Map actual = new LinkedHashMap<>(); + Iterable 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 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 loadBaseline( + @Nullable String baselineFilename) { + if (baselineFilename == null) { + return null; + } + Map baseline = new LinkedHashMap<>(); + List 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 configurations, Map actual, + @Nullable Map baseline, Instant start) { + List 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 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 configurations, Map actual, + @Nullable Map baseline, Instant start) { + if (javascriptFilename == null) { + return; + } + + List 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); + } +} 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 000000000000..0029a3653bbe --- /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 { + /** + * 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); + } +} + + 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 000000000000..569aef66d638 --- /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 { + @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); + } +} 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 000000000000..253415522ebb --- /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: + *

    + *
  • {@link Person} represents a person submitting an item for auction and/or making a bid + * on an auction. + *
  • {@link Auction} represents an item under auction. + *
  • {@link Bid} represents a bid for an item under auction. + *
+ * The queries exercise many aspects of streaming dataflow. + *

+ *

We synthesize the creation of people, auctions and bids in real-time. The data is not + * particularly sensible. + *

+ *

See + * + * http://datalab.cs.pdx.edu/niagaraST/NEXMark/ + */ +class NexmarkGoogleDriver extends NexmarkDriver { + /** + * 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); + } +} + 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 000000000000..4b735922ef76 --- /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 { + /** + * 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(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 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 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. + *

+ *

Returns a list of errors detected. + */ + private List checkWatermarks(DataflowPipelineJob job, long startMsSinceEpoch) { + long now = System.currentTimeMillis(); + List errors = new ArrayList<>(); + try { + JobMetrics metricResponse = job.getDataflowClient() + .projects() + .jobs() + .getMetrics(job.getProjectId(), job.getJobId()) + .execute(); + List 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 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 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 aggregator) { + try { + Collection 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 aggregator) { + try { + Collection 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; + } + } +} 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 000000000000..fe279c031f99 --- /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 { + /** + * 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); + } +} + 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 000000000000..ba141f947042 --- /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 { + 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"); + } +} 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 000000000000..6eb7267b7eae --- /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 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 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; + } +} 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 000000000000..462660948bd0 --- /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>> { + protected static final TupleTag AUCTION_TAG = new TupleTag<>("auctions"); + protected static final TupleTag BID_TAG = new TupleTag<>("bids"); + protected static final TupleTag PERSON_TAG = new TupleTag<>("person"); + + /** Predicate to detect a new person event. */ + protected static final SerializableFunction IS_NEW_PERSON = + new SerializableFunction() { + @Override + public Boolean apply(Event event) { + return event.newPerson != null; + } + }; + + /** DoFn to convert a new person event to a person. */ + protected static final DoFn AS_PERSON = new DoFn() { + @Override + public void processElement(ProcessContext c) { + c.output(c.element().newPerson); + } + }; + + /** Predicate to detect a new auction event. */ + protected static final SerializableFunction IS_NEW_AUCTION = + new SerializableFunction() { + @Override + public Boolean apply(Event event) { + return event.newAuction != null; + } + }; + + /** DoFn to convert a new auction event to an auction. */ + protected static final DoFn AS_AUCTION = new DoFn() { + @Override + public void processElement(ProcessContext c) { + c.output(c.element().newAuction); + } + }; + + /** Predicate to detect a new bid event. */ + protected static final SerializableFunction IS_BID = + new SerializableFunction() { + @Override + public Boolean apply(Event event) { + return event.bid != null; + } + }; + + /** DoFn to convert a bid event to a bid. */ + protected static final DoFn AS_BID = new DoFn() { + @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_BY_ID = + ParDo.named("PersonById") + .of(new DoFn>() { + @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_BY_ID = + ParDo.named("AuctionById") + .of(new DoFn>() { + @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_BY_SELLER = + ParDo.named("AuctionBySeller") + .of(new DoFn>() { + @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_BY_AUCTION = + ParDo.named("BidByAuction") + .of(new DoFn>() { + @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_TO_AUCTION = + ParDo.named("BidToAuction") + .of(new DoFn() { + @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_TO_PRICE = + ParDo.named("BidToPrice") + .of(new DoFn() { + @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_TIMESTAMP_FROM_DATA = + ParDo.named("OutputWithTimestamp") + .of(new DoFn() { + @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> JUST_NEW_AUCTIONS = + new PTransform, PCollection>("justNewAuctions") { + @Override + public PCollection apply(PCollection 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> JUST_NEW_PERSONS = + new PTransform, PCollection>("justNewPersons") { + @Override + public PCollection apply(PCollection 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> JUST_BIDS = + new PTransform, PCollection>("justBids") { + @Override + public PCollection apply(PCollection input) { + return input.apply(Filter.byPredicate(IS_BID).named("IsBid")) + .apply(ParDo.named("AsBid").of(AS_BID)); + } + }; + + protected final NexmarkConfiguration configuration; + public final Monitor eventMonitor; + public final Monitor resultMonitor; + public final Monitor 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 getFatalCount() { + return null; + } + + /** + * Implement the actual query. All we know about the result is it has a known encoded size. + */ + protected abstract PCollection applyPrim(PCollection events); + + @Override + public PCollection> apply(PCollection 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.cpuDelay(name, configuration.cpuDelayMs)); + } + + if (configuration.diskBusyBytes > 0) { + // Slow down by forcing bytes to durable store. + events = events.apply(NexmarkUtils.diskBusy(name, configuration.diskBusyBytes)); + } + + // Run the query. + PCollection 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.stamp(name)); + } +} 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 000000000000..b42042f0de8e --- /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 List toValueTimestampOrder(Iterator> itr) { + List strings = new ArrayList<>(); + while (itr.hasNext()) { + strings.add(itr.next().toString()); + } + return strings; + } + + /** + * Convert {@code itr} to strings capturing values and order. + */ + protected static List toValueOrder(Iterator> itr) { + List strings = new ArrayList<>(); + while (itr.hasNext()) { + strings.add(itr.next().getValue().toString()); + } + return strings; + } + + /** + * Convert {@code itr} to strings capturing values only. + */ + protected static Set toValue(Iterator> itr) { + Set 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> relevantResults( + Iterable> results) { + return results; + } + + /** + * Convert iterator of elements to collection of strings to use when testing coherence + * of model against actual query results. + */ + protected abstract Collection toCollection(Iterator> itr); + + /** + * Return assertion to use on results of pipeline for this query. + */ + public SerializableFunction>, Void> assertionFor() { + final Collection expectedStrings = toCollection(simulator().results()); + + return new SerializableFunction>, Void>() { + @Override + public Void apply(Iterable> actual) { + Collection actualStrings = toCollection(relevantResults(actual).iterator()); + Assert.assertEquals(expectedStrings, actualStrings); + return null; + } + }; + } +} 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 000000000000..b7151f87a143 --- /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 { + /** + * 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 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 { + 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 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 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 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 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 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 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 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 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: + *

    + *
  • {@code $outputPath/event*.avro} All Event entities. + *
  • {@code $outputPath/auction*.avro} Auction entities. + *
  • {@code $outputPath/bid*.avro} Bid entities. + *
  • {@code $outputPath/person*.avro} Person entities. + *
+ * + * @param source A PCollection of events. + */ + private void sinkEventsToAvro(PCollection 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 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 { + @Override + public void processElement(ProcessContext c) { + int n = ThreadLocalRandom.current().nextInt(10); + List 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 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 createSource(Pipeline p, final long now) { + PCollection 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(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 MAIN = new TupleTag(){}; + private static final TupleTag SIDE = new TupleTag(){}; + + private static class PartitionDoFn extends DoFn { + @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> results, long now) { + if (configuration.sinkType == NexmarkUtils.SinkType.COUNT_ONLY) { + // Avoid the cost of formatting the results. + results.apply(NexmarkUtils.devNull(queryName)); + return; + } + + PCollection formattedResults = results.apply(NexmarkUtils.format(queryName)); + if (options.getLogResults()) { + formattedResults = formattedResults.apply(NexmarkUtils.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 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 pathsToDelete = new ArrayList<>(); + + try { + NexmarkUtils.console("Running %s", configuration.toShortString()); + + if (configuration.numEvents < 0) { + NexmarkUtils.console("skipping since configuration is disabled"); + return null; + } + + List 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 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 source = createSource(p, now); + + if (options.getLogEvents()) { + source = source.apply(NexmarkUtils.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> 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 + } + } +} 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 000000000000..cccaeb146723 --- /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 defaultConf() { + List configurations = new ArrayList<>(); + configurations.add(new NexmarkConfiguration()); + return configurations; + } + + private static List smoke() { + List 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 stress() { + List configurations = smoke(); + for (NexmarkConfiguration configuration : configurations) { + if (configuration.numEvents >= 0) { + configuration.numEvents *= 1000; + } + } + return configurations; + } + + private static List fullThrottle() { + List configurations = smoke(); + for (NexmarkConfiguration configuration : configurations) { + if (configuration.numEvents >= 0) { + configuration.numEvents *= 1000; + } + } + return configurations; + } + + private final List configurations; + + NexmarkSuite(List 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 getConfigurations(Options options) { + Set results = new LinkedHashSet<>(); + for (NexmarkConfiguration configuration : configurations) { + NexmarkConfiguration result = configuration.clone(); + result.overrideFromOptions(options); + results.add(result); + } + return results; + } +} 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 000000000000..13ed5808617d --- /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> 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> 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> 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 snoop(final String name) { + return ParDo.named(name + ".Snoop") + .of(new DoFn() { + final Aggregator eventCounter = + createAggregator("events", new SumLongFn()); + final Aggregator newPersonCounter = + createAggregator("newPersons", new SumLongFn()); + final Aggregator newAuctionCounter = + createAggregator("newAuctions", new SumLongFn()); + final Aggregator bidCounter = + createAggregator("bids", new SumLongFn()); + final Aggregator 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 ParDo.Bound devNull(String name) { + return ParDo.named(name + ".DevNull") + .of(new DoFn() { + final Aggregator 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 ParDo.Bound log(final String name) { + return ParDo.named(name + ".Log") + .of(new DoFn() { + @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 ParDo.Bound format(String name) { + return ParDo.named(name + ".Format") + .of(new DoFn() { + final Aggregator 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 ParDo.Bound> stamp(String name) { + return ParDo.named(name + ".Stamp") + .of(new DoFn>() { + @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 PTransform, PCollection> hash( + final long numEvents, String name) { + return new PTransform, PCollection>(name) { + @Override + public PCollection apply(PCollection input) { + return input.apply(Window.into(new GlobalWindows()) + .triggering(AfterPane.elementCountAtLeast((int) numEvents)) + .withAllowedLateness(Duration.standardDays(1)) + .discardingFiredPanes()) + + .apply(ParDo.named(name + ".Hash").of(new DoFn() { + @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() { + @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 ParDo.Bound cpuDelay(String name, final long delayMs) { + return ParDo.named(name + ".CpuDelay") + .of(new DoFn() { + @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> 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 ParDo.Bound diskBusy(String name, final long bytes) { + return ParDo.named(name + ".DiskBusy") + .of(new DoFn() { + @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 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 ParDo.Bound castToKnownSize( + final String name) { + return ParDo.named(name + ".Forget") + .of(new DoFn() { + @Override + public void processElement(ProcessContext c) { + c.output(c.element()); + } + }); + } + + /** + * A coder for instances of {@code T} cast up to {@link KnownSize}. + * + * @param True type of object. + */ + private static class CastingCoder extends CustomCoder { + private final Coder trueCoder; + + public CastingCoder(Coder 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> getComponents() { + return ImmutableList.of(trueCoder); + } + } + + /** + * Return a coder for {@code KnownSize} that are known to be exactly of type {@code T}. + */ + private static Coder makeCastingCoder(Coder trueCoder) { + return new CastingCoder<>(trueCoder); + } + + /** + * Return {@code elements} as {@code KnownSize}s. + */ + public static PCollection castToKnownSize( + final String name, PCollection elements) { + return elements.apply(castToKnownSize(name)).setCoder(makeCastingCoder(elements.getCoder())); + } + + // Do not instantiate. + private NexmarkUtils() { + } +} 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 000000000000..4f5304d5a159 --- /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); +} 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 000000000000..6fcf388b7272 --- /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_CODER = VarLongCoder.of(); + private static final Coder STRING_CODER = StringUtf8Coder.of(); + public static final Coder CODER = new AtomicCoder() { + @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); + } + } +} 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 000000000000..1255154dc767 --- /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 createdTopics; + + /** + * Subscriptions we should delete on close. + */ + private final List 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 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 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); + } + } + } +} 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 000000000000..ea0d7ca20066 --- /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 applyTyped(PCollection events) { + final Coder coder = events.getCoder(); + + return events + + // Force round trip through coder. + .apply( + ParDo.named(name + ".Serialize") + .of(new DoFn() { + private final Aggregator 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 applyPrim(PCollection events) { + return NexmarkUtils.castToKnownSize(name, applyTyped(events)); + } +} 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 000000000000..f3ceca267a05 --- /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 { + public Simulator(NexmarkConfiguration configuration) { + super(NexmarkUtils.standardEventIterator(configuration)); + } + + @Override + protected void run() { + TimestampedValue 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 Collection toCollection(Iterator> itr) { + return toValueTimestampOrder(itr); + } +} 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 000000000000..7e60b9c783db --- /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: + * + *
+ * SELECT Istream(auction, DOLTOEUR(price), bidder, datetime)
+ * FROM bid [ROWS UNBOUNDED];
+ * 
+ * + *

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 applyTyped(PCollection 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() { + @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 applyPrim(PCollection events) { + return NexmarkUtils.castToKnownSize(name, applyTyped(events)); + } +} 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 000000000000..74fb28c3cc77 --- /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.) + * + *

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 applyTyped(PCollection events) { + final int numLogShards = maxNumWorkers * NUM_SHARDS_PER_WORKER; + + return events + .apply(ParDo.named(name + ".ShardEvents") + .of(new DoFn>() { + final Aggregator lateCounter = + createAggregator("actuallyLateEvent", new SumLongFn()); + final Aggregator 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.>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.create()) + .apply( + ParDo.named(name + ".CheckForLateEvents") + .of(new DoFnWithContext>, + KV>>() { + final Aggregator earlyCounter = + createAggregator("earlyShard", new SumLongFn()); + final Aggregator onTimeCounter = + createAggregator("onTimeShard", new SumLongFn()); + final Aggregator lateCounter = + createAggregator("lateShard", new SumLongFn()); + final Aggregator unexpectedLatePaneCounter = + createAggregator("ERROR_unexpectedLatePane", new SumLongFn()); + final Aggregator 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>() { + final Aggregator savedFileCounter = + createAggregator("savedFile", new SumLongFn()); + final Aggregator 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.of(null, outputFile)); + } + })) + // Clear fancy triggering from above. + .apply(Window.>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.create()) + .apply( + ParDo.named(name + ".Index") + .of(new DoFnWithContext>, Done>() { + final Aggregator unexpectedLateCounter = + createAggregator("ERROR_unexpectedLate", new SumLongFn()); + final Aggregator unexpectedEarlyCounter = + createAggregator("ERROR_unexpectedEarly", new SumLongFn()); + final Aggregator unexpectedIndexCounter = + createAggregator("ERROR_unexpectedIndex", new SumLongFn()); + final Aggregator 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 applyPrim(PCollection events) { + return NexmarkUtils.castToKnownSize(name, applyTyped(events)); + } +} 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 000000000000..9841421ee15e --- /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.) + * + *

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 applyTyped(PCollection events) { + return events.apply(JUST_BIDS) + .apply( + ParDo.named(name + ".Rekey") + .of(new DoFn>() { + @Override + public void processElement(ProcessContext c) { + Bid bid = c.element(); + c.output(KV.of(bid.bidder, (Void) null)); + } + })) + .apply(Window.>into( + Sessions.withGapDuration(Duration.standardSeconds(configuration.windowSizeSec))) + .triggering(Repeatedly.forever(AfterPane.elementCountAtLeast(configuration.maxLogEvents))) + .discardingFiredPanes() + .withAllowedLateness(Duration.standardSeconds(configuration.occasionalDelaySec / 2))) + .apply(Count.perKey()) + .apply( + ParDo.named(name + ".ToResult") + .of(new DoFn, BidsPerSession>() { + @Override + public void processElement(ProcessContext c) { + c.output(new BidsPerSession(c.element().getKey(), c.element().getValue())); + } + })); + } + + @Override + protected PCollection applyPrim(PCollection events) { + return NexmarkUtils.castToKnownSize(name, applyTyped(events)); + } +} 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 000000000000..dd39971654fb --- /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.) + *

+ *

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 applyTyped(PCollection events) { + return events + .apply(JUST_BIDS) + .apply( + ParDo.named(name + ".Rekey") + .of(new DoFn>() { + @Override + public void processElement(ProcessContext c) { + Bid bid = c.element(); + c.output(KV.of(bid.bidder, (Void) null)); + } + })) + .apply(Window.>into(new GlobalWindows()) + .triggering( + Repeatedly.forever( + AfterProcessingTime.pastFirstElementInPane() + .plusDelayOf( + Duration.standardSeconds(configuration.windowSizeSec)))) + .discardingFiredPanes() + .withAllowedLateness(Duration.ZERO)) + .apply(Count.perKey()) + .apply( + ParDo.named(name + ".ToResult") + .of(new DoFn, BidsPerSession>() { + @Override + public void processElement(ProcessContext c) { + c.output( + new BidsPerSession(c.element().getKey(), c.element().getValue())); + } + })); + } + + @Override + protected PCollection applyPrim(PCollection events) { + return NexmarkUtils.castToKnownSize(name, applyTyped(events)); + } +} 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 000000000000..462d4262237a --- /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 { + public Simulator(NexmarkConfiguration configuration) { + super(NexmarkUtils.standardEventIterator(configuration)); + } + + @Override + protected void run() { + TimestampedValue 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 result = + TimestampedValue.of(resultBid, timestampedEvent.getTimestamp()); + addResult(result); + } + } + + public Query1Model(NexmarkConfiguration configuration) { + super(configuration); + } + + @Override + public AbstractSimulator simulator() { + return new Simulator(configuration); + } + + @Override + protected Collection toCollection(Iterator> itr) { + return toValueTimestampOrder(itr); + } +} 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 000000000000..cede2f36eb7c --- /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: + * + *

+ * SELECT Rstream(auction, price)
+ * FROM Bid [NOW]
+ * WHERE auction = 1007 OR auction = 1020 OR auction = 2001 OR auction = 2019 OR auction = 2087;
+ * 
+ * + *

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 applyTyped(PCollection 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() { + @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() { + @Override + public void processElement(ProcessContext c) { + Bid bid = c.element(); + c.output(new AuctionPrice(bid.auction, bid.price)); + } + })); + } + + @Override + protected PCollection applyPrim(PCollection events) { + return NexmarkUtils.castToKnownSize(name, applyTyped(events)); + } +} 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 000000000000..6ccfeeb9d7ba --- /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 { + public Simulator(NexmarkConfiguration configuration) { + super(NexmarkUtils.standardEventIterator(configuration)); + } + + @Override + protected void run() { + TimestampedValue 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 result = + TimestampedValue.of(auctionPrice, timestampedEvent.getTimestamp()); + addResult(result); + } + } + + public Query2Model(NexmarkConfiguration configuration) { + super(configuration); + } + + @Override + public AbstractSimulator simulator() { + return new Simulator(configuration); + } + + @Override + protected Collection toCollection(Iterator> itr) { + return toValueTimestampOrder(itr); + } +} 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 000000000000..5b9b17bc6a79 --- /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: + * + *

+ * 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;
+ * 
+ * + *

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. + * + *

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>> AUCTION_LIST_CODED_TAG = + StateTags.value("left", ListCoder.of(Auction.CODER)); + private static final StateTag> 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. + * + *

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. + * + *

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> { + private final Aggregator newAuctionCounter = + createAggregator("newAuction", new SumLongFn()); + private final Aggregator newPersonCounter = + createAggregator("newPerson", new SumLongFn()); + private final Aggregator newNewOutputCounter = + createAggregator("newNewOutput", new SumLongFn()); + private final Aggregator newOldOutputCounter = + createAggregator("newOldOutput", new SumLongFn()); + private final Aggregator oldNewOutputCounter = + createAggregator("oldNewOutput", new SumLongFn()); + public final Aggregator 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 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> 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 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 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 getFatalCount() { + return joinDoFn.fatalCounter; + } + + private PCollection applyTyped(PCollection events) { + // Batch into incremental results windows. + events = events.apply( + Window.into(FixedWindows.of(Duration.standardSeconds(configuration.windowSizeSec)))); + + PCollection> auctionsBySellerId = + events + // Only want the new auction events. + .apply(JUST_NEW_AUCTIONS) + + // We only want auctions in category 10. + .apply(Filter.byPredicate(new SerializableFunction() { + @Override + public Boolean apply(Auction auction) { + return auction.category == 10; + } + }).named(name + ".InCategory")) + + // Key auctions by their seller id. + .apply(AUCTION_BY_SELLER); + + PCollection> 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() { + @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.create()) + .apply(ParDo.named(name + ".Join").of(joinDoFn)) + + // Project what we want. + .apply( + ParDo.named(name + ".Project") + .of(new DoFn, 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 applyPrim(PCollection events) { + return NexmarkUtils.castToKnownSize(name, applyTyped(events)); + } +} 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 000000000000..b865eda61337 --- /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 { + /** Auctions, indexed by seller id. */ + private final Multimap newAuctions; + + /** Persons, indexed by id. */ + private final Map 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 result = TimestampedValue.of( + new NameCityStateId(person.name, person.city, person.state, auction.id), timestamp); + addResult(result); + } + + @Override + protected void run() { + TimestampedValue 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 Collection toCollection(Iterator> itr) { + return toValue(itr); + } +} 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 000000000000..bc695b7fafb3 --- /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: + * + *

+ * 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;
+ * 
+ * + *

For extra spiciness our implementation differs slightly from the above: + *

    + *
  • We select both the average winning price and the category. + *
  • We don't bother joining with a static category table, since it's contents are never used. + *
  • We only consider bids which are above the auction's reserve price. + *
  • We accept the highest-price, earliest valid bid as the winner. + *
  • We calculate the averages oven a sliding window of size {@code windowSizeSec} and + * period {@code windowPeriodSec}. + *
+ */ +class Query4 extends NexmarkQuery { + private final Monitor winningBidsMonitor; + + public Query4(NexmarkConfiguration configuration) { + super(configuration, "Query4"); + winningBidsMonitor = new Monitor<>(name + ".WinningBids", "winning"); + } + + private PCollection applyTyped(PCollection events) { + PCollection 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>() { + @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.>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.perKey().withHotKeyFanout(configuration.fanout)) + + // For testing against Query4Model, capture which results are 'final'. + .apply( + ParDo.named(name + ".Project") + .of(new DoFn, 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 applyPrim(PCollection events) { + return NexmarkUtils.castToKnownSize(name, applyTyped(events)); + } +} 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 000000000000..24103067020d --- /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 { + /** The prices and categories for all winning bids in the last window size. */ + private final List> 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> 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 counts = new TreeMap<>(); + Map totals = new TreeMap<>(); + for (TimestampedValue 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 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> 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 timestampedWinningBid = nextInput(); + if (timestampedWinningBid == null) { + prune(NexmarkUtils.END_OF_TIME); + for (TimestampedValue 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> relevantResults( + Iterable> results) { + // Find the last (in processing time) reported average price for each category. + Map> finalAverages = new TreeMap<>(); + for (TimestampedValue 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 Collection toCollection(Iterator> itr) { + return toValue(itr); + } +} 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 000000000000..91a4a28c67c4 --- /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: + * + *
+ * 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);
+ * 
+ * + *

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 applyTyped(PCollection events) { + return events + // Only want the bid events. + .apply(JUST_BIDS) + // Window the bids into sliding windows. + .apply(Window.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.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>>() { + @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, Long>>() { + @Override + public KV, Long> apply( + KV, Long> left, KV, Long> right) { + List leftBestAuctions = left.getKey(); + long leftCount = left.getValue(); + List rightBestAuctions = right.getKey(); + long rightCount = right.getValue(); + if (leftCount > rightCount) { + return left; + } else if (leftCount < rightCount) { + return right; + } else { + List 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, 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 applyPrim(PCollection events) { + return NexmarkUtils.castToKnownSize(name, applyTyped(events)); + } +} 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 000000000000..a7dd8f0bc8a7 --- /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 { + /** Time of bids still contributing to open windows, indexed by their auction id. */ + private final Map> 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 counts = new TreeMap<>(); + long maxCount = 0L; + for (Map.Entry> 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 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> entry : bids.entrySet()) { + long auction = entry.getKey(); + Iterator 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 existing = bids.get(bid.auction); + if (existing == null) { + existing = new ArrayList<>(); + bids.put(bid.auction, existing); + } + existing.add(timestamp); + } + + @Override + public void run() { + TimestampedValue 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 Collection toCollection(Iterator> itr) { + return toValue(itr); + } +} 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 000000000000..49c0d68321c9 --- /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: + * + *

+ * 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;
+ * 
+ * + *

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, Long> { + private final int maxNumBids; + + public MovingMeanSellingPrice(int maxNumBids) { + this.maxNumBids = maxNumBids; + } + + @Override + public List createAccumulator() { + return new ArrayList<>(); + } + + @Override + public List addInput(List 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 mergeAccumulators(Iterable> accumulators) { + List result = new ArrayList<>(); + for (List 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 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 applyTyped(PCollection 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>() { + @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.>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.perKey(new MovingMeanSellingPrice(10))) + + // Project into our datatype. + .apply( + ParDo.named(name + ".Select") + .of(new DoFn, SellerPrice>() { + @Override + public void processElement(ProcessContext c) { + c.output(new SellerPrice(c.element().getKey(), c.element().getValue())); + } + })); + } + + @Override + protected PCollection applyPrim(PCollection events) { + return NexmarkUtils.castToKnownSize(name, applyTyped(events)); + } +} 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 000000000000..639ec9f76a1a --- /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 { + /** The cumulative count of winning bids, indexed by seller id. */ + private final Map numWinningBidsPerSeller; + + /** The cumulative total of winning bid prices, indexed by seller id. */ + private final Map 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 intermediateResult = TimestampedValue.of( + new SellerPrice(auction.seller, Math.round((double) total / count)), timestamp); + addIntermediateResult(intermediateResult); + } + + + @Override + protected void run() { + TimestampedValue 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> relevantResults( + Iterable> results) { + // Find the last (in processing time) reported average price for each seller. + Map> finalAverages = new TreeMap<>(); + for (TimestampedValue 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 Collection toCollection(Iterator> itr) { + return toValue(itr); + } +} 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 000000000000..1f63b35bdca8 --- /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: + * + *

+ * 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);
+ * 
+ * + *

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 applyTyped(PCollection events) { + // Window the bids. + PCollection slidingBids = events.apply(JUST_BIDS).apply( + Window.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 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() { + @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 applyPrim(PCollection events) { + return NexmarkUtils.castToKnownSize(name, applyTyped(events)); + } +} 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 000000000000..e8351336ad5b --- /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 { + /** Bids with highest bid price seen in the current window. */ + private final List 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 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 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 Collection toCollection(Iterator> itr) { + return toValueOrder(itr); + } +} 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 000000000000..e58453bca361 --- /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: + * + *

+ * SELECT Rstream(P.id, P.name, A.reserve)
+ * FROM Person [RANGE 12 HOUR] P, Auction [RANGE 12 HOUR] A
+ * WHERE P.id = A.seller;
+ * 
+ * + *

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 applyTyped(PCollection events) { + // Window and key new people by their id. + PCollection> personsById = + events.apply(JUST_NEW_PERSONS) + .apply(Window.into( + FixedWindows.of(Duration.standardSeconds(configuration.windowSizeSec))) + .named("Query8.WindowPersons")) + .apply(PERSON_BY_ID); + + // Window and key new auctions by their id. + PCollection> auctionsBySeller = + events.apply(JUST_NEW_AUCTIONS) + .apply(Window.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.create()) + .apply( + ParDo.named(name + ".Select") + .of(new DoFn, 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 applyPrim(PCollection events) { + return NexmarkUtils.castToKnownSize(name, applyTyped(events)); + } +} 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 000000000000..00f7355fe1d1 --- /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 { + /** New persons seen in the current window, indexed by id. */ + private final Map newPersons; + + /** New auctions seen in the current window, indexed by seller id. */ + private final Multimap 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 entry : newPersons.entrySet()) { + NexmarkUtils.info("retire: %s", entry.getValue()); + } + newPersons.clear(); + } + + /** + * Retire all auctions added in last window. + */ + private void retireAuctions() { + for (Map.Entry 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 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 Collection toCollection(Iterator> itr) { + return toValue(itr); + } +} 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 000000000000..2c0a5266a391 --- /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 applyTyped(PCollection events) { + return events.apply(new WinningBids(name, configuration)); + } + + @Override + protected PCollection applyPrim(PCollection events) { + return NexmarkUtils.castToKnownSize(name, applyTyped(events)); + } +} 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 000000000000..1fad648c954a --- /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 Collection toCollection(Iterator> itr) { + return toValue(itr); + } +} 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 000000000000..62f107604680 --- /dev/null +++ b/integration/java/src/main/java/org/apache/beam/integration/nexmark/README.md @@ -0,0 +1,147 @@ +# 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-0.1.0-incubating-SNAPSHOT.jar \ + org.apache.beam.integration.nexmark.NexmarkGoogleDriver \ + --project= \ + --zone= \ + --workerMachineType=n1-highmem-8 \ + --stagingLocation= \ + --streaming=true \ + --sourceType=PUBSUB \ + --pubSubMode=PUBLISH_ONLY \ + --pubsubTopic= \ + --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-0.1.0-incubating-SNAPSHOT.jar \ + org.apache.beam.integration.nexmark.NexmarkGoogleDriver \ + --project= \ + --zone= \ + --workerMachineType=n1-highmem-8 \ + --stagingLocation= \ + --streaming=true \ + --sourceType=PUBSUB \ + --pubSubMode=SUBSCRIBE_ONLY \ + --pubsubSubscription= \ + --resourceNameMode=VERBATIM \ + --manageResources=false \ + --monitorJobs=false \ + --numWorkers=64 \ + --maxNumWorkers=64 \ + --query=10 \ + --usePubsubPublishTime=true \ + --outputPath= \ + --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. 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 000000000000..4324b994f8c3 --- /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_CODER = VarLongCoder.of(); + + public static final Coder CODER = new AtomicCoder() { + @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); + } + } +} 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 000000000000..2898251f62c5 --- /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. + * + *

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 { + 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 { + /** 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 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 currentEvent; + + /** Events which have been held back so as to force them to be late. */ + private Queue 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 getCheckpointMarkCoder() { + return Generator.Checkpoint.CODER_INSTANCE; + } + + @Override + public List generateInitialSplits( + int desiredNumSplits, PipelineOptions options) { + NexmarkUtils.error( + "splitting unbounded source %s into %d sub-sources", config, numEventGenerators); + List 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 getDefaultOutputCoder() { + return Event.CODER; + } + + @Override + public String toString() { + return String.format( + "UnboundedEventSource(%d, %d)", config.getStartEventId(), config.getStopEventId()); + } +} 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 000000000000..16f901c7a6c4 --- /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: + * + *

+ * 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
+ * 
+ * + *

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. + * + *

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> { + /** 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 { + private static final AuctionOrBidWindowCoder INSTANCE = new AuctionOrBidWindowCoder(); + private static final Coder SUPER_CODER = IntervalWindow.getCoder(); + private static final Coder ID_CODER = VarLongCoder.of(); + private static final Coder 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 { + /** Expected duration of auctions in ms. */ + private final long expectedAuctionDurationMs; + + public AuctionOrBidWindowFn(long expectedAuctionDurationMs) { + this.expectedAuctionDurationMs = expectedAuctionDurationMs; + } + + @Override + public Collection 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 idToTrueAuctionWindow = new TreeMap<>(); + Map> idToBidAuctionWindows = new TreeMap<>(); + for (AuctionOrBidWindow window : c.windows()) { + if (window.isAuctionWindow()) { + idToTrueAuctionWindow.put(window.auction, window); + } else { + List 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 bidWindows = idToBidAuctionWindows.get(auction); + if (bidWindows != null) { + List 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 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. + * + *

+ * 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. + * + *

+ * 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 apply(PCollection 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> auctionsById = + events.apply(NexmarkQuery.JUST_NEW_AUCTIONS).apply(NexmarkQuery.AUCTION_BY_ID); + + // Key bids by their auction id. + PCollection> 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.create()) + + // Filter and select. + .apply( + ParDo.named(name + ".Join") + .of(new DoFn, AuctionBid>() { + final Aggregator noAuctionCounter = + createAggregator("noAuction", new SumLongFn()); + final Aggregator underReserveCounter = + createAggregator("underReserve", new SumLongFn()); + final Aggregator 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)); + } + })); + } +} 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 000000000000..b61aed1c220b --- /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 { + /** Auctions currently still open, indexed by auction id. */ + private final Map openAuctions; + + /** The ids of auctions known to be closed. */ + private final Set closedAuctions; + + /** Current best valid bids for open auctions, indexed by auction id. */ + private final Map bestBids; + + /** Bids for auctions we havn't seen yet. */ + private final List 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 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 nextWinningBid(long timestamp) { + Map> toBeRetired = new TreeMap<>(); + for (Map.Entry entry : openAuctions.entrySet()) { + if (entry.getValue().expires <= timestamp) { + List 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> 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 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 result = nextWinningBid(lastTimestamp); + if (result != null) { + addResult(result); + return; + } + } + + TimestampedValue timestampedEvent = nextInput(); + if (timestampedEvent == null) { + // No more events. Flush any still open auctions. + TimestampedValue 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. + } +} diff --git a/pom.xml b/pom.xml index 2a6e584fb0b7..d6dd0749eee5 100644 --- a/pom.xml +++ b/pom.xml @@ -135,6 +135,7 @@ and examples/java. Until these are refactored out, we need to modify the build order. --> runners/google-cloud-dataflow-java runners + integration/java From 2fe7f059fe20e0391aa177ef3770ce438ae0d2b9 Mon Sep 17 00:00:00 2001 From: Mark Shields Date: Thu, 2 Jun 2016 17:32:49 -0700 Subject: [PATCH 2/4] Cleanup pom --- integration/java/pom.xml | 143 +++++++++++------- .../nexmark/BEAM_ON_FLINK_ON_GCP.md | 9 +- .../apache/beam/integration/nexmark/README.md | 4 +- 3 files changed, 93 insertions(+), 63 deletions(-) diff --git a/integration/java/pom.xml b/integration/java/pom.xml index b158ff4c555c..b160b56123b0 100644 --- a/integration/java/pom.xml +++ b/integration/java/pom.xml @@ -23,18 +23,20 @@ org.apache.beam parent - 0.1.0-incubating-SNAPSHOT + 0.2.0-incubating-SNAPSHOT ../../pom.xml java-integration-all + Apache Beam :: Integration Tests :: Java All - Beam SDK Java All provides a simple, Java-based - interface for processing virtually any size data. This - artifact includes some Java SDK integration tests. jar + + true + + @@ -47,11 +49,18 @@ + org.apache.maven.plugins maven-compiler-plugin - 3.1 + + + + org.apache.maven.plugins + maven-surefire-plugin - 1.7 - 1.7 + + + + @@ -65,10 +74,17 @@ maven-checkstyle-plugin + + + + org.apache.maven.plugins + maven-source-plugin + + + org.apache.maven.plugins maven-shade-plugin - 2.4.1 package @@ -76,6 +92,7 @@ shade + ${project.artifactId}-bundled-${project.version} *:* @@ -96,6 +113,11 @@ + + org.apache.maven.plugins + maven-jar-plugin + + org.apache.avro @@ -116,25 +138,6 @@ - - org.apache.maven.plugins - maven-jar-plugin - - - default-jar - - jar - - - - default-test-jar - - test-jar - - - - - org.jacoco @@ -152,14 +155,8 @@ - org.apache.beam - google-cloud-dataflow-java-runner - ${project.version} - - - - org.apache.beam - runners-core + org.apache.beam.runners + google-cloud-dataflow-java ${project.version} @@ -172,12 +169,60 @@ - org.apache.beam - flink-runner_2.10 + org.apache.beam.runners + flink_2.10 ${project.version} + + org.apache.flink + flink-shaded-hadoop2 + 1.0.3 + provided + + + + com.google.apis + google-api-services-dataflow + ${dataflow.version} + + + + joda-time + joda-time + + + + com.fasterxml.jackson.core + jackson-core + ${jackson.version} + + + + com.google.apis + google-api-services-bigquery + ${bigquery.version} + + + + com.google.cloud.bigdataoss + gcsio + ${google-cloud-bigdataoss.version} + + + + com.fasterxml.jackson.core + jackson-annotations + ${jackson.version} + + + + com.fasterxml.jackson.core + jackson-databind + ${jackson.version} + + org.apache.avro avro @@ -200,6 +245,7 @@ org.slf4j slf4j-api ${slf4j.version} + compile @@ -214,30 +260,21 @@ com.google.code.findbugs jsr305 ${jsr305.version} - - - - commons-logging - commons-logging - 1.2 + runtime log4j log4j 1.2.17 - - - - org.apache.logging.log4j - log4j-api - 2.4 + runtime junit junit - provided + ${junit.version} + compile @@ -245,11 +282,7 @@ netty-tcnative-boringssl-static 1.1.33.Fork13 ${os.detected.classifier} - - - org.apache.beam.runners - direct - 0.1.0-incubating-SNAPSHOT + runtime 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 index 7b6131e69cab..2bb18bb21361 100644 --- 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 @@ -150,9 +150,6 @@ cp -f \ ~/masters ~/slaves \ ~/log4j.properties \ $FLINK/conf/ -cp -f \ - $BEAM/integration/java/target/java-integration-all-0.1.0-incubating-SNAPSHOT.jar \ - $FLINK/lib/ ``` Package configured Flink for distribution to workers: @@ -191,14 +188,14 @@ Bring up the Flink monitoring UI: Distribute the Beam + NexMark jar to all workers: ``` -$GSUTIL cp $BEAM/integration/java/target/java-integration-all-0.1.0-incubating-SNAPSHOT.jar gs://nexmark +$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-0.1.0-incubating-SNAPSHOT.jar ~/$FLINK_VER/lib/" + --command "gsutil cp gs://nexmark/java-integration-all-bundled-0.2.0-incubating-SNAPSHOT.jar ~/$FLINK_VER/lib/" done ``` @@ -228,7 +225,7 @@ $GCLOUD compute ssh \ $MASTER \ --command "~/$FLINK_VER/bin/flink run \ -c org.apache.beam.integration.nexmark.NexmarkFlinkDriver \ - ~/$FLINK_VER/lib/java-integration-all-0.1.0-incubating-SNAPSHOT.jar \ + ~/$FLINK_VER/lib/java-integration-all-bundled-0.2.0-incubating-SNAPSHOT.jar \ --project=$PROJECT \ --streaming=true \ --query=0 \ 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 index 62f107604680..e046e460dafa 100644 --- 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 @@ -83,7 +83,7 @@ An example invocation for **Query10** on the Google Cloud Dataflow service. ``` -java -cp integration/java/target/java-integration-all-0.1.0-incubating-SNAPSHOT.jar \ +java -cp integration/java/target/java-integration-all-bundled-0.2.0-incubating-SNAPSHOT.jar \ org.apache.beam.integration.nexmark.NexmarkGoogleDriver \ --project= \ --zone= \ @@ -116,7 +116,7 @@ java -cp integration/java/target/java-integration-all-0.1.0-incubating-SNAPSHOT. ``` ``` -java -cp integration/java/target/java-integration-all-0.1.0-incubating-SNAPSHOT.jar \ +java -cp integration/java/target/java-integration-all-bundled-0.2.0-incubating-SNAPSHOT.jar \ org.apache.beam.integration.nexmark.NexmarkGoogleDriver \ --project= \ --zone= \ From f015d7462e982a6a8d37d51627c5042906c823b5 Mon Sep 17 00:00:00 2001 From: Mark Shields Date: Thu, 2 Jun 2016 17:54:20 -0700 Subject: [PATCH 3/4] Port unit tests. --- .../nexmark/BoundedEventSourceTest.java | 71 +++++++++++ .../integration/nexmark/GeneratorTest.java | 111 ++++++++++++++++++ .../beam/integration/nexmark/QueryTest.java | 103 ++++++++++++++++ .../nexmark/UnboundedEventSourceTest.java | 109 +++++++++++++++++ 4 files changed, 394 insertions(+) create mode 100644 integration/java/src/test/java/org/apache/beam/integration/nexmark/BoundedEventSourceTest.java create mode 100644 integration/java/src/test/java/org/apache/beam/integration/nexmark/GeneratorTest.java create mode 100644 integration/java/src/test/java/org/apache/beam/integration/nexmark/QueryTest.java create mode 100644 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/BoundedEventSourceTest.java b/integration/java/src/test/java/org/apache/beam/integration/nexmark/BoundedEventSourceTest.java new file mode 100644 index 000000000000..f0172679e463 --- /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); + } +} 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 000000000000..bbaee26c25c4 --- /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 long consume(long n, Iterator itr) { + for (long i = 0; i < n; i++) { + assertTrue(itr.hasNext()); + itr.next(); + } + return n; + } + + private long consume(Iterator 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 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); + } +} 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 000000000000..860fa78b13a6 --- /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> 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)); + } +} 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 000000000000..5d72f775d6ab --- /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 seenPersonIds = new HashSet<>(); + private Set 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 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 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()); + } +} From 885655ca36db9cd0f0c9fd061fe9473a32e2ad8b Mon Sep 17 00:00:00 2001 From: Mark Shields Date: Fri, 3 Jun 2016 13:12:48 -0700 Subject: [PATCH 4/4] Add license to readmes --- .../nexmark/BEAM_ON_FLINK_ON_GCP.md | 19 +++++++++++++++++++ .../apache/beam/integration/nexmark/README.md | 19 +++++++++++++++++++ 2 files changed, 38 insertions(+) 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 index 2bb18bb21361..d1b51e8a02be 100644 --- 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 @@ -1,3 +1,22 @@ + + # Running NexMark on Beam on Flink on Google Compute Platform Here's how to create a cluster of VMs on Google Compute Platform, deploy 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 index e046e460dafa..5e3332722496 100644 --- 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 @@ -1,3 +1,22 @@ + + # NEXMark integration suite This is a suite of pipelines inspired by the 'continuous data stream'