From 768acfeafd291af5606fecd056c059d0bf871051 Mon Sep 17 00:00:00 2001 From: Mark Shields Date: Thu, 7 Apr 2016 11:06:51 -0700 Subject: [PATCH 01/14] Pub/sub client with gRPC impl --- pom.xml | 1 + sdks/java/core/pom.xml | 34 ++ .../cloud/dataflow/sdk/io/PubsubClient.java | 176 ++++++++ .../dataflow/sdk/io/PubsubGrpcClient.java | 391 ++++++++++++++++++ 4 files changed, 602 insertions(+) create mode 100644 sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/io/PubsubClient.java create mode 100644 sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/io/PubsubGrpcClient.java diff --git a/pom.xml b/pom.xml index 4c6ce7c6fbe8..6a1206751a7e 100644 --- a/pom.xml +++ b/pom.xml @@ -103,6 +103,7 @@ 1.7.7 v2-rev248-1.21.0 0.2.3 + 0.0.2 v2-rev6-1.21.0 v1b3-rev22-1.21.0 0.5.160222 diff --git a/sdks/java/core/pom.xml b/sdks/java/core/pom.xml index 6595a214e4f8..97621362cbf8 100644 --- a/sdks/java/core/pom.xml +++ b/sdks/java/core/pom.xml @@ -389,6 +389,40 @@ 0.12.0 + + com.google.auth + google-auth-library-oauth2-http + 0.3.1 + + + + com.google.guava + guava-jdk5 + + + + + + io.netty + netty-handler + 4.1.0.Beta8 + + + + com.google.api.grpc + grpc-pubsub-v1 + ${pubsubgrpc.version} + + + + com.google.guava + guava-jdk5 + + + + com.google.cloud.bigtable bigtable-protos diff --git a/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/io/PubsubClient.java b/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/io/PubsubClient.java new file mode 100644 index 000000000000..3f9cf74a5ea7 --- /dev/null +++ b/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/io/PubsubClient.java @@ -0,0 +1,176 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package com.google.cloud.dataflow.sdk.io; + +import java.io.IOException; +import java.util.Collection; + +/** + * A helper interface for talking to Pubsub via an underlying transport. + */ +public interface PubsubClient extends AutoCloseable { + /** + * Gracefully close the underlying transport. + */ + @Override + void close(); + + /** + * A message to be sent to Pubsub. + */ + class OutgoingMessage { + /** + * Underlying (encoded) element. + */ + public final byte[] elementBytes; + + /** + * Timestamp for element (ms since epoch). + */ + public final long timestampMsSinceEpoch; + + public OutgoingMessage(byte[] elementBytes, long timestampMsSinceEpoch) { + this.elementBytes = elementBytes; + this.timestampMsSinceEpoch = timestampMsSinceEpoch; + } + } + + /** + * A message received from Pubsub. + */ + class IncomingMessage { + /** + * Underlying (encoded) element. + */ + public final byte[] elementBytes; + + /** + * Timestamp for element (ms since epoch). Either Pubsub's processing time, + * or the custom timestamp associated with the message. + */ + public final long timestampMsSinceEpoch; + + /** + * Timestamp (in system time) at which we requested the message (ms since epoch). + */ + public final long requestTimeMsSinceEpoch; + + /** + * Id to pass back to Pubsub to acknowledge receipt of this message. + */ + public final String ackId; + + /** + * Id to pass to the runner to distinguish this message from all others. + */ + public final byte[] recordId; + + public IncomingMessage( + byte[] elementBytes, + long timestampMsSinceEpoch, + long requestTimeMsSinceEpoch, + String ackId, + byte[] recordId) { + this.elementBytes = elementBytes; + this.timestampMsSinceEpoch = timestampMsSinceEpoch; + this.requestTimeMsSinceEpoch = requestTimeMsSinceEpoch; + this.ackId = ackId; + this.recordId = recordId; + } + } + + /** + * Publish {@code outgoingMessages} to Pubsub {@code topic}. Return number of messages + * published. + * + * @throws IOException + */ + int publish(String topic, Iterable outgoingMessages) throws IOException; + + /** + * Request the next batch of up to {@code batchSize} messages from {@code subscription}. + * Return the received messages, or empty collection if none were available. Does not + * wait for messages to arrive. Returned messages will record heir request time + * as {@code requestTimeMsSinceEpoch}. + * + * @throws IOException + */ + Collection pull( + long requestTimeMsSinceEpoch, String subscription, int + batchSize) throws IOException; + + /** + * Acknowldege messages from {@code subscription} with {@code ackIds}. + * + * @throws IOException + */ + void acknowledge(String subscription, Iterable ackIds) throws IOException; + + /** + * Modify the ack deadline for messages from {@code subscription} with {@code ackIds} to + * be {@code deadlineSeconds} from now. + * + * @throws IOException + */ + void modifyAckDeadline(String subscription, Iterable ackIds, int deadlineSeconds) + throws IOException; + + /** + * Create {@code topic}. + * + * @throws IOException + */ + void createTopic(String topic) throws IOException; + + /* + * Delete {@code topic}. + * + * @throws IOException + */ + void deleteTopic(String topic) throws IOException; + + /** + * Return a list of topics for {@code project}. + * + * @throws IOException + */ + Collection listTopics(String project) throws IOException; + + /** + * Create {@code subscription} to {@code topic}. + * + * @throws IOException + */ + void createSubscription(String topic, String subscription, int ackDeadlineSeconds) throws + IOException; + + /** + * Delete {@code subscription}. + * + * @throws IOException + */ + void deleteSubscription(String subscription) throws IOException; + + /** + * Return a list of subscriptions for {@code topic} in {@code project}. + * + * @throws IOException + */ + Collection listSubscriptions(String project, String topic) throws IOException; +} diff --git a/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/io/PubsubGrpcClient.java b/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/io/PubsubGrpcClient.java new file mode 100644 index 000000000000..2d75679f7846 --- /dev/null +++ b/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/io/PubsubGrpcClient.java @@ -0,0 +1,391 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package com.google.cloud.dataflow.sdk.io; + +import com.google.api.client.util.DateTime; +import com.google.auth.oauth2.GoogleCredentials; +import com.google.cloud.dataflow.sdk.options.GcpOptions; +import com.google.common.base.Preconditions; +import com.google.common.collect.ImmutableList; +import com.google.common.hash.Hashing; +import com.google.protobuf.ByteString; +import com.google.protobuf.Timestamp; +import com.google.pubsub.v1.AcknowledgeRequest; +import com.google.pubsub.v1.DeleteSubscriptionRequest; +import com.google.pubsub.v1.DeleteTopicRequest; +import com.google.pubsub.v1.ListSubscriptionsRequest; +import com.google.pubsub.v1.ListSubscriptionsResponse; +import com.google.pubsub.v1.ListTopicsRequest; +import com.google.pubsub.v1.ListTopicsResponse; +import com.google.pubsub.v1.ModifyAckDeadlineRequest; +import com.google.pubsub.v1.PublishRequest; +import com.google.pubsub.v1.PublishResponse; +import com.google.pubsub.v1.PublisherGrpc; +import com.google.pubsub.v1.PubsubMessage; +import com.google.pubsub.v1.PullRequest; +import com.google.pubsub.v1.PullResponse; +import com.google.pubsub.v1.ReceivedMessage; +import com.google.pubsub.v1.SubscriberGrpc; +import com.google.pubsub.v1.Subscription; +import com.google.pubsub.v1.Topic; +import io.grpc.Channel; +import io.grpc.ClientInterceptors; +import io.grpc.ManagedChannel; +import io.grpc.auth.ClientAuthInterceptor; +import io.grpc.netty.GrpcSslContexts; +import io.grpc.netty.NegotiationType; +import io.grpc.netty.NettyChannelBuilder; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.Collection; +import java.util.Collections; +import java.util.List; +import java.util.Map; +import java.util.concurrent.Executors; +import java.util.concurrent.TimeUnit; + +import javax.annotation.Nullable; + +/** + * A helper class for talking to Pubsub via grpc. + */ +public class PubsubGrpcClient implements PubsubClient { + private static final String PUBSUB_ADDRESS = "pubsub.googleapis.com"; + private static final int PUBSUB_PORT = 443; + private static final List PUBSUB_SCOPES = + Collections.singletonList("https://www.googleapis.com/auth/pubsub"); + private static final int LIST_BATCH_SIZE = 1000; + + /** + * Timeout for grpc calls (in s). + */ + private static final int TIMEOUT_S = 15; + + /** + * Underlying netty channel, or {@literal null} if closed. + */ + @Nullable + private ManagedChannel publisherChannel; + + /** + * Credentials determined from options and environment. + */ + private final GoogleCredentials credentials; + + /** + * Label to use for custom timestamps, or {@literal null} if should use Pubsub publish time + * instead. + */ + @Nullable + private final String timestampLabel; + + /** + * Label to use for custom ids, or {@literal null} if should use Pubsub provided ids. + */ + @Nullable + private final String idLabel; + + /** + * Cached stubs, or null if not cached. + */ + @Nullable + private PublisherGrpc.PublisherBlockingStub cachedPublisherStub; + private SubscriberGrpc.SubscriberBlockingStub cachedSubscriberStub; + + private PubsubGrpcClient( + @Nullable String timestampLabel, @Nullable String idLabel, + ManagedChannel publisherChannel, GoogleCredentials credentials) { + this.timestampLabel = timestampLabel; + this.idLabel = idLabel; + this.publisherChannel = publisherChannel; + this.credentials = credentials; + } + + /** + * Construct a new Pubsub grpc client. It should be closed via {@link #close} in order + * to ensure tidy cleanup of underlying netty resources. (Or use the try-with-resources + * construct since this class is {@link AutoCloseable}. If non-{@literal null}, use + * {@code timestampLabel} and {@code idLabel} to store custom timestamps/ids within + * message metadata. + */ + public static PubsubGrpcClient newClient( + @Nullable String timestampLabel, @Nullable String idLabel, + GcpOptions options) throws IOException { + ManagedChannel channel = NettyChannelBuilder + .forAddress(PUBSUB_ADDRESS, PUBSUB_PORT) + .negotiationType(NegotiationType.TLS) + .sslContext(GrpcSslContexts.forClient().ciphers(null).build()) + .build(); + // TODO: GcpOptions needs to support building com.google.auth.oauth2.Credentials from the + // various command line options. It currently only supports the older + // com.google.api.client.auth.oauth2.Credentials. + GoogleCredentials credentials = GoogleCredentials.getApplicationDefault(); + return new PubsubGrpcClient(timestampLabel, idLabel, channel, credentials); + } + + /** + * Gracefully close the underlying netty channel. + */ + @Override + public void close() { + Preconditions.checkState(publisherChannel != null, "Client has already been closed"); + publisherChannel.shutdown(); + try { + publisherChannel.awaitTermination(TIMEOUT_S, TimeUnit.SECONDS); + } catch (InterruptedException e) { + // Ignore. + Thread.currentThread().interrupt(); + } + publisherChannel = null; + cachedPublisherStub = null; + cachedSubscriberStub = null; + } + + /** + * Return channel with interceptor for returning credentials. + */ + private Channel newChannel() throws IOException { + Preconditions.checkState(publisherChannel != null, "PubsubGrpcClient has been closed"); + ClientAuthInterceptor interceptor = + new ClientAuthInterceptor(credentials, Executors.newSingleThreadExecutor()); + return ClientInterceptors.intercept(publisherChannel, interceptor); + } + + /** + * Return a stub for making a publish request with a timeout. + */ + private PublisherGrpc.PublisherBlockingStub publisherStub() throws IOException { + if (cachedPublisherStub == null) { + cachedPublisherStub = PublisherGrpc.newBlockingStub(newChannel()); + } + return cachedPublisherStub.withDeadlineAfter(TIMEOUT_S, TimeUnit.SECONDS); + } + + /** + * Return a stub for making a subscribe request with a timeout. + */ + private SubscriberGrpc.SubscriberBlockingStub subscriberStub() throws IOException { + if (cachedSubscriberStub == null) { + cachedSubscriberStub = SubscriberGrpc.newBlockingStub(newChannel()); + } + return cachedSubscriberStub.withDeadlineAfter(TIMEOUT_S, TimeUnit.SECONDS); + } + + @Override + public int publish(String topic, Iterable outgoingMessages) throws IOException { + PublishRequest.Builder request = PublishRequest.newBuilder() + .setTopic(topic); + for (OutgoingMessage outgoingMessage : outgoingMessages) { + PubsubMessage.Builder message = + PubsubMessage.newBuilder() + .setData(ByteString.copyFrom(outgoingMessage.elementBytes)); + + if (timestampLabel != null) { + message.getMutableAttributes() + .put(timestampLabel, String.valueOf(outgoingMessage.timestampMsSinceEpoch)); + } + + if (idLabel != null) { + message.getMutableAttributes() + .put(idLabel, + Hashing.murmur3_128().hashBytes(outgoingMessage.elementBytes).toString()); + } + + request.addMessages(message); + } + + PublishResponse response = publisherStub().publish(request.build()); + return response.getMessageIdsCount(); + } + + @Override + public Collection pull( + long requestTimeMsSinceEpoch, + String subscription, + int batchSize) throws IOException { + PullRequest request = PullRequest.newBuilder() + .setSubscription(subscription) + .setReturnImmediately(true) + .setMaxMessages(batchSize) + .build(); + PullResponse response = subscriberStub().pull(request); + if (response.getReceivedMessagesCount() == 0) { + return ImmutableList.of(); + } + List incomingMessages = new ArrayList<>(response.getReceivedMessagesCount()); + for (ReceivedMessage message : response.getReceivedMessagesList()) { + PubsubMessage pubsubMessage = message.getMessage(); + Map attributes = pubsubMessage.getAttributes(); + + // Payload. + byte[] elementBytes = pubsubMessage.getData().toByteArray(); + + // Timestamp. + // Start with Pubsub processing time. + Timestamp timestampProto = pubsubMessage.getPublishTime(); + long timestampMsSinceEpoch = timestampProto.getSeconds() + timestampProto.getNanos() / 1000L; + if (timestampLabel != null && attributes != null) { + String timestampString = attributes.get(timestampLabel); + if (timestampString != null && !timestampString.isEmpty()) { + try { + // Try parsing as milliseconds since epoch. Note there is no way to parse a + // string in RFC 3339 format here. + // Expected IllegalArgumentException if parsing fails; we use that to fall back + // to RFC 3339. + timestampMsSinceEpoch = Long.parseLong(timestampString); + } catch (IllegalArgumentException e1) { + try { + // Try parsing as RFC3339 string. DateTime.parseRfc3339 will throw an + // IllegalArgumentException if parsing fails, and the caller should handle. + timestampMsSinceEpoch = DateTime.parseRfc3339(timestampString).getValue(); + } catch (IllegalArgumentException e2) { + // Fallback to Pubsub processing time. + } + } + } + // else: fallback to Pubsub processing time. + } + // else: fallback to Pubsub processing time. + + // Ack id. + String ackId = message.getAckId(); + Preconditions.checkState(ackId != null && !ackId.isEmpty()); + + // Record id, if any. + @Nullable byte[] recordId = null; + if (idLabel != null && attributes != null) { + String recordIdString = attributes.get(idLabel); + if (recordIdString != null && !recordIdString.isEmpty()) { + recordId = recordIdString.getBytes(); + } + } + if (recordId == null) { + recordId = pubsubMessage.getMessageId().getBytes(); + } + + incomingMessages.add(new IncomingMessage(elementBytes, timestampMsSinceEpoch, + requestTimeMsSinceEpoch, ackId, recordId)); + } + return incomingMessages; + } + + @Override + public void acknowledge(String subscription, Iterable ackIds) throws IOException { + AcknowledgeRequest request = AcknowledgeRequest.newBuilder() + .setSubscription(subscription) + .addAllAckIds(ackIds) + .build(); + subscriberStub().acknowledge(request); // ignore Empty result. + } + + @Override + public void modifyAckDeadline(String subscription, Iterable ackIds, int deadlineSeconds) + throws IOException { + ModifyAckDeadlineRequest request = + ModifyAckDeadlineRequest.newBuilder() + .setSubscription(subscription) + .addAllAckIds(ackIds) + .setAckDeadlineSeconds(deadlineSeconds) + .build(); + subscriberStub().modifyAckDeadline(request); // ignore Empty result. + } + + @Override + public void createTopic(String topic) throws IOException { + Topic request = Topic.newBuilder() + .setName(topic) + .build(); + publisherStub().createTopic(request); // ignore Topic result. + } + + @Override + public void deleteTopic(String topic) throws IOException { + DeleteTopicRequest request = DeleteTopicRequest.newBuilder().setTopic(topic).build(); + publisherStub().deleteTopic(request); // ignore Empty result. + } + + @Override + public Collection listTopics(String project) throws IOException { + ListTopicsRequest.Builder request = + ListTopicsRequest.newBuilder() + .setProject(project) + .setPageSize(LIST_BATCH_SIZE); + ListTopicsResponse response = publisherStub().listTopics(request.build()); + if (response.getTopicsCount() == 0) { + return ImmutableList.of(); + } + List topics = new ArrayList<>(response.getTopicsCount()); + while (true) { + for (Topic topic : response.getTopicsList()) { + topics.add(topic.getName()); + } + if (response.getNextPageToken().isEmpty()) { + break; + } + request.setPageToken(response.getNextPageToken()); + response = publisherStub().listTopics(request.build()); + } + return topics; + } + + @Override + public void createSubscription(String topic, String subscription, int ackDeadlineSeconds) + throws IOException { + Subscription request = Subscription.newBuilder() + .setTopic(topic) + .setName(subscription) + .setAckDeadlineSeconds(ackDeadlineSeconds) + .build(); + subscriberStub().createSubscription(request); // ignore Subscription result. + } + + @Override + public void deleteSubscription(String subscription) throws IOException { + DeleteSubscriptionRequest request = + DeleteSubscriptionRequest.newBuilder().setSubscription(subscription).build(); + subscriberStub().deleteSubscription(request); // ignore Empty result. + } + + @Override + public Collection listSubscriptions(String project, String topic) throws IOException { + ListSubscriptionsRequest.Builder request = + ListSubscriptionsRequest.newBuilder() + .setProject(project) + .setPageSize(LIST_BATCH_SIZE); + ListSubscriptionsResponse response = subscriberStub().listSubscriptions(request.build()); + if (response.getSubscriptionsCount() == 0) { + return ImmutableList.of(); + } + List subscriptions = new ArrayList<>(response.getSubscriptionsCount()); + while (true) { + for (Subscription subscription : response.getSubscriptionsList()) { + if (subscription.getTopic().equals(topic)) { + subscriptions.add(subscription.getName()); + } + } + if (response.getNextPageToken().isEmpty()) { + break; + } + request.setPageToken(response.getNextPageToken()); + response = subscriberStub().listSubscriptions(request.build()); + } + return subscriptions; + } +} From 95fe15c695a348dfb28405c195174ae6f34dba0a Mon Sep 17 00:00:00 2001 From: Mark Shields Date: Mon, 4 Apr 2016 18:09:27 -0700 Subject: [PATCH 02/14] Pub/sub sink --- .../dataflow/sdk/io/PubsubUnboundedSink.java | 303 ++++++++++++++++++ 1 file changed, 303 insertions(+) create mode 100644 sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/io/PubsubUnboundedSink.java diff --git a/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/io/PubsubUnboundedSink.java b/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/io/PubsubUnboundedSink.java new file mode 100644 index 000000000000..e90d7bbc9903 --- /dev/null +++ b/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/io/PubsubUnboundedSink.java @@ -0,0 +1,303 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package com.google.cloud.dataflow.sdk.io; + +import com.google.cloud.dataflow.sdk.coders.AtomicCoder; +import com.google.cloud.dataflow.sdk.coders.BigEndianLongCoder; +import com.google.cloud.dataflow.sdk.coders.ByteArrayCoder; +import com.google.cloud.dataflow.sdk.coders.Coder; +import com.google.cloud.dataflow.sdk.coders.CoderException; +import com.google.cloud.dataflow.sdk.coders.KvCoder; +import com.google.cloud.dataflow.sdk.coders.VarIntCoder; +import com.google.cloud.dataflow.sdk.options.GcpOptions; +import com.google.cloud.dataflow.sdk.transforms.Aggregator; +import com.google.cloud.dataflow.sdk.transforms.DoFn; +import com.google.cloud.dataflow.sdk.transforms.GroupByKey; +import com.google.cloud.dataflow.sdk.transforms.PTransform; +import com.google.cloud.dataflow.sdk.transforms.ParDo; +import com.google.cloud.dataflow.sdk.transforms.Sum; +import com.google.cloud.dataflow.sdk.transforms.windowing.AfterFirst; +import com.google.cloud.dataflow.sdk.transforms.windowing.AfterPane; +import com.google.cloud.dataflow.sdk.transforms.windowing.AfterProcessingTime; +import com.google.cloud.dataflow.sdk.transforms.windowing.GlobalWindows; +import com.google.cloud.dataflow.sdk.transforms.windowing.Repeatedly; +import com.google.cloud.dataflow.sdk.transforms.windowing.Window; +import com.google.cloud.dataflow.sdk.util.CoderUtils; +import com.google.cloud.dataflow.sdk.values.KV; +import com.google.cloud.dataflow.sdk.values.PCollection; +import com.google.cloud.dataflow.sdk.values.PDone; +import com.google.common.base.Preconditions; + +import org.joda.time.Duration; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.IOException; +import java.io.InputStream; +import java.io.OutputStream; +import java.util.ArrayList; +import java.util.List; +import java.util.concurrent.ThreadLocalRandom; + +import javax.annotation.Nullable; + +/** + * A PTransform which streams messages to pub/sub. + *
    + *
  • The underlying implementation is just a {@link DoFn} which publishes as a side effect. + *
  • We use gRPC for its speed and low memory overhead. + *
  • We try to send messages in batches while also limiting send latency. + *
  • No stats are logged. Rather some counters are used to keep track of elements and batches. + *
  • Though some background threads are used by the underlying netty system all actual pub/sub + * calls are blocking. We rely on the underlying runner to allow multiple {@link DoFn} instances + * to execute concurrently and thus hide latency. + *
+ */ +public class PubsubUnboundedSink extends PTransform, PDone> { + private static final Logger LOG = LoggerFactory.getLogger(PubsubUnboundedSink.class); + + /** + * Maximum number of messages per publish. + */ + private static final int PUBLISH_BATCH_SIZE = 1000; + + /** + * Maximum size of a publish batch, in bytes. + */ + private static final long PUBLISH_BATCH_BYTES = 400000; + + /** + * Longest delay between receiving a message and pushing it to pub/sub. + */ + private static final Duration MAX_LATENCY = Duration.standardSeconds(2); + + /** + * Period of samples for stats. + */ + private static final Duration SAMPLE_PERIOD = Duration.standardMinutes(1); + + /** + * Period of updates for stats. + */ + private static final Duration SAMPLE_UPDATE = Duration.standardSeconds(5); + + /** + * How frequently to log stats. + */ + private static final Duration LOG_PERIOD = Duration.standardSeconds(30); + + /** + * Additional sharding so that we can hide publish message latency. + */ + private static final int SCALE_OUT = 4; + + public static final Coder CODER = new + AtomicCoder() { + @Override + public void encode( + PubsubClient.OutgoingMessage value, OutputStream outStream, Context context) + throws CoderException, IOException { + ByteArrayCoder.of().encode(value.elementBytes, outStream, Context.NESTED); + BigEndianLongCoder.of().encode(value.timestampMsSinceEpoch, outStream, Context.NESTED); + } + + @Override + public PubsubClient.OutgoingMessage decode( + InputStream inStream, Context context) throws CoderException, IOException { + byte[] elementBytes = ByteArrayCoder.of().decode(inStream, Context.NESTED); + long timestampMsSinceEpoch = BigEndianLongCoder.of().decode(inStream, Context.NESTED); + return new PubsubClient.OutgoingMessage(elementBytes, timestampMsSinceEpoch); + } + }; + + // ================================================================================ + // ShardFv + // ================================================================================ + + /** + * Convert elements to messages and shard them. + */ + private class ShardFn extends DoFn> { + /** + * Number of cores available for publishing. + */ + private final int numCores; + + private final Aggregator elementCounter = + createAggregator("elements", new Sum.SumLongFn()); + + public ShardFn(int numCores) { + this.numCores = numCores; + } + + @Override + public void processElement(ProcessContext c) throws Exception { + elementCounter.addValue(1L); + byte[] elementBytes = CoderUtils.encodeToByteArray(elementCoder, c.element()); + long timestampMsSinceEpoch = c.timestamp().getMillis(); + c.output(KV.of(ThreadLocalRandom.current().nextInt(numCores * SCALE_OUT), + new PubsubClient.OutgoingMessage(elementBytes, timestampMsSinceEpoch))); + } + } + + // ================================================================================ + // WriterFn + // ================================================================================ + + /** + * Publish messages to pub/sub in batches. + */ + private class WriterFn + extends DoFn>, Void> { + + /** + * Client on which to talk to pub/sub. Null until created by {@link #startBundle}. + */ + @Nullable + private transient PubsubClient pubsubClient; + + private final Aggregator batchCounter = + createAggregator("batches", new Sum.SumLongFn()); + private final Aggregator elementCounter = + createAggregator("elements", new Sum.SumLongFn()); + private final Aggregator byteCounter = + createAggregator("bytes", new Sum.SumLongFn()); + + /** + * BLOCKING + * Send {@code messages} as a batch to pub/sub. + */ + private void publishBatch(List messages, int bytes) + throws IOException { + long nowMsSinceEpoch = System.currentTimeMillis(); + int n = pubsubClient.publish(topic, messages); + Preconditions.checkState(n == messages.size()); + batchCounter.addValue(1L); + elementCounter.addValue((long) messages.size()); + byteCounter.addValue((long) bytes); + } + + @Override + public void startBundle(Context c) throws Exception { + Preconditions.checkState(pubsubClient == null); + pubsubClient = PubsubGrpcClient.newClient(timestampLabel, idLabel, + c.getPipelineOptions().as(GcpOptions.class)); + super.startBundle(c); + } + + @Override + public void processElement(ProcessContext c) throws Exception { + List pubsubMessages = new ArrayList<>(PUBLISH_BATCH_SIZE); + int bytes = 0; + for (PubsubClient.OutgoingMessage message : c.element().getValue()) { + if (!pubsubMessages.isEmpty() + && bytes + message.elementBytes.length > PUBLISH_BATCH_BYTES) { + // Break large (in bytes) batches into smaller. + // (We've already broken by batch size using the trigger belom, though that may + // run slightly over the actual PUBLISH_BATCH_SIZE) + // BLOCKS until published. + publishBatch(pubsubMessages, bytes); + pubsubMessages.clear(); + bytes = 0; + } + pubsubMessages.add(message); + bytes += message.elementBytes.length; + } + if (!pubsubMessages.isEmpty()) { + // BLOCKS until published. + publishBatch(pubsubMessages, bytes); + } + } + + @Override + public void finishBundle(Context c) throws Exception { + pubsubClient.close(); + pubsubClient = null; + super.finishBundle(c); + } + } + + + // ================================================================================ + // PubsubUnboundedSink + // ================================================================================ + + /** + * Number of cores available for publishing. + */ + private final int numCores; + + /** + * Pub/sub topic to publish to. + */ + private final String topic; + + /** + * Coder for elements. Elements are effectively double-encoded: first to a byte array + * using this checkpointCoder, then to a base-64 string to conform to pub/sub's payload + * conventions. + */ + private final Coder elementCoder; + + /** + * Pub/sub metadata field holding timestamp of each element, or {@literal null} if should use + * pub/sub message publish timestamp instead. + */ + @Nullable + private final String timestampLabel; + + /** + * Pub/sub metadata field holding id for each element, or {@literal null} if need to generate + * a unique id ourselves. + * CAUTION: Currently ignored. + */ + @Nullable + private final String idLabel; + + public PubsubUnboundedSink( + int numCores, + String topic, + Coder elementCoder, + String timestampLabel, + String idLabel) { + this.numCores = numCores; + this.topic = topic; + this.elementCoder = elementCoder; + this.timestampLabel = timestampLabel; + this.idLabel = idLabel; + } + + @Override + public PDone apply(PCollection input) { + String label = "PubsubSink(" + topic.replaceFirst(".*/", "") + ")"; + input.apply(Window.into(new GlobalWindows()) + .triggering( + Repeatedly.forever( + AfterFirst.of(AfterPane.elementCountAtLeast(PUBLISH_BATCH_SIZE), + AfterProcessingTime.pastFirstElementInPane() + .plusDelayOf(MAX_LATENCY)))) + .discardingFiredPanes() + .withAllowedLateness(Duration.ZERO)) + .apply(ParDo.named(label + ".Shard").of(new ShardFn(numCores))) + .setCoder(KvCoder.of(VarIntCoder.of(), CODER)) + .apply(GroupByKey.create()) + .apply(ParDo.named(label + ".Writer").of(new WriterFn())); + return PDone.in(input.getPipeline()); + } +} From ca67017333e08278aa7669902205581e3a80939d Mon Sep 17 00:00:00 2001 From: Mark Shields Date: Mon, 4 Apr 2016 18:10:02 -0700 Subject: [PATCH 03/14] Pub/sub source --- .../sdk/io/PubsubUnboundedSource.java | 1026 +++++++++++++++++ .../dataflow/sdk/util/BucketingFunction.java | 152 +++ .../dataflow/sdk/util/MovingFunction.java | 152 +++ .../sdk/util/BucketingFunctionTest.java | 104 ++ .../dataflow/sdk/util/MovingFunctionTest.java | 115 ++ 5 files changed, 1549 insertions(+) create mode 100644 sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/io/PubsubUnboundedSource.java create mode 100644 sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/util/BucketingFunction.java create mode 100644 sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/util/MovingFunction.java create mode 100644 sdks/java/core/src/test/java/com/google/cloud/dataflow/sdk/util/BucketingFunctionTest.java create mode 100644 sdks/java/core/src/test/java/com/google/cloud/dataflow/sdk/util/MovingFunctionTest.java diff --git a/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/io/PubsubUnboundedSource.java b/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/io/PubsubUnboundedSource.java new file mode 100644 index 000000000000..52fec011b7ab --- /dev/null +++ b/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/io/PubsubUnboundedSource.java @@ -0,0 +1,1026 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package com.google.cloud.dataflow.sdk.io; + +import com.google.api.client.util.Preconditions; +import com.google.cloud.dataflow.sdk.coders.AtomicCoder; +import com.google.cloud.dataflow.sdk.coders.Coder; +import com.google.cloud.dataflow.sdk.coders.CoderException; +import com.google.cloud.dataflow.sdk.coders.ListCoder; +import com.google.cloud.dataflow.sdk.coders.StringUtf8Coder; +import com.google.cloud.dataflow.sdk.options.GcpOptions; +import com.google.cloud.dataflow.sdk.options.PipelineOptions; +import com.google.cloud.dataflow.sdk.transforms.Aggregator; +import com.google.cloud.dataflow.sdk.transforms.Combine; +import com.google.cloud.dataflow.sdk.transforms.DoFn; +import com.google.cloud.dataflow.sdk.transforms.PTransform; +import com.google.cloud.dataflow.sdk.transforms.ParDo; +import com.google.cloud.dataflow.sdk.transforms.Sum; +import com.google.cloud.dataflow.sdk.transforms.windowing.BoundedWindow; +import com.google.cloud.dataflow.sdk.util.BucketingFunction; +import com.google.cloud.dataflow.sdk.util.CoderUtils; +import com.google.cloud.dataflow.sdk.util.MovingFunction; +import com.google.cloud.dataflow.sdk.values.PBegin; +import com.google.cloud.dataflow.sdk.values.PCollection; +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.security.GeneralSecurityException; +import java.util.ArrayDeque; +import java.util.ArrayList; +import java.util.Collection; +import java.util.LinkedHashMap; +import java.util.List; +import java.util.Map; +import java.util.NoSuchElementException; +import java.util.Queue; +import java.util.concurrent.ConcurrentLinkedQueue; +import java.util.concurrent.atomic.AtomicInteger; + +import javax.annotation.Nullable; + +/** + * A PTransform which streams messages from pub/sub. + *
    + *
  • The underlying implementation in an {@link UnboundedSource} which receives messages + * in batches and hands them out one at a time. + *
  • We use gRPC for its speed and low memory overhead. + *
  • The watermark (either in pub/sub processing time or custom timestamp time) is estimated + * by keeping track of the minimum of the last minutes worth of messages. This assumes pub/sub + * delivers the oldest (in pub/sub processing time) available message at least once a minute, + * and that custom timestamps are 'mostly' monotonic with pub/sub processing time. Unfortunately + * both of those assumptions are currently false. Thus the estimated watermark may get ahead of + * the 'true' watermark and cause some messages to be late. + *
  • Checkpoints are used both to ACK received messages back to pub/sub (so that they may + * be retired on the pub/sub end), and to NACK already consumed messages should a checkpoint + * need to be restored (so that pub/sub will resend those messages promptly). + *
  • The subscription must already exist. + *
  • The subscription should have an ACK timeout of 60 seconds. + *
  • We log vital stats every 30 seconds. + *
  • Though some background threads are used by the underlying netty system all actual + * pub/sub calls are blocking. We rely on the underlying runner to allow multiple + * {@link UnboundedSource.UnboundedReader} instance to execute concurrently and thus hide latency. + *
+ */ +public class PubsubUnboundedSource extends PTransform> { + private static final Logger LOG = LoggerFactory.getLogger(PubsubUnboundedSource.class); + + /** + * Coder for checkpoints. + */ + private static final CheckpointCoder CHECKPOINT_CODER = new CheckpointCoder<>(); + + /** + * Maximum number of messages per pull. + */ + private static final int PULL_BATCH_SIZE = 1000; + + /** + * Maximum number of ack ids per ack or ack extension. + */ + private static final int ACK_BATCH_SIZE = 2000; + + /** + * Maximum number of messages in flight. + */ + private static final int MAX_IN_FLIGHT = 20000; + + /** + * Ack timeout for initial get. + */ + private static final Duration ACK_TIMEOUT = Duration.standardSeconds(60); + + /** + * Duration by which to extend acks when they are near timeout. + */ + private static final Duration ACK_EXTENSION = Duration.standardSeconds(30); + + /* + * How close we can get to a deadline before we need to extend it. + */ + private static final Duration ACK_SAFETY = Duration.standardSeconds(15); + + /** + * How close we can get to a deadline before we need to consider it passed. + */ + private static final Duration ACK_TOO_LATE = Duration.standardSeconds(5); + + /** + * Period of samples to determine watermark and other stats. + */ + private static final Duration SAMPLE_PERIOD = Duration.standardMinutes(1); + + /** + * Period of updates to determine watermark and other stats. + */ + private static final Duration SAMPLE_UPDATE = Duration.standardSeconds(5); + + /** + * How frequently to log stats. + */ + private static final Duration LOG_PERIOD = Duration.standardSeconds(30); + + /** + * Minimum number of unread messages required before considering updating watermark. + */ + private static final int MIN_WATERMARK_MESSAGES = 10; + + /** + * Minimum number of SAMPLE_UPDATE periods over which unread messages shoud be spread + * before considering updating watermark. + */ + private static final int MIN_WATERMARK_SPREAD = 2; + + /** + * Additional sharding so that we can hide read message latency. + */ + private static final int SCALE_OUT = 4; + + private static final Combine.BinaryCombineLongFn MIN = + new Combine.BinaryCombineLongFn() { + @Override + public long apply(long left, long right) { + return Math.min(left, right); + } + + @Override + public long identity() { + return Long.MAX_VALUE; + } + }; + + private static final Combine.BinaryCombineLongFn MAX = + new Combine.BinaryCombineLongFn() { + @Override + public long apply(long left, long right) { + return Math.max(left, right); + } + + @Override + public long identity() { + return Long.MIN_VALUE; + } + }; + + private static final Combine.BinaryCombineLongFn SUM = + new Combine.BinaryCombineLongFn() { + @Override + public long apply(long left, long right) { + return left + right; + } + + @Override + public long identity() { + return 0; + } + }; + + + // ================================================================================ + // Checkpoint + // ================================================================================ + + /** + * Which messages have been durably committed and thus can now be acked. + * Which messages have been read but not yet committed, in which case they should be nacked if + * we need to restore. + */ + public static class Checkpoint implements UnboundedSource.CheckpointMark { + /** + * If the checkpoint is for persisting: the reader who's snapshotted state we are persisting. + * If the checkpoint is for restoring: initially {@literal null}, then explicitly set. + * Not persisted in durable checkpoint. + * CAUTION: Between a checkpoint being taken and {@link #finalizeCheckpoint()} being called + * the 'true' active reader may have changed. + */ + @Nullable + private Reader reader; + + /** + * If the checkpoint is for persisting: The ack ids of messages which have been passed + * downstream since the last checkpoint. + * If the checkpoint is for restoring: {@literal null}. + * Not persisted in durable checkpoint. + */ + @Nullable + private final List safeToAckIds; + + /** + * If the checkpoint is for persisting: The ack ids of messages which have been received + * from pub/sub but not yet passed downstream at the time of the snapshot. + * If the checkpoint is for restoring: Same, but recovered from durable storage. + */ + private final List notYetReadIds; + + public Checkpoint( + @Nullable Reader reader, @Nullable List safeToAckIds, + List notYetReadIds) { + this.reader = reader; + this.safeToAckIds = safeToAckIds; + this.notYetReadIds = notYetReadIds; + } + + public void setReader(Reader reader) { + Preconditions.checkState(this.reader == null, "Cannot setReader on persisting checkpoint"); + this.reader = reader; + } + + /** + * BLOCKING + * All messages which have been passed downstream have now been durably committed. + * We can ack them upstream. + */ + @Override + public void finalizeCheckpoint() throws IOException { + Preconditions.checkState(reader != null && safeToAckIds != null, + "Cannot finalize a restored checkpoint"); + // Even if the 'true' active reader has changed since the checkpoint was taken we are + // fine: + // - The underlying pub/sub topic will not have changed, so the following acks will still + // go to the right place. + // - We'll delete the ack ids from the readers in-flight state, but that only effects + // flow control and stats, neither of which are relevent anymore. + try { + int n = safeToAckIds.size(); + List batchSafeToAckIds = new ArrayList<>(Math.min(n, ACK_BATCH_SIZE)); + for (String ackId : safeToAckIds) { + batchSafeToAckIds.add(ackId); + if (batchSafeToAckIds.size() >= ACK_BATCH_SIZE) { + reader.ackBatch(batchSafeToAckIds); + n -= batchSafeToAckIds.size(); + // CAUTION: Don't reuse the same list since ackBatch holds on to it. + batchSafeToAckIds = new ArrayList<>(Math.min(n, ACK_BATCH_SIZE)); + } + } + if (!batchSafeToAckIds.isEmpty()) { + reader.ackBatch(batchSafeToAckIds); + } + } finally { + Preconditions.checkState(reader.numInFlightCheckpoints.decrementAndGet() >= 0); + } + } + + /** + * BLOCKING + * Nack all messages which have been read from pub/sub but not passed downstream. + * This way pub/sub will send them again promptly. + */ + public void nackAll() throws IOException { + Preconditions.checkState(reader != null, "Reader was not set"); + List batchYetToAckIds = + new ArrayList<>(Math.min(notYetReadIds.size(), ACK_BATCH_SIZE)); + for (String ackId : notYetReadIds) { + batchYetToAckIds.add(ackId); + if (batchYetToAckIds.size() >= ACK_BATCH_SIZE) { + long nowMsSinceEpoch = System.currentTimeMillis(); + reader.nackBatch(nowMsSinceEpoch, batchYetToAckIds); + batchYetToAckIds.clear(); + } + } + if (!batchYetToAckIds.isEmpty()) { + long nowMsSinceEpoch = System.currentTimeMillis(); + reader.nackBatch(nowMsSinceEpoch, batchYetToAckIds); + } + } + } + + /** + * The coder for our checkpoints. + */ + private static class CheckpointCoder extends AtomicCoder> { + private static final Coder> LIST_CODER = ListCoder.of(StringUtf8Coder.of()); + + @Override + public void encode(Checkpoint value, OutputStream outStream, Context context) + throws IOException { + LIST_CODER.encode(value.notYetReadIds, outStream, context); + } + + @Override + public Checkpoint decode(InputStream inStream, Context context) throws IOException { + List notYetReadIds = LIST_CODER.decode(inStream, context); + return new Checkpoint<>(null, null, notYetReadIds); + } + } + + // ================================================================================ + // Reader + // ================================================================================ + + /** + * A reader which keeps track of which messages have been received from pub/sub + * but not yet consumed downstream and/or acked back to pub/sub. + */ + private static class Reader extends UnboundedSource.UnboundedReader { + /** + * For access to topic and checkpointCoder. + */ + private final Source outer; + + /** + * Client on which to talk to pub/sub. Null if closed. + */ + @Nullable + private PubsubGrpcClient pubsubClient; + + /** + * Ack ids of messages we have delivered downstream but not yet acked. + */ + private List safeToAckIds; + + /** + * Messages we have received from pub/sub and not yet delivered downstream. + * We preserve their order. + */ + private final Queue notYetRead; + + /** + * Map from ack ids of messages we have received from pub/sub but not yet acked to their + * current deadline. Ordered from earliest to latest deadline. + */ + private final LinkedHashMap inFlight; + + /** + * Batches of successfully acked ids which need to be pruned from the above. + * CAUTION: Accessed by both reader and checkpointing threads. + */ + private final Queue> ackedIds; + + /** + * Byte size of undecoded elements in {@link #notYetRead}. + */ + private long notYetReadBytes; + + /** + * Bucketed map from received time (as system time, ms since epoch) to message + * timestamps (mssince epoch) of all received but not-yet read messages. + * Used to estimate watermark. + */ + private BucketingFunction minUnreadTimestampMsSinceEpoch; + + /** + * Minimum of timestamps (ms since epoch) of all recently read messages. + * Used to estimate watermark. + */ + private MovingFunction minReadTimestampMsSinceEpoch; + + /** + * System time (ms since epoch) we last received a message from pub/sub, or -1 if + * not yet received any messages. + */ + private long lastReceivedMsSinceEpoch; + + /** + * The last reported watermark (ms since epoch), or beginning of time if none yet reported. + */ + private long lastWatermarkMsSinceEpoch; + + /** + * The current message, or {@literal null} if none. + */ + @Nullable + private PubsubClient.IncomingMessage current; + + /** + * Stats only: System time (ms since epoch) we last logs stats, or -1 if never. + */ + private long lastLogTimestampMsSinceEpoch; + + /** + * Stats only: Total number of messages received. + */ + private long numReceived; + + /** + * Stats only: Number of messages which have recently been received. + */ + private MovingFunction numReceivedRecently; + + /** + * Stats only: Number of messages which have recently had their deadline extended. + */ + private MovingFunction numExtendedDeadlines; + + /** + * Stats only: Number of messages which have recenttly had their deadline extended even + * though it may be too late to do so. + */ + private MovingFunction numLateDeadlines; + + + /** + * Stats only: Number of messages which have recently been acked. + */ + private MovingFunction numAcked; + + /** + * Stats only: Number of messages which have recently been nacked. + */ + private MovingFunction numNacked; + + /** + * Stats only: Number of message bytes which have recently been read by downstream consumer. + */ + private MovingFunction numReadBytes; + + /** + * Stats only: Minimum of timestamp (ms since epoch) of all recently received messages. + * Used to estimate timestamp skew. Does not contribute to watermark estimator. + */ + private MovingFunction minReceivedTimestampMsSinceEpoch; + + /** + * Stats only: Maximum of timestamp (ms since epoch) of all recently received messages. + * Used to estimate timestamp skew. + */ + private MovingFunction maxReceivedTimestampMsSinceEpoch; + + /** + * Stats only: Minimum of recent estimated watermarks (ms since epoch). + */ + private MovingFunction minWatermarkMsSinceEpoch; + + /** + * Stats ony: Maximum of recent estimated watermarks (ms since epoch). + */ + private MovingFunction maxWatermarkMsSinceEpoch; + + /** + * Stats only: Number of messages with timestamps strictly behind the estimated watermark + * at the time they are received. These may be considered 'late' by downstream computations. + */ + private MovingFunction numLateMessages; + + /** + * Stats only: Current number of checkpoints in flight. + * CAUTION: Accessed by both checkpointing and reader threads. + */ + private AtomicInteger numInFlightCheckpoints; + + /** + * Stats only: Maximum number of checkpoints in flight at any time. + */ + private int maxInFlightCheckpoints; + + private static MovingFunction newFun(Combine.BinaryCombineLongFn function) { + return new MovingFunction(SAMPLE_PERIOD.getMillis(), + SAMPLE_UPDATE.getMillis(), + MIN_WATERMARK_SPREAD, + MIN_WATERMARK_MESSAGES, + function); + } + + /** + * Construct a reader. + */ + public Reader(GcpOptions options, Source outer) throws IOException, + GeneralSecurityException { + this.outer = outer; + pubsubClient = PubsubGrpcClient.newClient(outer.outer.timestampLabel, + outer.outer.idLabel, + options); + safeToAckIds = new ArrayList<>(); + notYetRead = new ArrayDeque<>(); + inFlight = new LinkedHashMap<>(); + ackedIds = new ConcurrentLinkedQueue<>(); + notYetReadBytes = 0; + minUnreadTimestampMsSinceEpoch = new BucketingFunction(SAMPLE_UPDATE.getMillis(), + MIN_WATERMARK_SPREAD, + MIN_WATERMARK_MESSAGES, + MIN); + minReadTimestampMsSinceEpoch = newFun(MIN); + lastReceivedMsSinceEpoch = -1; + lastWatermarkMsSinceEpoch = BoundedWindow.TIMESTAMP_MIN_VALUE.getMillis(); + current = null; + lastLogTimestampMsSinceEpoch = -1; + numReceived = 0L; + numReceivedRecently = newFun(SUM); + numExtendedDeadlines = newFun(SUM); + numLateDeadlines = newFun(SUM); + numAcked = newFun(SUM); + numNacked = newFun(SUM); + numReadBytes = newFun(SUM); + minReceivedTimestampMsSinceEpoch = newFun(MIN); + maxReceivedTimestampMsSinceEpoch = newFun(MAX); + minWatermarkMsSinceEpoch = newFun(MIN); + maxWatermarkMsSinceEpoch = newFun(MAX); + numLateMessages = newFun(SUM); + numInFlightCheckpoints = new AtomicInteger(); + maxInFlightCheckpoints = 0; + } + + /** + * BLOCKING + * Ack receipt of messages from pub/sub with the given {@code ackIds}. + * CAUTION: May be invoked from a separate checkpointing thread. + * CAUTION: Retains {@code ackIds}. + */ + public void ackBatch(List ackIds) throws IOException { + pubsubClient.acknowledge(outer.outer.subscription, ackIds); + ackedIds.add(ackIds); + } + + /** + * BLOCKING + * 'Nack' (ie request deadline extension of 0) receipt of messages from pub/sub + * with the given {@code ockIds}. Does not retain {@code ackIds}. + */ + public void nackBatch(long nowMsSinceEpoch, List ackIds) throws IOException { + pubsubClient.modifyAckDeadline(outer.outer.subscription, ackIds, 0); + numNacked.add(nowMsSinceEpoch, ackIds.size()); + } + + /** + * BLOCKING + * Extend the processing deadline for messages from pub/sub with the given {@code ackIds}. + * Does not retain {@code ackIds}. + */ + private void extendBatch(long nowMsSinceEpoch, List ackIds) throws IOException { + pubsubClient.modifyAckDeadline(outer.outer.subscription, ackIds, + (int) ACK_EXTENSION.getStandardSeconds()); + numExtendedDeadlines.add(nowMsSinceEpoch, ackIds.size()); + } + + /** + * Messages which have been acked (via the checkpoint finalize) are no longer in flight. + * This is only used for flow control and stats. + */ + private void retire() { + long nowMsSinceEpoch = System.currentTimeMillis(); + while (true) { + List ackIds = ackedIds.poll(); + if (ackIds == null) { + return; + } + numAcked.add(nowMsSinceEpoch, ackIds.size()); + for (String ackId : ackIds) { + inFlight.remove(ackId); + } + } + } + + /** + * BLOCKING + * Extend deadline for all messages which need it. + *

+ * CAUTION: If extensions can't keep up with wallclock then we'll never return. + */ + private void extend() throws IOException { + while (true) { + long nowMsSinceEpoch = System.currentTimeMillis(); + List> toBeExtended = new ArrayList<>(); + for (Map.Entry entry : inFlight.entrySet()) { + long deadlineMsSinceEpoch = entry.getValue().getMillis(); + if (deadlineMsSinceEpoch < nowMsSinceEpoch + ACK_TOO_LATE.getMillis()) { + // This message may have already expired, in which case it will (eventually) be + // made available on a future pull request. + // If the message ends up being commited then a future resend will be ignored + // downsteam and acked as usual. + numLateDeadlines.add(nowMsSinceEpoch, 1); + } + if (deadlineMsSinceEpoch > nowMsSinceEpoch + ACK_SAFETY.getMillis()) { + // No later messages need extending. + break; + } + toBeExtended.add(entry); + if (toBeExtended.size() >= ACK_BATCH_SIZE) { + // Enough for one batch. + break; + } + } + if (toBeExtended.isEmpty()) { + // No messages need to be extended. + return; + } + List extensionAckIds = new ArrayList<>(toBeExtended.size()); + long newDeadlineMsSinceEpoch = nowMsSinceEpoch + ACK_EXTENSION.getMillis(); + for (Map.Entry entry : toBeExtended) { + inFlight.remove(entry.getKey()); + inFlight.put(entry.getKey(), new Instant(newDeadlineMsSinceEpoch)); + extensionAckIds.add(entry.getKey()); + } + // BLOCKs until extended. + extendBatch(nowMsSinceEpoch, extensionAckIds); + } + } + + /** + * BLOCKING + * Fetch another batch of messages from pub/sub. + */ + private void pull() throws IOException { + if (inFlight.size() >= MAX_IN_FLIGHT) { + // Wait for checkpoint to be finalized before pulling anymore. + // There may be lag while checkpoints are persisted and the finalizeCheckpoint method + // is invoked. By limiting the in-flight messages we can ensure we don't end up consuming + // messages faster than we can checkpoint them. + return; + } + + long requestTimeMsSinceEpoch = System.currentTimeMillis(); + long deadlineMsSinceEpoch = requestTimeMsSinceEpoch + ACK_TIMEOUT.getMillis(); + + // Pull the next batch. + // BLOCKs until received. + Collection receivedMessages = + pubsubClient.pull(requestTimeMsSinceEpoch, + outer.outer.subscription, + PULL_BATCH_SIZE); + if (receivedMessages.isEmpty()) { + // Nothing available yet. Try again later. + return; + } + + lastReceivedMsSinceEpoch = requestTimeMsSinceEpoch; + + // Capture the received messages. + for (PubsubClient.IncomingMessage incomingMessage : receivedMessages) { + notYetRead.add(incomingMessage); + notYetReadBytes += incomingMessage.elementBytes.length; + inFlight.put(incomingMessage.ackId, new Instant(deadlineMsSinceEpoch)); + numReceived++; + numReceivedRecently.add(requestTimeMsSinceEpoch, 1L); + minReceivedTimestampMsSinceEpoch.add(requestTimeMsSinceEpoch, + incomingMessage.timestampMsSinceEpoch); + maxReceivedTimestampMsSinceEpoch.add(requestTimeMsSinceEpoch, + incomingMessage.timestampMsSinceEpoch); + minUnreadTimestampMsSinceEpoch.add(requestTimeMsSinceEpoch, + incomingMessage.timestampMsSinceEpoch); + } + } + + /** + * Log stats if time to do so. + */ + private void stats() { + long nowMsSinceEpoch = System.currentTimeMillis(); + if (lastLogTimestampMsSinceEpoch < 0) { + lastLogTimestampMsSinceEpoch = nowMsSinceEpoch; + return; + } + long deltaMs = nowMsSinceEpoch - lastLogTimestampMsSinceEpoch; + if (deltaMs < LOG_PERIOD.getMillis()) { + return; + } + + String messageSkew = "unknown"; + long minTimestamp = minReceivedTimestampMsSinceEpoch.get(nowMsSinceEpoch); + long maxTimestamp = maxReceivedTimestampMsSinceEpoch.get(nowMsSinceEpoch); + if (minTimestamp < Long.MAX_VALUE && maxTimestamp > Long.MIN_VALUE) { + messageSkew = (maxTimestamp - minTimestamp) + "ms"; + } + + String watermarkSkew = "unknown"; + long minWatermark = minWatermarkMsSinceEpoch.get(nowMsSinceEpoch); + long maxWatermark = maxWatermarkMsSinceEpoch.get(nowMsSinceEpoch); + if (minWatermark < Long.MAX_VALUE && maxWatermark > Long.MIN_VALUE) { + watermarkSkew = (maxWatermark - minWatermark) + "ms"; + } + + LOG.warn("Pubsub {} has " + + "{} received messages, " + + "{} current unread messages, " + + "{} current unread bytes, " + + "{} current in-flight msgs, " + + "{} current in-flight checkpoints, " + + "{} max in-flight checkpoints, " + + "{}B/s recent read, " + + "{} recent received, " + + "{} recent extended, " + + "{} recent late extended, " + + "{} recent acked, " + + "{} recent nacked, " + + "{} recent message timestamp skew, " + + "{} recent watermark skew, " + + "{} recent late messages, " + + "{} last reported watermark", + outer.outer.subscription, + numReceived, + notYetRead.size(), + notYetReadBytes, + inFlight.size(), + numInFlightCheckpoints.get(), + maxInFlightCheckpoints, + numReadBytes.get(nowMsSinceEpoch) / (SAMPLE_PERIOD.getMillis() / 1000L), + numReceivedRecently.get(nowMsSinceEpoch), + numExtendedDeadlines.get(nowMsSinceEpoch), + numLateDeadlines.get(nowMsSinceEpoch), + numAcked.get(nowMsSinceEpoch), + numNacked.get(nowMsSinceEpoch), + messageSkew, + watermarkSkew, + numLateMessages.get(nowMsSinceEpoch), + new Instant(lastWatermarkMsSinceEpoch)); + + lastLogTimestampMsSinceEpoch = nowMsSinceEpoch; + } + + @Override + public boolean start() throws IOException { + return advance(); + } + + /** + * BLOCKING + * Return {@literal true} if a pub/sub messaage is available, {@literal false} if + * none is available at this time or we are over-subscribed. May BLOCK while extending + * acks or fetching available messages. Will not block waiting for messages. + */ + @Override + public boolean advance() throws IOException { + // Emit stats. + stats(); + + if (current != null) { + // Current is consumed. It can no longer contribute to holding back the watermark. + minUnreadTimestampMsSinceEpoch.remove(current.requestTimeMsSinceEpoch); + current = null; + } + + // Retire state associated with acked messages. + retire(); + + // Extend all pressing deadlines. + // Will BLOCK until done. + // If the system is pulling messages only to let them sit in a downsteam queue then + // this will have the effect of slowing down the pull rate. + // However, if the system is genuinely taking longer to process each message then + // the work to extend acks would be better done in the background. + extend(); + + if (notYetRead.isEmpty()) { + // Pull another batch. + // Will BLOCK until fetch returns, but will not block until a message is available. + pull(); + } + + // Take one message from queue. + current = notYetRead.poll(); + if (current == null) { + // Try again later. + return false; + } + notYetReadBytes -= current.elementBytes.length; + Preconditions.checkState(notYetReadBytes >= 0); + long nowMsSinceEpoch = System.currentTimeMillis(); + numReadBytes.add(nowMsSinceEpoch, current.elementBytes.length); + minReadTimestampMsSinceEpoch.add(nowMsSinceEpoch, current.timestampMsSinceEpoch); + if (current.timestampMsSinceEpoch < lastWatermarkMsSinceEpoch) { + numLateMessages.add(nowMsSinceEpoch, 1L); + } + + // Current message will be persisted by the next checkpoint so it is now safe to ack. + safeToAckIds.add(current.ackId); + return true; + } + + @Override + public T getCurrent() throws NoSuchElementException { + if (current == null) { + throw new NoSuchElementException(); + } + try { + return CoderUtils.decodeFromByteArray(outer.outer.elementCoder, current.elementBytes); + } catch (CoderException e) { + throw new RuntimeException("Unable to decode element from pub/sub message: ", e); + } + } + + @Override + public Instant getCurrentTimestamp() throws NoSuchElementException { + if (current == null) { + throw new NoSuchElementException(); + } + return new Instant(current.timestampMsSinceEpoch); + } + + @Override + public byte[] getCurrentRecordId() throws NoSuchElementException { + if (current == null) { + throw new NoSuchElementException(); + } + return current.recordId; + } + + @Override + public void close() throws IOException { + pubsubClient.close(); + pubsubClient = null; + } + + @Override + public Source getCurrentSource() { + return outer; + } + + @Override + public Instant getWatermark() { + // NOTE: We'll allow the watermark to go backwards. The underlying runner is responsible + // for aggregating all reported watermarks and ensuring the aggregate is latched. + // If we attempt to latch locally then it is possible a temporary starvation of one reader + // could cause its estimated watermark to fast forward to current system time. Then when + // the reader resumes its watermark would be unable to resume tracking. + // By letting the underlying runner latch we avoid any problems due to localized starvation. + long nowMsSinceEpoch = System.currentTimeMillis(); + long readMin = minReadTimestampMsSinceEpoch.get(nowMsSinceEpoch); + long unreadMin = minUnreadTimestampMsSinceEpoch.get(); + if (readMin == Long.MAX_VALUE && + unreadMin == Long.MAX_VALUE && + lastReceivedMsSinceEpoch >= 0 && + nowMsSinceEpoch > lastReceivedMsSinceEpoch + SAMPLE_PERIOD.getMillis()) { + // We don't currently have any unread messages pending, we have not had any messages + // read for a while, and we have not received any new messages from pub/sub for a while. + // Advance watermark to current time. + // TODO: Estimate a timestamp lag. + lastWatermarkMsSinceEpoch = nowMsSinceEpoch; + } else if (minReadTimestampMsSinceEpoch.isSignificant() || + minUnreadTimestampMsSinceEpoch.isSignificant()) { + // Take minimum of the timestamps in all unread messages and recently read messages. + lastWatermarkMsSinceEpoch = Math.min(readMin, unreadMin); + } + // else: We're not confident enough to estimate a new watermark. Stick with the old one. + minWatermarkMsSinceEpoch.add(nowMsSinceEpoch, lastWatermarkMsSinceEpoch); + maxWatermarkMsSinceEpoch.add(nowMsSinceEpoch, lastWatermarkMsSinceEpoch); + return new Instant(lastWatermarkMsSinceEpoch); + } + + @Override + public UnboundedSource.CheckpointMark getCheckpointMark() { + int cur = numInFlightCheckpoints.incrementAndGet(); + maxInFlightCheckpoints = Math.max(maxInFlightCheckpoints, cur); + // The checkpoint will either be finalized or we'll rollback to an earlier + // checkpoint. Thus we can hand off these ack ids to the checkpoint. + List snapshotSafeToAckIds = safeToAckIds; + safeToAckIds = new ArrayList<>(); + List snapshotNotYetReadIds = new ArrayList<>(notYetRead.size()); + for (PubsubClient.IncomingMessage incomingMessage : notYetRead) { + snapshotNotYetReadIds.add(incomingMessage.ackId); + } + return new Checkpoint<>(this, snapshotSafeToAckIds, snapshotNotYetReadIds); + } + + @Override + public long getSplitBacklogBytes() { + return notYetReadBytes; + } + } + + // ================================================================================ + // Source + // ================================================================================ + + private static class Source extends UnboundedSource> { + PubsubUnboundedSource outer; + + public Source(PubsubUnboundedSource outer) { + this.outer = outer; + } + + @Override + public List> generateInitialSplits( + int desiredNumSplits, PipelineOptions options) throws Exception { + List> result = new ArrayList<>(desiredNumSplits); + for (int i = 0; i < desiredNumSplits * SCALE_OUT; i++) { + // Since the source is immutable and pub/sub automatically shards we simply + // replicate ourselves the requested number of times + result.add(this); + } + return result; + } + + @Override + public UnboundedReader createReader( + PipelineOptions options, @Nullable Checkpoint checkpoint) { + PubsubUnboundedSource.Reader reader; + try { + reader = + new PubsubUnboundedSource.Reader<>(options.as(GcpOptions.class), this); + } catch (GeneralSecurityException | IOException e) { + throw new RuntimeException("Unable to subscribe to " + outer.subscription + ": ", e); + } + if (checkpoint != null) { + // Nack all messages we may have lost. + try { + checkpoint.setReader(reader); + // Will BLOCK until nacked. + checkpoint.nackAll(); + } catch (IOException e) { + LOG.error("Pubsub {} cannot have {} lost messages nacked, ignoring: {}", + outer.subscription, checkpoint.notYetReadIds.size(), e); + } + } + return reader; + } + + @Nullable + @Override + public Coder> getCheckpointMarkCoder() { + @SuppressWarnings("unchecked") CheckpointCoder typedCoder = + (CheckpointCoder) CHECKPOINT_CODER; + return typedCoder; + } + + @Override + public Coder getDefaultOutputCoder() { + return outer.elementCoder; + } + + @Override + public void validate() { + // Nothing to validate. + } + + @Override + public boolean requiresDeduping() { + // We cannot prevent re-offering already read messages after a restore from checkpoint. + return true; + } + } + + // ================================================================================ + // StatsFn + // ================================================================================ + + private static class StatsFn extends DoFn { + private final Aggregator elementCounter = + createAggregator("elements", new Sum.SumLongFn()); + + + @Override + public void processElement(ProcessContext c) throws Exception { + elementCounter.addValue(1L); + c.output(c.element()); + } + } + + // ================================================================================ + // PubsubUnboundedSource + // ================================================================================ + + /** + * Subscription to read from. + */ + private final String subscription; + + /** + * Coder for elements. Elements are effectively double-encoded: first to a byte array + * using this checkpointCoder, then to a base-64 string to conform to pub/sub's payload + * conventions. + */ + private final Coder elementCoder; + + /** + * Pub/sub metadata field holding timestamp of each element, or {@literal null} if should use + * pub/sub message publish timestamp instead. + */ + @Nullable + private final String timestampLabel; + + /** + * Pub/sub metadata field holding id for each element, or {@literal null} if need to generate + * a unique id ourselves. + */ + @Nullable + private final String idLabel; + + /** + * Construct an unbounded source to consume from the pub/sub {@code subscription}. + */ + public PubsubUnboundedSource( + String subscription, Coder elementCoder, @Nullable String timestampLabel, + @Nullable String idLabel) { + this.subscription = Preconditions.checkNotNull(subscription); + this.elementCoder = Preconditions.checkNotNull(elementCoder); + this.timestampLabel = timestampLabel; + this.idLabel = idLabel; + } + + @Override + public PCollection apply(PBegin input) { + String label = "PubsubSource(" + subscription.replaceFirst(".*/", "") + ")"; + return input.getPipeline().begin() + .apply(Read.from(new Source(this))) + .apply(ParDo.named(label).of(new StatsFn())); + } +} diff --git a/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/util/BucketingFunction.java b/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/util/BucketingFunction.java new file mode 100644 index 000000000000..56bec8db6fdb --- /dev/null +++ b/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/util/BucketingFunction.java @@ -0,0 +1,152 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package com.google.cloud.dataflow.sdk.util; + +import com.google.cloud.dataflow.sdk.transforms.Combine; +import com.google.common.base.Preconditions; +import java.util.HashMap; +import java.util.Map; + +/** + * Keep track of the minimum/maximum/sum of a set of timestamped long values. + * For efficiency, bucket values by their timestamp. + */ +public class BucketingFunction { + private static class Bucket { + private int numSamples; + private long combinedValue; + + public Bucket(BucketingFunction outer) { + numSamples = 0; + combinedValue = outer.function.identity(); + } + + public void add(BucketingFunction outer, long value) { + combinedValue = outer.function.apply(combinedValue, value); + numSamples++; + } + + public boolean remove() { + numSamples--; + Preconditions.checkState(numSamples >= 0); + return numSamples == 0; + } + + public long get() { + return combinedValue; + } + } + + /** + * How large a time interval to fit within each bucket. + */ + private final long bucketWidthMs; + + /** + * How many buckets are considered 'significant'? + */ + private final int numSignificantBuckets; + + /** + * How many samples are considered 'significant'? + */ + private final int numSignificantSamples; + + /** + * Function for combining sample values. + */ + private final Combine.BinaryCombineLongFn function; + + /** + * Active buckets. + */ + private final Map buckets; + + public BucketingFunction( + long bucketWidthMs, + int numSignificantBuckets, + int numSignificantSamples, + Combine.BinaryCombineLongFn function) { + this.bucketWidthMs = bucketWidthMs; + this.numSignificantBuckets = numSignificantBuckets; + this.numSignificantSamples = numSignificantSamples; + this.function = function; + this.buckets = new HashMap<>(); + } + + /** + * Which bucket key corresponds to {@code timeMsSinceEpoch}. + */ + private long key(long timeMsSinceEpoch) { + return timeMsSinceEpoch - (timeMsSinceEpoch % bucketWidthMs); + } + + /** + * Add one sample of {@code value} (to bucket) at {@code timeMsSinceEpoch}. + */ + public void add(long timeMsSinceEpoch, long value) { + long key = key(timeMsSinceEpoch); + Bucket bucket = buckets.get(key); + if (bucket == null) { + bucket = new Bucket(this); + buckets.put(key, bucket); + } + bucket.add(this, value); + } + + /** + * Remove one sample (from bucket) at {@code timeMsSinceEpoch}. + */ + public void remove(long timeMsSinceEpoch) { + long key = key(timeMsSinceEpoch); + Bucket bucket = buckets.get(key); + if (bucket == null) { + return; + } + if (bucket.remove()) { + buckets.remove(key); + } + } + + /** + * Return the (bucketized) combined value of all samples. + */ + public long get() { + long result = function.identity(); + for (Bucket bucket : buckets.values()) { + result = function.apply(result, bucket.get()); + } + return result; + } + + /** + * Is the current result 'significant'? Ie is it drawn from enough buckets + * or from enough samples? + */ + public boolean isSignificant() { + if (buckets.size() >= numSignificantBuckets) { + return true; + } + int totalSamples = 0; + for (Bucket bucket : buckets.values()) { + totalSamples += bucket.numSamples; + } + return totalSamples >= numSignificantSamples; + } +} diff --git a/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/util/MovingFunction.java b/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/util/MovingFunction.java new file mode 100644 index 000000000000..22827e62512a --- /dev/null +++ b/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/util/MovingFunction.java @@ -0,0 +1,152 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package com.google.cloud.dataflow.sdk.util; + +import com.google.cloud.dataflow.sdk.transforms.Combine; +import com.google.common.base.Preconditions; +import java.util.Arrays; + +/** + * Keep track of the moving minimum/maximum/sum of sampled long values. The minimum/maximum/sum + * is over at most the last {@link #samplePeriodMs}, and is updated every + * {@link #sampleUpdateMs}. + */ +public class MovingFunction { + /** + * How far back to retain samples, in ms. + */ + private final long samplePeriodMs; + + /** + * How frequently to update the moving function, in ms. + */ + private final long sampleUpdateMs; + + /** + * How many buckets are considered 'significant'? + */ + private final int numSignificantBuckets; + + /** + * How many samples are considered 'significant'? + */ + private final int numSignificantSamples; + + /** + * Function for combining sample values. + */ + private final Combine.BinaryCombineLongFn function; + + /** + * Minimum/maximum/sum of all values per bucket. + */ + private final long[] buckets; + + /** + * How many samples have been added to each bucket. + */ + private final int[] numSamples; + + /** + * Time of start of current bucket. + */ + private long currentMsSinceEpoch; + + /** + * Index of bucket corresponding to above timestamp, or -1 if no entries. + */ + private int currentIndex; + + public MovingFunction(long samplePeriodMs, long sampleUpdateMs, + int numSignificantBuckets, int numSignificantSamples, + Combine.BinaryCombineLongFn function) { + this.samplePeriodMs = samplePeriodMs; + this.sampleUpdateMs = sampleUpdateMs; + this.numSignificantBuckets = numSignificantBuckets; + this.numSignificantSamples = numSignificantSamples; + this.function = function; + int n = (int) (samplePeriodMs / sampleUpdateMs); + buckets = new long[n]; + Arrays.fill(buckets, function.identity()); + numSamples = new int[n]; + Arrays.fill(numSamples, 0); + currentMsSinceEpoch = -1; + currentIndex = -1; + } + + /** + * Flush stale values. + */ + private void flush(long nowMsSinceEpoch) { + Preconditions.checkArgument(nowMsSinceEpoch >= 0); + if (currentIndex < 0) { + currentMsSinceEpoch = nowMsSinceEpoch - (nowMsSinceEpoch % sampleUpdateMs); + currentIndex = 0; + } + Preconditions.checkArgument(nowMsSinceEpoch >= currentMsSinceEpoch); + int newBuckets = + Math.min((int) ((nowMsSinceEpoch - currentMsSinceEpoch) / sampleUpdateMs), + buckets.length); + while (newBuckets > 0) { + currentIndex = (currentIndex + 1) % buckets.length; + buckets[currentIndex] = function.identity(); + numSamples[currentIndex] = 0; + newBuckets--; + currentMsSinceEpoch += sampleUpdateMs; + } + } + + /** + * Add {@code value} at {@code nowMsSinceEpoch}. + */ + public void add(long nowMsSinceEpoch, long value) { + flush(nowMsSinceEpoch); + buckets[currentIndex] = function.apply(buckets[currentIndex], value); + numSamples[currentIndex]++; + } + + /** + * Return the minimum/maximum/sum of all retained values within {@link #samplePeriodMs} + * of {@code nowMsSinceEpoch}. + */ + public long get(long nowMsSinceEpoch) { + flush(nowMsSinceEpoch); + long result = function.identity(); + for (int i = 0; i < buckets.length; i++) { + result = function.apply(result, buckets[i]); + } + return result; + } + + /** + * Is the current result 'significant'? Ie is it drawn from enough buckets + * or from enough samples? + */ + public boolean isSignificant() { + int totalSamples = 0; + int activeBuckets = 0; + for (int i = 0; i < buckets.length; i++) { + totalSamples += numSamples[i]; + if (numSamples[i] > 0) { + activeBuckets++; + } + } + return activeBuckets >= numSignificantBuckets || totalSamples >= numSignificantSamples; + } +} diff --git a/sdks/java/core/src/test/java/com/google/cloud/dataflow/sdk/util/BucketingFunctionTest.java b/sdks/java/core/src/test/java/com/google/cloud/dataflow/sdk/util/BucketingFunctionTest.java new file mode 100644 index 000000000000..f5e52e059ec1 --- /dev/null +++ b/sdks/java/core/src/test/java/com/google/cloud/dataflow/sdk/util/BucketingFunctionTest.java @@ -0,0 +1,104 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package com.google.cloud.dataflow.sdk.util; + +import com.google.cloud.dataflow.sdk.transforms.Combine; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertFalse; +import static org.junit.Assert.assertTrue; +import org.junit.Test; +import org.junit.runner.RunWith; +import org.junit.runners.JUnit4; + +/** + * Tests {@link BucketingFunction}. + */ +@RunWith(JUnit4.class) +public class BucketingFunctionTest { + + private static final long BUCKET_WIDTH = 10; + private static final int SIGNIFICANT_BUCKETS = 2; + private static final int SIGNIFICANT_SAMPLES = 10; + + private static final Combine.BinaryCombineLongFn SUM = + new Combine.BinaryCombineLongFn() { + @Override + public long apply(long left, long right) { + return left + right; + } + + @Override + public long identity() { + return 0; + } + }; + + private BucketingFunction newFunc() { + return new + BucketingFunction(BUCKET_WIDTH, SIGNIFICANT_BUCKETS, + SIGNIFICANT_SAMPLES, SUM); + } + + @Test + public void significantSamples() { + BucketingFunction f = newFunc(); + assertFalse(f.isSignificant()); + for (int i = 0; i < SIGNIFICANT_SAMPLES - 1; i++) { + f.add(0, 0); + assertFalse(f.isSignificant()); + } + f.add(0, 0); + assertTrue(f.isSignificant()); + } + + @Test + public void significantBuckets() { + BucketingFunction f = newFunc(); + assertFalse(f.isSignificant()); + f.add(0, 0); + assertFalse(f.isSignificant()); + f.add(BUCKET_WIDTH, 0); + assertTrue(f.isSignificant()); + } + + @Test + public void sum() { + BucketingFunction f = newFunc(); + for (int i = 0; i < 100; i++) { + f.add(i, i); + assertEquals(((i + 1) * i) / 2, f.get()); + } + } + + @Test + public void movingSum() { + BucketingFunction f = newFunc(); + int lost = 0; + for (int i = 0; i < 200; i++) { + f.add(i, 1); + if (i >= 100) { + f.remove(i - 100); + if (i % BUCKET_WIDTH == BUCKET_WIDTH - 1) { + lost += BUCKET_WIDTH; + } + } + assertEquals(i + 1 - lost, f.get()); + } + } +} diff --git a/sdks/java/core/src/test/java/com/google/cloud/dataflow/sdk/util/MovingFunctionTest.java b/sdks/java/core/src/test/java/com/google/cloud/dataflow/sdk/util/MovingFunctionTest.java new file mode 100644 index 000000000000..2ca8b7cc7c6d --- /dev/null +++ b/sdks/java/core/src/test/java/com/google/cloud/dataflow/sdk/util/MovingFunctionTest.java @@ -0,0 +1,115 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package com.google.cloud.dataflow.sdk.util; + +import com.google.cloud.dataflow.sdk.transforms.Combine; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertFalse; +import static org.junit.Assert.assertTrue; +import org.junit.Test; +import org.junit.runner.RunWith; +import org.junit.runners.JUnit4; + +/** + * Tests {@link MovingFunction}. + */ +@RunWith(JUnit4.class) +public class MovingFunctionTest { + + private static final long SAMPLE_PERIOD = 100; + private static final long SAMPLE_UPDATE = 10; + private static final int SIGNIFICANT_BUCKETS = 2; + private static final int SIGNIFICANT_SAMPLES = 10; + + private static final Combine.BinaryCombineLongFn SUM = + new Combine.BinaryCombineLongFn() { + @Override + public long apply(long left, long right) { + return left + right; + } + + @Override + public long identity() { + return 0; + } + }; + + private MovingFunction newFunc() { + return new + MovingFunction(SAMPLE_PERIOD, SAMPLE_UPDATE, SIGNIFICANT_BUCKETS, + SIGNIFICANT_SAMPLES, SUM); + + } + + @Test + public void significantSamples() { + MovingFunction f = newFunc(); + assertFalse(f.isSignificant()); + for (int i = 0; i < SIGNIFICANT_SAMPLES - 1; i++) { + f.add(0, 0); + assertFalse(f.isSignificant()); + } + f.add(0, 0); + assertTrue(f.isSignificant()); + } + + @Test + public void significantBuckets() { + MovingFunction f = newFunc(); + assertFalse(f.isSignificant()); + f.add(0, 0); + assertFalse(f.isSignificant()); + f.add(SAMPLE_UPDATE, 0); + assertTrue(f.isSignificant()); + } + + @Test + public void sum() { + MovingFunction f = newFunc(); + for (int i = 0; i < SAMPLE_PERIOD; i++) { + f.add(i, i); + assertEquals(((i + 1) * i) / 2, f.get(i)); + } + } + + @Test + public void movingSum() { + MovingFunction f = newFunc(); + int lost = 0; + for (int i = 0; i < SAMPLE_PERIOD * 2; i++) { + f.add(i , 1); + if (i >= SAMPLE_PERIOD) { + if (i % SAMPLE_UPDATE == 0) { + lost += SAMPLE_UPDATE; + } + } + assertEquals(i + 1 - lost, f.get(i)); + } + } + + @Test + public void jumpingSum() { + MovingFunction f = newFunc(); + f.add(0, 1); + f.add(SAMPLE_PERIOD - 1, 1); + assertEquals(2, f.get(SAMPLE_PERIOD - 1)); + assertEquals(1, f.get(SAMPLE_PERIOD + 3 * SAMPLE_UPDATE)); + assertEquals(0, f.get(SAMPLE_PERIOD * 2)); + } +} From 027bf6cd3062ed381b263aa60455a208de5263aa Mon Sep 17 00:00:00 2001 From: Mark Shields Date: Thu, 7 Apr 2016 11:32:26 -0700 Subject: [PATCH 04/14] Clarrify UnboundedSource comments --- .../dataflow/sdk/io/UnboundedSource.java | 25 ++++++++++++++----- 1 file changed, 19 insertions(+), 6 deletions(-) diff --git a/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/io/UnboundedSource.java b/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/io/UnboundedSource.java index 67b7f1067adc..096888a5f110 100644 --- a/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/io/UnboundedSource.java +++ b/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/io/UnboundedSource.java @@ -110,8 +110,21 @@ public interface CheckpointMark { *

For example, this could be sending acknowledgement requests to an external * data source such as Pub/Sub. * - *

This may be called from any thread, potentially at the same time as calls to the - * {@code UnboundedReader} that created it. + *

Note that: + *

    + *
  • This finalize method may be called from any thread, potentially at the same time as + * calls to the {@link UnboundedReader} it was created from. + *
  • Checkpoints will be finalized in the same order they are created from the + * {@link UnboundedReader}. + *
  • It is possible for multiple checkpoints created from the same {@link UnboundedReader} + * to be 'in-flight' (ie not yet finalized) simultaneously. + *
  • If this call throws an exception then the entire checkpoint will be abandoned and the + * reader restarted from an earlier, successfully-finalized checkpoint. + *
  • The system ensures that if a checkpoint fails for any reason then no later checkpoint + * is allowed to be finalized without the reader first being restarted. + *
  • It is not safe to assume the {@link UnboundedReader} from which this checkpoint was + * created still exists at the time this method is called. + *
*/ void finalizeCheckpoint() throws IOException; } @@ -205,10 +218,10 @@ public byte[] getCurrentRecordId() throws NoSuchElementException { /** * Returns a {@link CheckpointMark} representing the progress of this {@code UnboundedReader}. * - *

The elements read up until this is called will be processed together as a bundle. Once - * the result of this processing has been durably committed, - * {@link CheckpointMark#finalizeCheckpoint} will be called on the {@link CheckpointMark} - * object. + *

All elements read up until this method is called will be processed together as a bundle. + * Once the result of processing those elements and the returned checkpoint have been durably + * committed, {@link CheckpointMark#finalizeCheckpoint} will be called on the + * returned {@link CheckpointMark} object. * *

The returned object should not be modified. * From 1d6ec53f721cc315514275d2ca1afbaacf96edf7 Mon Sep 17 00:00:00 2001 From: Mark Shields Date: Mon, 28 Mar 2016 16:25:29 -0700 Subject: [PATCH 05/14] Copy over. Break deps on RetryHttpInitializer --- integration/java/pom.xml | 236 ++++ .../nexmark/AbstractSimulator.java | 210 +++ .../dataflow/integration/nexmark/Auction.java | 188 +++ .../integration/nexmark/AuctionBid.java | 85 ++ .../integration/nexmark/AuctionCount.java | 88 ++ .../integration/nexmark/AuctionPrice.java | 89 ++ .../dataflow/integration/nexmark/Bid.java | 176 +++ .../integration/nexmark/BidsPerSession.java | 87 ++ .../nexmark/BoundedEventSource.java | 195 +++ .../integration/nexmark/CategoryPrice.java | 98 ++ .../dataflow/integration/nexmark/Done.java | 81 ++ .../dataflow/integration/nexmark/Event.java | 180 +++ .../integration/nexmark/Generator.java | 611 +++++++++ .../integration/nexmark/GeneratorConfig.java | 277 ++++ .../integration/nexmark/IdNameReserve.java | 98 ++ .../integration/nexmark/KnownSize.java | 25 + .../dataflow/integration/nexmark/Monitor.java | 100 ++ .../integration/nexmark/NameCityStateId.java | 104 ++ .../nexmark/NexmarkConfiguration.java | 803 ++++++++++++ .../integration/nexmark/NexmarkDriver.java | 625 +++++++++ .../integration/nexmark/NexmarkPerf.java | 206 +++ .../integration/nexmark/NexmarkQuery.java | 271 ++++ .../nexmark/NexmarkQueryModel.java | 121 ++ .../integration/nexmark/NexmarkRunner.java | 1146 +++++++++++++++++ .../integration/nexmark/NexmarkSuite.java | 372 ++++++ .../integration/nexmark/NexmarkUtils.java | 605 +++++++++ .../dataflow/integration/nexmark/Person.java | 164 +++ .../integration/nexmark/PubsubHelper.java | 237 ++++ .../dataflow/integration/nexmark/Query0.java | 70 + .../integration/nexmark/Query0Model.java | 60 + .../dataflow/integration/nexmark/Query1.java | 62 + .../dataflow/integration/nexmark/Query10.java | 307 +++++ .../dataflow/integration/nexmark/Query11.java | 73 ++ .../integration/nexmark/Query1Model.java | 71 + .../dataflow/integration/nexmark/Query2.java | 73 ++ .../integration/nexmark/Query2Model.java | 74 ++ .../dataflow/integration/nexmark/Query3.java | 246 ++++ .../integration/nexmark/Query3Model.java | 117 ++ .../dataflow/integration/nexmark/Query4.java | 108 ++ .../integration/nexmark/Query4Model.java | 179 +++ .../dataflow/integration/nexmark/Query5.java | 125 ++ .../integration/nexmark/Query5Model.java | 172 +++ .../dataflow/integration/nexmark/Query6.java | 152 +++ .../integration/nexmark/Query6Model.java | 126 ++ .../dataflow/integration/nexmark/Query7.java | 85 ++ .../integration/nexmark/Query7Model.java | 126 ++ .../dataflow/integration/nexmark/Query8.java | 90 ++ .../integration/nexmark/Query8Model.java | 143 ++ .../dataflow/integration/nexmark/Query9.java | 38 + .../integration/nexmark/Query9Model.java | 42 + .../integration/nexmark/SellerPrice.java | 89 ++ .../nexmark/UnboundedEventSource.java | 320 +++++ .../integration/nexmark/WinningBids.java | 383 ++++++ .../nexmark/WinningBidsSimulator.java | 201 +++ pom.xml | 72 +- 55 files changed, 11032 insertions(+), 50 deletions(-) create mode 100644 integration/java/pom.xml create mode 100644 integration/java/src/main/java/com/google/cloud/dataflow/integration/nexmark/AbstractSimulator.java create mode 100644 integration/java/src/main/java/com/google/cloud/dataflow/integration/nexmark/Auction.java create mode 100644 integration/java/src/main/java/com/google/cloud/dataflow/integration/nexmark/AuctionBid.java create mode 100644 integration/java/src/main/java/com/google/cloud/dataflow/integration/nexmark/AuctionCount.java create mode 100644 integration/java/src/main/java/com/google/cloud/dataflow/integration/nexmark/AuctionPrice.java create mode 100644 integration/java/src/main/java/com/google/cloud/dataflow/integration/nexmark/Bid.java create mode 100644 integration/java/src/main/java/com/google/cloud/dataflow/integration/nexmark/BidsPerSession.java create mode 100644 integration/java/src/main/java/com/google/cloud/dataflow/integration/nexmark/BoundedEventSource.java create mode 100644 integration/java/src/main/java/com/google/cloud/dataflow/integration/nexmark/CategoryPrice.java create mode 100644 integration/java/src/main/java/com/google/cloud/dataflow/integration/nexmark/Done.java create mode 100644 integration/java/src/main/java/com/google/cloud/dataflow/integration/nexmark/Event.java create mode 100644 integration/java/src/main/java/com/google/cloud/dataflow/integration/nexmark/Generator.java create mode 100644 integration/java/src/main/java/com/google/cloud/dataflow/integration/nexmark/GeneratorConfig.java create mode 100644 integration/java/src/main/java/com/google/cloud/dataflow/integration/nexmark/IdNameReserve.java create mode 100644 integration/java/src/main/java/com/google/cloud/dataflow/integration/nexmark/KnownSize.java create mode 100644 integration/java/src/main/java/com/google/cloud/dataflow/integration/nexmark/Monitor.java create mode 100644 integration/java/src/main/java/com/google/cloud/dataflow/integration/nexmark/NameCityStateId.java create mode 100644 integration/java/src/main/java/com/google/cloud/dataflow/integration/nexmark/NexmarkConfiguration.java create mode 100644 integration/java/src/main/java/com/google/cloud/dataflow/integration/nexmark/NexmarkDriver.java create mode 100644 integration/java/src/main/java/com/google/cloud/dataflow/integration/nexmark/NexmarkPerf.java create mode 100644 integration/java/src/main/java/com/google/cloud/dataflow/integration/nexmark/NexmarkQuery.java create mode 100644 integration/java/src/main/java/com/google/cloud/dataflow/integration/nexmark/NexmarkQueryModel.java create mode 100644 integration/java/src/main/java/com/google/cloud/dataflow/integration/nexmark/NexmarkRunner.java create mode 100644 integration/java/src/main/java/com/google/cloud/dataflow/integration/nexmark/NexmarkSuite.java create mode 100644 integration/java/src/main/java/com/google/cloud/dataflow/integration/nexmark/NexmarkUtils.java create mode 100644 integration/java/src/main/java/com/google/cloud/dataflow/integration/nexmark/Person.java create mode 100644 integration/java/src/main/java/com/google/cloud/dataflow/integration/nexmark/PubsubHelper.java create mode 100644 integration/java/src/main/java/com/google/cloud/dataflow/integration/nexmark/Query0.java create mode 100644 integration/java/src/main/java/com/google/cloud/dataflow/integration/nexmark/Query0Model.java create mode 100644 integration/java/src/main/java/com/google/cloud/dataflow/integration/nexmark/Query1.java create mode 100644 integration/java/src/main/java/com/google/cloud/dataflow/integration/nexmark/Query10.java create mode 100644 integration/java/src/main/java/com/google/cloud/dataflow/integration/nexmark/Query11.java create mode 100644 integration/java/src/main/java/com/google/cloud/dataflow/integration/nexmark/Query1Model.java create mode 100644 integration/java/src/main/java/com/google/cloud/dataflow/integration/nexmark/Query2.java create mode 100644 integration/java/src/main/java/com/google/cloud/dataflow/integration/nexmark/Query2Model.java create mode 100644 integration/java/src/main/java/com/google/cloud/dataflow/integration/nexmark/Query3.java create mode 100644 integration/java/src/main/java/com/google/cloud/dataflow/integration/nexmark/Query3Model.java create mode 100644 integration/java/src/main/java/com/google/cloud/dataflow/integration/nexmark/Query4.java create mode 100644 integration/java/src/main/java/com/google/cloud/dataflow/integration/nexmark/Query4Model.java create mode 100644 integration/java/src/main/java/com/google/cloud/dataflow/integration/nexmark/Query5.java create mode 100644 integration/java/src/main/java/com/google/cloud/dataflow/integration/nexmark/Query5Model.java create mode 100644 integration/java/src/main/java/com/google/cloud/dataflow/integration/nexmark/Query6.java create mode 100644 integration/java/src/main/java/com/google/cloud/dataflow/integration/nexmark/Query6Model.java create mode 100644 integration/java/src/main/java/com/google/cloud/dataflow/integration/nexmark/Query7.java create mode 100644 integration/java/src/main/java/com/google/cloud/dataflow/integration/nexmark/Query7Model.java create mode 100644 integration/java/src/main/java/com/google/cloud/dataflow/integration/nexmark/Query8.java create mode 100644 integration/java/src/main/java/com/google/cloud/dataflow/integration/nexmark/Query8Model.java create mode 100644 integration/java/src/main/java/com/google/cloud/dataflow/integration/nexmark/Query9.java create mode 100644 integration/java/src/main/java/com/google/cloud/dataflow/integration/nexmark/Query9Model.java create mode 100644 integration/java/src/main/java/com/google/cloud/dataflow/integration/nexmark/SellerPrice.java create mode 100644 integration/java/src/main/java/com/google/cloud/dataflow/integration/nexmark/UnboundedEventSource.java create mode 100644 integration/java/src/main/java/com/google/cloud/dataflow/integration/nexmark/WinningBids.java create mode 100644 integration/java/src/main/java/com/google/cloud/dataflow/integration/nexmark/WinningBidsSimulator.java diff --git a/integration/java/pom.xml b/integration/java/pom.xml new file mode 100644 index 000000000000..f71534ca80b4 --- /dev/null +++ b/integration/java/pom.xml @@ -0,0 +1,236 @@ + + + + 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 + + + + + maven-compiler-plugin + 3.1 + + 1.7 + 1.7 + + + + + org.apache.maven.plugins + maven-dependency-plugin + + + + org.apache.maven.plugins + maven-checkstyle-plugin + 2.12 + + + com.puppycrawl.tools + checkstyle + 6.6 + + + + ../../checkstyle.xml + true + true + true + + + + + check + + + + + + + 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 + ${project.version} + + + + org.apache.beam + java-sdk-all + ${project.version} + test-jar + + + + 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 + ${junit.version} + + + diff --git a/integration/java/src/main/java/com/google/cloud/dataflow/integration/nexmark/AbstractSimulator.java b/integration/java/src/main/java/com/google/cloud/dataflow/integration/nexmark/AbstractSimulator.java new file mode 100644 index 000000000000..664aa952a02b --- /dev/null +++ b/integration/java/src/main/java/com/google/cloud/dataflow/integration/nexmark/AbstractSimulator.java @@ -0,0 +1,210 @@ +/* + * Copyright (C) 2015 Google Inc. + * + * Licensed under the Apache License, Version 2.0 (the "License"); you may not + * use this file except in compliance with the License. You may obtain a copy of + * the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT + * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the + * License for the specific language governing permissions and limitations under + * the License. + */ + +package com.google.cloud.dataflow.integration.nexmark; + +import com.google.cloud.dataflow.sdk.transforms.windowing.BoundedWindow; +import com.google.cloud.dataflow.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/com/google/cloud/dataflow/integration/nexmark/Auction.java b/integration/java/src/main/java/com/google/cloud/dataflow/integration/nexmark/Auction.java new file mode 100644 index 000000000000..1ff5847bbaed --- /dev/null +++ b/integration/java/src/main/java/com/google/cloud/dataflow/integration/nexmark/Auction.java @@ -0,0 +1,188 @@ +/* + * Copyright (C) 2015 Google Inc. + * + * Licensed under the Apache License, Version 2.0 (the "License"); you may not + * use this file except in compliance with the License. You may obtain a copy of + * the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT + * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the + * License for the specific language governing permissions and limitations under + * the License. + */ + +package com.google.cloud.dataflow.integration.nexmark; + +import com.google.cloud.dataflow.sdk.coders.AtomicCoder; +import com.google.cloud.dataflow.sdk.coders.Coder; +import com.google.cloud.dataflow.sdk.coders.CoderException; +import com.google.cloud.dataflow.sdk.coders.StringUtf8Coder; +import com.google.cloud.dataflow.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, + com.google.cloud.dataflow.sdk.coders.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, com.google.cloud.dataflow.sdk.coders.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/com/google/cloud/dataflow/integration/nexmark/AuctionBid.java b/integration/java/src/main/java/com/google/cloud/dataflow/integration/nexmark/AuctionBid.java new file mode 100644 index 000000000000..b1b505d06c7f --- /dev/null +++ b/integration/java/src/main/java/com/google/cloud/dataflow/integration/nexmark/AuctionBid.java @@ -0,0 +1,85 @@ +/* + * Copyright (C) 2015 Google Inc. + * + * Licensed under the Apache License, Version 2.0 (the "License"); you may not + * use this file except in compliance with the License. You may obtain a copy of + * the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT + * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the + * License for the specific language governing permissions and limitations under + * the License. + */ + +package com.google.cloud.dataflow.integration.nexmark; + +import com.google.cloud.dataflow.sdk.coders.AtomicCoder; +import com.google.cloud.dataflow.sdk.coders.Coder; +import com.google.cloud.dataflow.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, + com.google.cloud.dataflow.sdk.coders.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, com.google.cloud.dataflow.sdk.coders.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/com/google/cloud/dataflow/integration/nexmark/AuctionCount.java b/integration/java/src/main/java/com/google/cloud/dataflow/integration/nexmark/AuctionCount.java new file mode 100644 index 000000000000..663ab3aa1817 --- /dev/null +++ b/integration/java/src/main/java/com/google/cloud/dataflow/integration/nexmark/AuctionCount.java @@ -0,0 +1,88 @@ +/* + * Copyright (C) 2015 Google Inc. + * + * Licensed under the Apache License, Version 2.0 (the "License"); you may not + * use this file except in compliance with the License. You may obtain a copy of + * the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT + * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the + * License for the specific language governing permissions and limitations under + * the License. + */ + +package com.google.cloud.dataflow.integration.nexmark; + +import com.google.cloud.dataflow.sdk.coders.AtomicCoder; +import com.google.cloud.dataflow.sdk.coders.Coder; +import com.google.cloud.dataflow.sdk.coders.CoderException; +import com.google.cloud.dataflow.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, + com.google.cloud.dataflow.sdk.coders.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, com.google.cloud.dataflow.sdk.coders.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/com/google/cloud/dataflow/integration/nexmark/AuctionPrice.java b/integration/java/src/main/java/com/google/cloud/dataflow/integration/nexmark/AuctionPrice.java new file mode 100644 index 000000000000..e3a5678cf9a0 --- /dev/null +++ b/integration/java/src/main/java/com/google/cloud/dataflow/integration/nexmark/AuctionPrice.java @@ -0,0 +1,89 @@ +/* + * Copyright (C) 2015 Google Inc. + * + * Licensed under the Apache License, Version 2.0 (the "License"); you may not + * use this file except in compliance with the License. You may obtain a copy of + * the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT + * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the + * License for the specific language governing permissions and limitations under + * the License. + */ + +package com.google.cloud.dataflow.integration.nexmark; + +import com.google.cloud.dataflow.sdk.coders.AtomicCoder; +import com.google.cloud.dataflow.sdk.coders.Coder; +import com.google.cloud.dataflow.sdk.coders.CoderException; +import com.google.cloud.dataflow.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, + com.google.cloud.dataflow.sdk.coders.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, com.google.cloud.dataflow.sdk.coders.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/com/google/cloud/dataflow/integration/nexmark/Bid.java b/integration/java/src/main/java/com/google/cloud/dataflow/integration/nexmark/Bid.java new file mode 100644 index 000000000000..bb654c9ae170 --- /dev/null +++ b/integration/java/src/main/java/com/google/cloud/dataflow/integration/nexmark/Bid.java @@ -0,0 +1,176 @@ +/* + * Copyright (C) 2015 Google Inc. + * + * Licensed under the Apache License, Version 2.0 (the "License"); you may not + * use this file except in compliance with the License. You may obtain a copy of + * the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT + * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the + * License for the specific language governing permissions and limitations under + * the License. + */ + +package com.google.cloud.dataflow.integration.nexmark; + +import com.google.cloud.dataflow.sdk.coders.AtomicCoder; +import com.google.cloud.dataflow.sdk.coders.Coder; +import com.google.cloud.dataflow.sdk.coders.CoderException; +import com.google.cloud.dataflow.sdk.coders.StringUtf8Coder; +import com.google.cloud.dataflow.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, + com.google.cloud.dataflow.sdk.coders.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, com.google.cloud.dataflow.sdk.coders.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/com/google/cloud/dataflow/integration/nexmark/BidsPerSession.java b/integration/java/src/main/java/com/google/cloud/dataflow/integration/nexmark/BidsPerSession.java new file mode 100644 index 000000000000..41223b0e56e7 --- /dev/null +++ b/integration/java/src/main/java/com/google/cloud/dataflow/integration/nexmark/BidsPerSession.java @@ -0,0 +1,87 @@ +/* + * Copyright (C) 2015 Google Inc. + * + * Licensed under the Apache License, Version 2.0 (the "License"); you may not + * use this file except in compliance with the License. You may obtain a copy of + * the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT + * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the + * License for the specific language governing permissions and limitations under + * the License. + */ + +package com.google.cloud.dataflow.integration.nexmark; + +import com.google.cloud.dataflow.sdk.coders.AtomicCoder; +import com.google.cloud.dataflow.sdk.coders.Coder; +import com.google.cloud.dataflow.sdk.coders.CoderException; +import com.google.cloud.dataflow.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, + com.google.cloud.dataflow.sdk.coders.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, com.google.cloud.dataflow.sdk.coders.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/com/google/cloud/dataflow/integration/nexmark/BoundedEventSource.java b/integration/java/src/main/java/com/google/cloud/dataflow/integration/nexmark/BoundedEventSource.java new file mode 100644 index 000000000000..4d1688a99517 --- /dev/null +++ b/integration/java/src/main/java/com/google/cloud/dataflow/integration/nexmark/BoundedEventSource.java @@ -0,0 +1,195 @@ +/* + * Copyright (C) 2015 Google Inc. + * + * Licensed under the Apache License, Version 2.0 (the "License"); you may not + * use this file except in compliance with the License. You may obtain a copy of + * the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT + * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the + * License for the specific language governing permissions and limitations under + * the License. + */ + +package com.google.cloud.dataflow.integration.nexmark; + +import com.google.cloud.dataflow.sdk.coders.Coder; +import com.google.cloud.dataflow.sdk.io.BoundedSource; +import com.google.cloud.dataflow.sdk.options.PipelineOptions; +import com.google.cloud.dataflow.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/com/google/cloud/dataflow/integration/nexmark/CategoryPrice.java b/integration/java/src/main/java/com/google/cloud/dataflow/integration/nexmark/CategoryPrice.java new file mode 100644 index 000000000000..db50535119c6 --- /dev/null +++ b/integration/java/src/main/java/com/google/cloud/dataflow/integration/nexmark/CategoryPrice.java @@ -0,0 +1,98 @@ +/* + * Copyright (C) 2015 Google Inc. + * + * Licensed under the Apache License, Version 2.0 (the "License"); you may not + * use this file except in compliance with the License. You may obtain a copy of + * the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT + * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the + * License for the specific language governing permissions and limitations under + * the License. + */ + +package com.google.cloud.dataflow.integration.nexmark; + +import com.google.cloud.dataflow.sdk.coders.AtomicCoder; +import com.google.cloud.dataflow.sdk.coders.Coder; +import com.google.cloud.dataflow.sdk.coders.CoderException; +import com.google.cloud.dataflow.sdk.coders.VarIntCoder; +import com.google.cloud.dataflow.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, + com.google.cloud.dataflow.sdk.coders.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, com.google.cloud.dataflow.sdk.coders.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/com/google/cloud/dataflow/integration/nexmark/Done.java b/integration/java/src/main/java/com/google/cloud/dataflow/integration/nexmark/Done.java new file mode 100644 index 000000000000..7467cf95dec4 --- /dev/null +++ b/integration/java/src/main/java/com/google/cloud/dataflow/integration/nexmark/Done.java @@ -0,0 +1,81 @@ +/* + * Copyright (C) 2015 Google Inc. + * + * Licensed under the Apache License, Version 2.0 (the "License"); you may not + * use this file except in compliance with the License. You may obtain a copy of + * the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT + * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the + * License for the specific language governing permissions and limitations under + * the License. + */ + +package com.google.cloud.dataflow.integration.nexmark; + +import com.google.cloud.dataflow.sdk.coders.AtomicCoder; +import com.google.cloud.dataflow.sdk.coders.Coder; +import com.google.cloud.dataflow.sdk.coders.CoderException; +import com.google.cloud.dataflow.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, + com.google.cloud.dataflow.sdk.coders.Coder.Context context) + throws CoderException, IOException { + STRING_CODER.encode(value.message, outStream, context.nested()); + } + + @Override + public Done decode( + InputStream inStream, com.google.cloud.dataflow.sdk.coders.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/com/google/cloud/dataflow/integration/nexmark/Event.java b/integration/java/src/main/java/com/google/cloud/dataflow/integration/nexmark/Event.java new file mode 100644 index 000000000000..1c4ffec4273f --- /dev/null +++ b/integration/java/src/main/java/com/google/cloud/dataflow/integration/nexmark/Event.java @@ -0,0 +1,180 @@ +/* + * Copyright (C) 2015 Google Inc. + * + * Licensed under the Apache License, Version 2.0 (the "License"); you may not + * use this file except in compliance with the License. You may obtain a copy of + * the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT + * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the + * License for the specific language governing permissions and limitations under + * the License. + */ + +package com.google.cloud.dataflow.integration.nexmark; + +import com.google.cloud.dataflow.sdk.coders.AtomicCoder; +import com.google.cloud.dataflow.sdk.coders.Coder; +import com.google.cloud.dataflow.sdk.coders.CoderException; +import com.google.cloud.dataflow.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, + com.google.cloud.dataflow.sdk.coders.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, com.google.cloud.dataflow.sdk.coders.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/com/google/cloud/dataflow/integration/nexmark/Generator.java b/integration/java/src/main/java/com/google/cloud/dataflow/integration/nexmark/Generator.java new file mode 100644 index 000000000000..6afa0c6df04b --- /dev/null +++ b/integration/java/src/main/java/com/google/cloud/dataflow/integration/nexmark/Generator.java @@ -0,0 +1,611 @@ +/* + * Copyright (C) 2015 Google Inc. + * + * Licensed under the Apache License, Version 2.0 (the "License"); you may not + * use this file except in compliance with the License. You may obtain a copy of + * the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT + * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the + * License for the specific language governing permissions and limitations under + * the License. + */ + +package com.google.cloud.dataflow.integration.nexmark; + +import com.google.cloud.dataflow.sdk.coders.AtomicCoder; +import com.google.cloud.dataflow.sdk.coders.Coder; +import com.google.cloud.dataflow.sdk.coders.CoderException; +import com.google.cloud.dataflow.sdk.coders.VarLongCoder; +import com.google.cloud.dataflow.sdk.io.UnboundedSource; +import com.google.cloud.dataflow.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 com.google.cloud.dataflow.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, + com.google.cloud.dataflow.sdk.coders.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, com.google.cloud.dataflow.sdk.coders.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); + long newPreloadEvents = Math.min(config.numPreloadEvents, newMaxEvents); + GeneratorConfig remainConfig = + config.cloneWith(config.firstEventId, config.maxEvents - newMaxEvents, + config.numPreloadEvents - newPreloadEvents, config.firstEventNumber + newMaxEvents); + config = config.cloneWith( + config.firstEventId, newMaxEvents, newPreloadEvents, 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(); + // In preload we'll emit each event at preload intervals instead of 'regular' intervals. + long preloadInterEventDelayUs = NexmarkUtils.QpsShape.SQUARE.interEventDelayUs( + config.configuration.preloadEventQps, config.configuration.numEventGenerators); + // When, in wallclock time, we should emit the event. + long wallclockTimestamp; + if (numEvents < config.numPreloadEvents) { + // While preloading, emit as fast as the preload rate will allow us. + wallclockTimestamp = wallclockBaseTime + (numEvents * preloadInterEventDelayUs) / 1000L; + } else { + // Once preload is done, switch to true event time, but offset appropriately. + long wallclockOfFirstPostPreloadEvent = + wallclockBaseTime + (config.numPreloadEvents * preloadInterEventDelayUs) / 1000L; + // = wallclockBaseTime if numPreloadEvents = 0 + long timestampOfFirstPostPreloadEvent = + config.timestampAndInterEventDelayUsForEvent( + config.nextEventNumber(config.numPreloadEvents)).getKey(); + // = getCurrentConfig().baseTime if numPreloadEvents = 0 + wallclockTimestamp = + wallclockOfFirstPostPreloadEvent + (eventTimestamp - timestampOfFirstPostPreloadEvent); + // = wallclockBaseTime + (eventTimestamp - getCurrentConfig().baseTime) + // if numPreloadEvents = 0 + } + + // 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/com/google/cloud/dataflow/integration/nexmark/GeneratorConfig.java b/integration/java/src/main/java/com/google/cloud/dataflow/integration/nexmark/GeneratorConfig.java new file mode 100644 index 000000000000..6f6f2572f035 --- /dev/null +++ b/integration/java/src/main/java/com/google/cloud/dataflow/integration/nexmark/GeneratorConfig.java @@ -0,0 +1,277 @@ +/* + * Copyright (C) 2015 Google Inc. + * + * Licensed under the Apache License, Version 2.0 (the "License"); you may not + * use this file except in compliance with the License. You may obtain a copy of + * the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT + * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the + * License for the specific language governing permissions and limitations under + * the License. + */ + +package com.google.cloud.dataflow.integration.nexmark; + +import com.google.cloud.dataflow.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_BID_ID = 0L; + 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; + + /** + * How many events to pump without rate limiting when first start. + */ + public final long numPreloadEvents; + + /** + * 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 numPreloadEvents, long firstEventNumber) { + this.configuration = configuration; + this.interEventDelayUs = configuration.qpsShape.interEventDelayUs( + configuration.firstEventQps, configuration.nextEventQps, configuration.numEventGenerators); + this.stepLengthSec = configuration.qpsShape.stepLengthSec(configuration.qpsPeriodSec); + 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.numPreloadEvents = numPreloadEvents; + 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, numPreloadEvents, firstEventNumber); + } + + /** + * Return clone of this config except with given parameters. + */ + public GeneratorConfig cloneWith( + long firstEventId, long maxEvents, long numPreloadEvents, long firstEventNumber) { + return new GeneratorConfig( + configuration, baseTime, firstEventId, maxEvents, numPreloadEvents, 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; + long subNumPreloadEvents = numPreloadEvents / n; + for (int i = 0; i < n; i++) { + if (i == n - 1) { + // Don't loose any events to round-down. + subMaxEvents = maxEvents - subMaxEvents * (n - 1); + subNumPreloadEvents = numPreloadEvents - subNumPreloadEvents * (n - 1); + } + results.add( + cloneWith(subFirstEventId, subMaxEvents, subNumPreloadEvents, 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() { + return String.format( + "GeneratorConfig{configuration:%s, baseTime:%d, firstEventId:%d, " + + "maxEvents:%d, numPreloadEvents:%d, firstEventNumber:%d, epochPeriodMs:%d, " + + "eventsPerEpoch:%d}", + configuration, baseTime, firstEventId, maxEvents, numPreloadEvents, firstEventNumber, + epochPeriodMs, eventsPerEpoch); + } +} diff --git a/integration/java/src/main/java/com/google/cloud/dataflow/integration/nexmark/IdNameReserve.java b/integration/java/src/main/java/com/google/cloud/dataflow/integration/nexmark/IdNameReserve.java new file mode 100644 index 000000000000..d15c97caab7e --- /dev/null +++ b/integration/java/src/main/java/com/google/cloud/dataflow/integration/nexmark/IdNameReserve.java @@ -0,0 +1,98 @@ +/* + * Copyright (C) 2015 Google Inc. + * + * Licensed under the Apache License, Version 2.0 (the "License"); you may not + * use this file except in compliance with the License. You may obtain a copy of + * the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT + * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the + * License for the specific language governing permissions and limitations under + * the License. + */ + +package com.google.cloud.dataflow.integration.nexmark; + +import com.google.cloud.dataflow.sdk.coders.AtomicCoder; +import com.google.cloud.dataflow.sdk.coders.Coder; +import com.google.cloud.dataflow.sdk.coders.CoderException; +import com.google.cloud.dataflow.sdk.coders.StringUtf8Coder; +import com.google.cloud.dataflow.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, + com.google.cloud.dataflow.sdk.coders.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, com.google.cloud.dataflow.sdk.coders.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/com/google/cloud/dataflow/integration/nexmark/KnownSize.java b/integration/java/src/main/java/com/google/cloud/dataflow/integration/nexmark/KnownSize.java new file mode 100644 index 000000000000..89abe7a2df21 --- /dev/null +++ b/integration/java/src/main/java/com/google/cloud/dataflow/integration/nexmark/KnownSize.java @@ -0,0 +1,25 @@ +/* + * Copyright (C) 2015 Google Inc. + * + * Licensed under the Apache License, Version 2.0 (the "License"); you may not + * use this file except in compliance with the License. You may obtain a copy of + * the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT + * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the + * License for the specific language governing permissions and limitations under + * the License. + */ + +package com.google.cloud.dataflow.integration.nexmark; + +/** + * Interface for elements which know their encoded byte size. + */ +public interface KnownSize { + long sizeInBytes(); +} + diff --git a/integration/java/src/main/java/com/google/cloud/dataflow/integration/nexmark/Monitor.java b/integration/java/src/main/java/com/google/cloud/dataflow/integration/nexmark/Monitor.java new file mode 100644 index 000000000000..064fe3570a2b --- /dev/null +++ b/integration/java/src/main/java/com/google/cloud/dataflow/integration/nexmark/Monitor.java @@ -0,0 +1,100 @@ +/* + * Copyright (C) 2015 Google Inc. + * + * Licensed under the Apache License, Version 2.0 (the "License"); you may not + * use this file except in compliance with the License. You may obtain a copy of + * the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT + * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the + * License for the specific language governing permissions and limitations under + * the License. + */ + +package com.google.cloud.dataflow.integration.nexmark; + +import com.google.cloud.dataflow.sdk.transforms.Aggregator; +import com.google.cloud.dataflow.sdk.transforms.DoFn; +import com.google.cloud.dataflow.sdk.transforms.Max.MaxLongFn; +import com.google.cloud.dataflow.sdk.transforms.Min.MinLongFn; +import com.google.cloud.dataflow.sdk.transforms.PTransform; +import com.google.cloud.dataflow.sdk.transforms.ParDo; +import com.google.cloud.dataflow.sdk.transforms.Sum.SumLongFn; +import com.google.cloud.dataflow.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. + */ +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/com/google/cloud/dataflow/integration/nexmark/NameCityStateId.java b/integration/java/src/main/java/com/google/cloud/dataflow/integration/nexmark/NameCityStateId.java new file mode 100644 index 000000000000..e007240d6b5e --- /dev/null +++ b/integration/java/src/main/java/com/google/cloud/dataflow/integration/nexmark/NameCityStateId.java @@ -0,0 +1,104 @@ +/* + * Copyright (C) 2015 Google Inc. + * + * Licensed under the Apache License, Version 2.0 (the "License"); you may not + * use this file except in compliance with the License. You may obtain a copy of + * the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT + * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the + * License for the specific language governing permissions and limitations under + * the License. + */ + +package com.google.cloud.dataflow.integration.nexmark; + +import com.google.cloud.dataflow.sdk.coders.AtomicCoder; +import com.google.cloud.dataflow.sdk.coders.Coder; +import com.google.cloud.dataflow.sdk.coders.CoderException; +import com.google.cloud.dataflow.sdk.coders.StringUtf8Coder; +import com.google.cloud.dataflow.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, + com.google.cloud.dataflow.sdk.coders.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, com.google.cloud.dataflow.sdk.coders.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/com/google/cloud/dataflow/integration/nexmark/NexmarkConfiguration.java b/integration/java/src/main/java/com/google/cloud/dataflow/integration/nexmark/NexmarkConfiguration.java new file mode 100644 index 000000000000..d79622b02122 --- /dev/null +++ b/integration/java/src/main/java/com/google/cloud/dataflow/integration/nexmark/NexmarkConfiguration.java @@ -0,0 +1,803 @@ +/* + * Copyright (C) 2015 Google Inc. + * + * Licensed under the Apache License, Version 2.0 (the "License"); you may not + * use this file except in compliance with the License. You may obtain a copy of + * the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT + * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the + * License for the specific language governing permissions and limitations under + * the License. + */ + +package com.google.cloud.dataflow.integration.nexmark; + +import com.google.cloud.dataflow.sdk.options.DataflowPipelineOptions; +import com.google.cloud.dataflow.sdk.options.DataflowPipelineWorkerPoolOptions; +import com.google.cloud.dataflow.sdk.options.StreamingOptions; +import com.google.cloud.dataflow.sdk.testing.DataflowAssert; + +import com.fasterxml.jackson.annotation.JsonProperty; +import com.fasterxml.jackson.core.JsonProcessingException; + +import java.io.IOException; +import java.io.Serializable; +import java.util.ArrayList; +import java.util.List; +import java.util.Objects; + +/** + * Configuration controlling how a query is run. May be supplied by command line or + * programmatically. Some properties override those supplied by {@link DataflowPipelineOptions} (eg + * {@code isStreaming}). We try to capture everything which may influence the resulting + * pipeline performance, as captured by {@link NexmarkPerf}. + */ +class NexmarkConfiguration implements Serializable { + public static final NexmarkConfiguration DEFAULT = new NexmarkConfiguration(); + + /** Worker machine type. */ + @JsonProperty + public String workerMachineType = "n1-standard-4"; + + /** Same as {@link DataflowPipelineWorkerPoolOptions#setNumWorkers}. */ + @JsonProperty + public int numWorkers = 1; + + /** Same as {@link DataflowPipelineWorkerPoolOptions#setMaxNumWorkers}. */ + @JsonProperty + public int maxNumWorkers = 1; + + /** Same as {@link DataflowPipelineWorkerPoolOptions#setAutoscalingAlgorithm}. */ + @JsonProperty + public DataflowPipelineWorkerPoolOptions.AutoscalingAlgorithmType autoscalingAlgorithm = + DataflowPipelineWorkerPoolOptions.AutoscalingAlgorithmType.NONE; + + /** Same as {@link StreamingOptions#setStreaming}. */ + @JsonProperty + public boolean streaming = false; + + /** Which query to run, in [0,9]. */ + @JsonProperty + public int query = 0; + + /** If true, emit query result as ERROR log entries. */ + @JsonProperty + public boolean logResults = false; + + /** + * If true, use {@link DataflowAssert} to assert that query results match a hand-written + * model. Only works if {@link #numEvents} is small. + */ + @JsonProperty + public boolean assertCorrectness = false; + + /** 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 events to generate at the special pre-load rate. This is useful for generating + * a backlog of events on pub/sub before the main query begins. + */ + @JsonProperty + public long numPreloadEvents = 0; + + /** + * Number of event generators to use. Each generates events in its own timeline. + */ + @JsonProperty + public int numEventGenerators = 100; + + /** + * Shape of event qps curve. + */ + @JsonProperty + public NexmarkUtils.QpsShape qpsShape = NexmarkUtils.QpsShape.SINE; + + /** + * Initial overall event qps. + */ + @JsonProperty + public int firstEventQps = 10000; + + /** + * Next overall event qps. + */ + @JsonProperty + public int nextEventQps = 10000; + + /** + * Overall period of qps shape, in seconds. + */ + @JsonProperty + public int qpsPeriodSec = 600; + + /** + * Overall event qps while pre-loading. Typcially as large as possible for given pub/sub quota. + */ + @JsonProperty + public int preloadEventQps = 10000; + + /** + * 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; + + /** + * 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. + */ + @JsonProperty + public boolean justModelResultRate = false; + + /** 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; + + /** + * Experiments to enable. + */ + @JsonProperty + public List experiments = new ArrayList<>(); + + /** + * 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; + + /** + * In debug mode, include Monitor & Snoop. + */ + @JsonProperty + public boolean debug = true; + + /** Return number of cores for configuration's machine type. */ + public int coresPerWorker() { + String[] split = workerMachineType.split("-"); + if (split.length != 3) { + return 1; + } + try { + return Integer.parseInt(split[2]); + } catch (NumberFormatException ex) { + return 1; + } + } + + /** + * Replace any properties of this configuration which have been supplied by the command line. + * However *never replace* isStreaming since we can't tell if it was supplied by the command line + * or merely has its default false value. + */ + public void overrideFromOptions(NexmarkDriver.Options options) { + if (options.getWorkerMachineType() != null) { + workerMachineType = options.getWorkerMachineType(); + } + if (options.getNumWorkers() > 0) { + numWorkers = options.getNumWorkers(); + } + if (options.getMaxNumWorkers() > 0) { + maxNumWorkers = options.getMaxNumWorkers(); + } + if (options.getAutoscalingAlgorithm() != null) { + autoscalingAlgorithm = options.getAutoscalingAlgorithm(); + } + if (options.getQuery() != null) { + query = options.getQuery(); + } + if (options.getLogResults() != null) { + logResults = options.getLogResults(); + } + if (options.getAssertCorrectness() != null) { + assertCorrectness = options.getAssertCorrectness(); + } + 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.getNumPreloadEvents() != null) { + numPreloadEvents = options.getNumPreloadEvents(); + } + if (options.getNumEventGenerators() != null) { + numEventGenerators = options.getNumEventGenerators(); + } + if (options.getQpsShape() != null) { + qpsShape = options.getQpsShape(); + } + if (options.getFirstEventQps() != null) { + firstEventQps = options.getFirstEventQps(); + } + if (options.getNextEventQps() != null) { + nextEventQps = options.getNextEventQps(); + } + if (options.getQpsPeriodSec() != null) { + qpsPeriodSec = options.getQpsPeriodSec(); + } + if (options.getPreloadEventQps() != null) { + preloadEventQps = options.getPreloadEventQps(); + } + 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.getJustModelResultRate() != null) { + justModelResultRate = options.getJustModelResultRate(); + } + 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.getExperiments() != null) { + experiments.clear(); + experiments.addAll(options.getExperiments()); + } + 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(); + } + if (options.getDebug() != null) { + debug = options.getDebug(); + } + } + + /** + * Return clone of configuration with given label. + */ + @Override + public NexmarkConfiguration clone() { + NexmarkConfiguration result = new NexmarkConfiguration(); + result.workerMachineType = workerMachineType; + result.numWorkers = numWorkers; + result.maxNumWorkers = maxNumWorkers; + result.autoscalingAlgorithm = autoscalingAlgorithm; + result.streaming = streaming; + result.query = query; + result.logResults = logResults; + result.assertCorrectness = assertCorrectness; + result.sourceType = sourceType; + result.sinkType = sinkType; + result.pubSubMode = pubSubMode; + result.numEvents = numEvents; + result.numPreloadEvents = numPreloadEvents; + result.numEventGenerators = numEventGenerators; + result.qpsShape = qpsShape; + result.firstEventQps = firstEventQps; + result.nextEventQps = nextEventQps; + result.qpsPeriodSec = qpsPeriodSec; + result.preloadEventQps = preloadEventQps; + 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.justModelResultRate = justModelResultRate; + result.coderStrategy = coderStrategy; + result.cpuDelayMs = cpuDelayMs; + result.diskBusyBytes = diskBusyBytes; + result.auctionSkip = auctionSkip; + result.fanout = fanout; + result.experiments.addAll(experiments); + 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)); + sb.append(String.format("; streaming:%s", streaming)); + if (!workerMachineType.equals(DEFAULT.workerMachineType)) { + sb.append(String.format("; workerMachineType:%s", workerMachineType)); + } + if (numWorkers != DEFAULT.numWorkers) { + sb.append(String.format("; numWorkers:%d", numWorkers)); + } + if (maxNumWorkers != DEFAULT.maxNumWorkers) { + sb.append(String.format("; maxNumWorkers:%d", maxNumWorkers)); + } + if (autoscalingAlgorithm != DEFAULT.autoscalingAlgorithm) { + sb.append(String.format("; autosalingAlgorithms:%s", autoscalingAlgorithm)); + } + 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 (numPreloadEvents != DEFAULT.numPreloadEvents) { + sb.append(String.format("; numPreloadEvents:%d", numPreloadEvents)); + } + if (numEventGenerators != DEFAULT.numEventGenerators) { + sb.append(String.format("; numEventGenerators:%d", numEventGenerators)); + } + if (qpsShape != DEFAULT.qpsShape) { + sb.append(String.format("; qpsShare:%s", qpsShape)); + } + if (firstEventQps != DEFAULT.firstEventQps || nextEventQps != DEFAULT.nextEventQps) { + sb.append(String.format("; firstEventQps:%d", firstEventQps)); + sb.append(String.format("; nextEventQps:%d", nextEventQps)); + } + if (qpsPeriodSec != DEFAULT.qpsPeriodSec) { + sb.append(String.format("; qpsPeriodSec:%d", qpsPeriodSec)); + } + if (preloadEventQps != DEFAULT.preloadEventQps) { + sb.append(String.format("; preloadEventQps:%d", preloadEventQps)); + } + 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 (justModelResultRate != DEFAULT.justModelResultRate) { + sb.append(String.format("; justModelResutRate:%s", justModelResultRate)); + } + 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 (!experiments.equals(DEFAULT.experiments)) { + sb.append(String.format("; experiments:%s", experiments)); + } + 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) throws IOException { + return NexmarkUtils.MAPPER.readValue(string, NexmarkConfiguration.class); + } + + @Override + public int hashCode() { + return Objects.hash(workerMachineType, numWorkers, maxNumWorkers, autoscalingAlgorithm, + streaming, query, logResults, assertCorrectness, sourceType, sinkType, pubSubMode, + numEvents, numPreloadEvents, numEventGenerators, qpsShape, firstEventQps, nextEventQps, + qpsPeriodSec, preloadEventQps, isRateLimited, useWallclockEventTime, avgPersonByteSize, + avgAuctionByteSize, avgBidByteSize, hotAuctionRatio, hotSellersRatio, hotBiddersRatio, + windowSizeSec, windowPeriodSec, watermarkHoldbackSec, numInFlightAuctions, numActivePeople, + justModelResultRate, coderStrategy, cpuDelayMs, diskBusyBytes, auctionSkip, fanout, + experiments, 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 (!workerMachineType.equals(other.workerMachineType)) { + return false; + } + if (assertCorrectness != other.assertCorrectness) { + return false; + } + if (auctionSkip != other.auctionSkip) { + return false; + } + if (autoscalingAlgorithm != other.autoscalingAlgorithm) { + 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 (experiments == null) { + if (other.experiments != null) { + return false; + } + } else if (!experiments.equals(other.experiments)) { + return false; + } + if (fanout != other.fanout) { + return false; + } + if (firstEventQps != other.firstEventQps) { + 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 (streaming != other.streaming) { + return false; + } + if (justModelResultRate != other.justModelResultRate) { + return false; + } + if (logResults != other.logResults) { + return false; + } + if (maxLogEvents != other.maxLogEvents) { + return false; + } + if (maxNumWorkers != other.maxNumWorkers) { + return false; + } + if (nextEventQps != other.nextEventQps) { + 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 (numPreloadEvents != other.numPreloadEvents) { + return false; + } + if (numWorkers != other.numWorkers) { + return false; + } + if (occasionalDelaySec != other.occasionalDelaySec) { + return false; + } + if (preloadEventQps != other.preloadEventQps) { + return false; + } + if (Double.doubleToLongBits(probDelayedEvent) + != Double.doubleToLongBits(other.probDelayedEvent)) { + return false; + } + if (pubSubMode != other.pubSubMode) { + return false; + } + if (qpsPeriodSec != other.qpsPeriodSec) { + return false; + } + if (qpsShape != other.qpsShape) { + 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/com/google/cloud/dataflow/integration/nexmark/NexmarkDriver.java b/integration/java/src/main/java/com/google/cloud/dataflow/integration/nexmark/NexmarkDriver.java new file mode 100644 index 000000000000..e2289a6c838c --- /dev/null +++ b/integration/java/src/main/java/com/google/cloud/dataflow/integration/nexmark/NexmarkDriver.java @@ -0,0 +1,625 @@ +/* + * Copyright (C) 2015 Google Inc. + * + * Licensed under the Apache License, Version 2.0 (the "License"); you may not + * use this file except in compliance with the License. You may obtain a copy of + * the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT + * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the + * License for the specific language governing permissions and limitations under + * the License. + */ + +package com.google.cloud.dataflow.integration.nexmark; + +import com.google.cloud.dataflow.sdk.options.DataflowPipelineOptions; +import com.google.cloud.dataflow.sdk.options.Default; +import com.google.cloud.dataflow.sdk.options.Description; +import com.google.cloud.dataflow.sdk.options.PipelineOptionsFactory; +import com.google.common.collect.Sets; + +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.' These are 8 queries in Oracle's 'Continuous Query + * Language' (CQL) 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 NexmarkDriver { + /** + * Command line flags. + */ + public interface Options extends DataflowPipelineOptions { + @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("Avro input file pattern; only valid if source type is avro") + @Nullable + String getInputFilePrefix(); + + void setInputFilePrefix(String filepattern); + + @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("Approximate number of events to generate." + + "Zero for effectively unlimited in streaming mode.") + @Nullable + Long getNumEvents(); + + void setNumEvents(Long numEvents); + + @Description("Number of events to generate at the special pre-load rate. This is useful " + + "for generating a backlog of events on pub/sub before the main query begins.") + @Nullable + Long getNumPreloadEvents(); + + void setNumPreloadEvents(Long numPreloadEvents); + + @Description("Number of unbounded sources to create events.") + @Nullable + Integer getNumEventGenerators(); + + void setNumEventGenerators(Integer numEventGenerators); + + @Description("Shape of event qps curve.") + @Nullable + NexmarkUtils.QpsShape getQpsShape(); + + void setQpsShape(NexmarkUtils.QpsShape qpsShape); + + @Description("Initial overall event qps.") + @Nullable + Integer getFirstEventQps(); + + void setFirstEventQps(Integer firstEventQps); + + @Description("Next overall event qps.") + @Nullable + Integer getNextEventQps(); + + void setNextEventQps(Integer nextEventQps); + + @Description("Overall period of qps shape, in seconds.") + @Nullable + Integer getQpsPeriodSec(); + + void setQpsPeriodSec(Integer qpsPeriodSec); + + @Description("Overall event qps while pre-loading. " + + "Typcially as large as possible for given pub/sub quota.") + @Nullable + Integer getPreloadEventQps(); + + void setPreloadEventQps(Integer preloadEventQps); + + @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 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("If true, make sure all topics, subscriptions and gcs filenames are unique.") + @Default.Boolean(true) + boolean getUniqify(); + + void setUniqify(boolean uniqify); + + @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 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("If false, do not add the Monitor and Snoop transforms.") + @Nullable + Boolean getDebug(); + void setDebug(Boolean value); + } + + /** + * Entry point. + * + * @throws IOException + * @throws InterruptedException + */ + public static void main(String[] args) throws IOException, InterruptedException { + // Gather command line args, baseline, configurations, etc. + Options options = PipelineOptionsFactory.fromArgs(args).withValidation().as(Options.class); + NexmarkRunner runner = + new NexmarkRunner(options, options.getOutputPath(), options.getPubsubTopic(), + options.getMonitorJobs(), options.getUniqify(), options.getManageResources()); + Instant start = Instant.now(); + Map baseline = loadBaseline(options.getBaselineFilename()); + Map actual = new LinkedHashMap<>(); + Iterable configurations = + Sets.newLinkedHashSet(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); + } + } + + System.exit(successful ? 0 : 1); + } + + /** + * Append the pair of {@code configuration} and {@code perf} to perf file. + * + * @throws IOException + */ + private static void appendPerf(@Nullable String perfFilename, NexmarkConfiguration configuration, + NexmarkPerf perf) throws IOException { + 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()); + Files.write(Paths.get(perfFilename), lines, StandardCharsets.UTF_8, StandardOpenOption.CREATE, + StandardOpenOption.APPEND); + NexmarkUtils.console(null, "appended results to perf file %s.", perfFilename); + } + + /** + * Load the baseline perf. + * + * @throws IOException + */ + @Nullable + private static Map loadBaseline( + @Nullable String baselineFilename) throws IOException { + if (baselineFilename == null) { + return null; + } + Map baseline = new LinkedHashMap<>(); + List lines = Files.readAllLines(Paths.get(baselineFilename), StandardCharsets.UTF_8); + 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( + null, "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) throws IOException { + 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("NexmarkRunner 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) { + Files.write(Paths.get(summaryFilename), lines, StandardCharsets.UTF_8, + StandardOpenOption.CREATE, StandardOpenOption.APPEND); + NexmarkUtils.console(null, "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) throws IOException { + 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("];"); + + Files.write(Paths.get(javascriptFilename), lines, StandardCharsets.UTF_8, + StandardOpenOption.CREATE, StandardOpenOption.TRUNCATE_EXISTING); + NexmarkUtils.console(null, "saved javascript to file %s.", javascriptFilename); + } +} + diff --git a/integration/java/src/main/java/com/google/cloud/dataflow/integration/nexmark/NexmarkPerf.java b/integration/java/src/main/java/com/google/cloud/dataflow/integration/nexmark/NexmarkPerf.java new file mode 100644 index 000000000000..fe74f65f7118 --- /dev/null +++ b/integration/java/src/main/java/com/google/cloud/dataflow/integration/nexmark/NexmarkPerf.java @@ -0,0 +1,206 @@ +/* + * Copyright (C) 2015 Google Inc. + * + * Licensed under the Apache License, Version 2.0 (the "License"); you may not + * use this file except in compliance with the License. You may obtain a copy of + * the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT + * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the + * License for the specific language governing permissions and limitations under + * the License. + */ + +package com.google.cloud.dataflow.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) throws IOException { + return NexmarkUtils.MAPPER.readValue(string, NexmarkPerf.class); + } + + /** + * 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/com/google/cloud/dataflow/integration/nexmark/NexmarkQuery.java b/integration/java/src/main/java/com/google/cloud/dataflow/integration/nexmark/NexmarkQuery.java new file mode 100644 index 000000000000..dc0f69c73c7c --- /dev/null +++ b/integration/java/src/main/java/com/google/cloud/dataflow/integration/nexmark/NexmarkQuery.java @@ -0,0 +1,271 @@ +/* + * Copyright (C) 2015 Google Inc. + * + * Licensed under the Apache License, Version 2.0 (the "License"); you may not + * use this file except in compliance with the License. You may obtain a copy of + * the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT + * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the + * License for the specific language governing permissions and limitations under + * the License. + */ + +package com.google.cloud.dataflow.integration.nexmark; + +import com.google.cloud.dataflow.sdk.transforms.Aggregator; +import com.google.cloud.dataflow.sdk.transforms.DoFn; +import com.google.cloud.dataflow.sdk.transforms.Filter; +import com.google.cloud.dataflow.sdk.transforms.PTransform; +import com.google.cloud.dataflow.sdk.transforms.ParDo; +import com.google.cloud.dataflow.sdk.transforms.SerializableFunction; +import com.google.cloud.dataflow.sdk.values.KV; +import com.google.cloud.dataflow.sdk.values.PCollection; +import com.google.cloud.dataflow.sdk.values.TimestampedValue; +import com.google.cloud.dataflow.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); + } + }); + + public static final DoFn EVENT_TIMESTAMP_FROM_DATA = 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/com/google/cloud/dataflow/integration/nexmark/NexmarkQueryModel.java b/integration/java/src/main/java/com/google/cloud/dataflow/integration/nexmark/NexmarkQueryModel.java new file mode 100644 index 000000000000..7484672bd887 --- /dev/null +++ b/integration/java/src/main/java/com/google/cloud/dataflow/integration/nexmark/NexmarkQueryModel.java @@ -0,0 +1,121 @@ +/* + * Copyright (C) 2015 Google Inc. + * + * Licensed under the Apache License, Version 2.0 (the "License"); you may not + * use this file except in compliance with the License. You may obtain a copy of + * the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT + * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the + * License for the specific language governing permissions and limitations under + * the License. + */ + +package com.google.cloud.dataflow.integration.nexmark; + +import com.google.cloud.dataflow.sdk.transforms.SerializableFunction; +import com.google.cloud.dataflow.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/com/google/cloud/dataflow/integration/nexmark/NexmarkRunner.java b/integration/java/src/main/java/com/google/cloud/dataflow/integration/nexmark/NexmarkRunner.java new file mode 100644 index 000000000000..bcda95d1fa4e --- /dev/null +++ b/integration/java/src/main/java/com/google/cloud/dataflow/integration/nexmark/NexmarkRunner.java @@ -0,0 +1,1146 @@ +/* + * Copyright (C) 2015 Google Inc. + * + * Licensed under the Apache License, Version 2.0 (the "License"); you may not + * use this file except in compliance with the License. You may obtain a copy of + * the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT + * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the + * License for the specific language governing permissions and limitations under + * the License. + */ + +package com.google.cloud.dataflow.integration.nexmark; + +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.api.services.dataflow.model.JobMetrics; +import com.google.api.services.dataflow.model.MetricUpdate; +import com.google.cloud.dataflow.integration.nexmark.NexmarkUtils.PubSubMode; +import com.google.cloud.dataflow.sdk.Pipeline; +import com.google.cloud.dataflow.sdk.PipelineResult; +import com.google.cloud.dataflow.sdk.io.AvroIO; +import com.google.cloud.dataflow.sdk.io.BigQueryIO; +import com.google.cloud.dataflow.sdk.io.PubsubIO; +import com.google.cloud.dataflow.sdk.io.TextIO; +import com.google.cloud.dataflow.sdk.runners.AggregatorRetrievalException; +import com.google.cloud.dataflow.sdk.runners.BlockingDataflowPipelineRunner; +import com.google.cloud.dataflow.sdk.runners.DataflowPipelineJob; +import com.google.cloud.dataflow.sdk.runners.DataflowPipelineRunner; +import com.google.cloud.dataflow.sdk.runners.DirectPipelineRunner; +import com.google.cloud.dataflow.sdk.testing.DataflowAssert; +import com.google.cloud.dataflow.sdk.transforms.Aggregator; +import com.google.cloud.dataflow.sdk.transforms.DoFn; +import com.google.cloud.dataflow.sdk.transforms.ParDo; +import com.google.cloud.dataflow.sdk.values.PCollection; +import com.google.cloud.dataflow.sdk.values.PCollection.IsBounded; +import com.google.cloud.dataflow.sdk.values.TimestampedValue; +import com.google.common.base.Preconditions; +import com.google.common.collect.Iterables; +import com.google.common.collect.Lists; + +import org.joda.time.Duration; +import org.joda.time.Instant; + +import java.io.IOException; +import java.io.OutputStream; +import java.io.PrintStream; +import java.math.BigDecimal; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collection; +import java.util.Collections; +import java.util.List; +import java.util.Random; +import java.util.concurrent.TimeUnit; + +import javax.annotation.Nullable; + +/** + * Run a query according to specific set of options. + */ +class 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); + + // Following is valid over all runs. + + private final NexmarkDriver.Options options; + + @Nullable + private final String outputPath; + + private final String pubsubTopic; + + private final boolean monitorJobs; + + // Following is valid per-run only. + + /** Which configuration should we run. */ + @Nullable + private NexmarkConfiguration configuration; + + /** + * Accumulate the pub/sub subscriptions etc which should be cleaned up on end of run. + */ + @Nullable + private PubsubHelper pubsub; + + /** Pipeline 'result' for the publishing pipeline if in pub/sub COMBINED mode. */ + @Nullable + private PipelineResult publisherResult; + + /** Result for the main query pipeline. */ + @Nullable + private PipelineResult mainResult; + + /** Monitor for published events if in pub/sub COMBINED mode. */ + @Nullable + private Monitor publisherMonitor; + + /** Query name. */ + @Nullable + private String queryName; + + /** If sending events via pub/sub, the full topic name to use. */ + @Nullable + private String inputTopic; + + /** If true, make sure all topic, subscription and gcs file names are unique. */ + private final boolean uniqify; + + /** If true, manage the creation and cleanup of topics, subscriptions and gcs files. */ + private final boolean manageResources; + + public NexmarkRunner(NexmarkDriver.Options options, @Nullable String outputPath, + String pubsubTopic, boolean monitorJobs, boolean uniqify, boolean manageResources) { + this.options = options; + this.outputPath = outputPath != null && outputPath.isEmpty() ? null : outputPath; + this.pubsubTopic = pubsubTopic != null && pubsubTopic.isEmpty() ? null : pubsubTopic; + this.monitorJobs = monitorJobs; + this.uniqify = uniqify; + this.manageResources = manageResources; + } + + /** + * Return a pub/sub helper. + * + * @throws IOException + */ + private PubsubHelper getPubsub() throws IOException { + if (pubsub == null) { + pubsub = PubsubHelper.create(options); + } + return pubsub; + } + + /** + * Return a source of synthetic events. + */ + private PCollection createSyntheticSource(Pipeline p) { + if (p.getRunner() instanceof DirectPipelineRunner || !options.isStreaming()) { + return p.apply(NexmarkUtils.batchEventsSource(queryName, configuration)); + } else { + return p.apply(NexmarkUtils.streamEventsSource(queryName, configuration)); + } + } + + /** + * Return pub/sub sink. + */ + private PubsubIO.Write.Bound pubsubEventSink() { + PubsubIO.Write.Bound io = + PubsubIO.Write.named(queryName + ".Write(" + inputTopic + ")") + .topic(inputTopic) + .idLabel(NexmarkUtils.PUBSUB_ID) + .withCoder(Event.CODER) + .named(queryName + ".PubsubSourceWrite"); + if (!configuration.usePubsubPublishTime) { + io = io.timestampLabel(NexmarkUtils.PUBSUB_TIMESTAMP); + } + return io; + } + + /** + * Return pub/sub source. + */ + private PubsubIO.Read.Bound pubsubEventSource(String subscription) { + PubsubIO.Read.Bound io = + PubsubIO.Read.named(queryName + ".Read(" + subscription + ")") + .subscription(subscription) + .idLabel(NexmarkUtils.PUBSUB_ID) + .withCoder(Event.CODER) + .named(queryName + ".PubsubSourceRead"); + if (!configuration.usePubsubPublishTime) { + io = io.timestampLabel(NexmarkUtils.PUBSUB_TIMESTAMP); + } + return io; + } + + /** + * Return pub/sub sink for results. + */ + private PubsubIO.Write.Bound pubsubResultSink(String topic) { + PubsubIO.Write.Bound io = + PubsubIO.Write.named(queryName + ".Write(" + topic + ")") + .topic(topic) + .idLabel(NexmarkUtils.PUBSUB_ID) + .named(queryName + ".PubsubSinkWrite"); + if (!configuration.usePubsubPublishTime) { + io = io.timestampLabel(NexmarkUtils.PUBSUB_TIMESTAMP); + } + return io; + } + + /** + * Return source of events for this run, or null if we are simply publishing events + * to pub/sub. + * + * @throws IOException + * @throws InterruptedException + */ + private PCollection createSource(Pipeline p, long now) + throws IOException, InterruptedException { + PCollection source = null; + switch (configuration.sourceType) { + case DIRECT: + source = createSyntheticSource(p); + break; + case AVRO: + if (options.getInputFilePrefix() == null) { + throw new RuntimeException( + "If sourceType is AVRO, --inputFilePrefix must be specified."); + } + PCollection preTimestamp = p.apply(AvroIO.Read.named("ReadFromAvro") + .from(options.getInputFilePrefix() + "*.avro") + .withSchema(Event.class)); + source = preTimestamp.apply("adjust timestamp", + ParDo.of(NexmarkQuery.EVENT_TIMESTAMP_FROM_DATA)); + break; + case PUBSUB: + if (pubsubTopic == null) { + throw new RuntimeException("must supply a --pubsubTopic if using --sourceType=PUBSUB"); + } + + // Check some flags. + switch (configuration.pubSubMode) { + case SUBSCRIBE_ONLY: + break; + case PUBLISH_ONLY: + if (manageResources && options.getRunner() != BlockingDataflowPipelineRunner.class) { + throw new RuntimeException( + "If --manageResources=true and --pubSubMode=PUBLISH_ONLY then " + + "--runner=BlockingDataflowPipelineRunner so that this program " + + "can cleanup the pub/sub topic on exit."); + } + break; + case COMBINED: + if (options.getRunner() != DataflowPipelineRunner.class || !monitorJobs) { + throw new RuntimeException( + "if --pubSubMode=COMBINED then --runner=DataflowPipelineRunner and " + + "--monitorJobs=true so that the two pipelines can be managed."); + } + break; + } + + // Choose a topic name for events, and optionally topics to use to synchronize + // the publisher and subscriber jobs. + String shortTopic; + if (uniqify) { + // Salt the topic name so we can run multiple jobs in parallel. + shortTopic = String.format("%s_%d_%s_source", pubsubTopic, now, queryName); + } else { + shortTopic = String.format("%s_%s_source", pubsubTopic, queryName); + } + String shortSubscription = shortTopic; + + // Create/confirm the topic. + if (!manageResources || configuration.pubSubMode == PubSubMode.SUBSCRIBE_ONLY) { + // The topic should already have been created by the user or + // a companion 'PUBLISH_ONLY' process. + inputTopic = getPubsub().reuseTopic(shortTopic); + } else { + // Create a fresh topic to loopback via. It will be destroyed when the + // (necessarily blocking) job is done. + inputTopic = getPubsub().createTopic(shortTopic); + } + + // Create/confirm the subscription. + String subscription = null; + if (configuration.pubSubMode == PubSubMode.PUBLISH_ONLY) { + // Nothing to consume. + } else if (!manageResources) { + // The subscription should already have been created by the user. + subscription = getPubsub().reuseSubscription(shortTopic, shortSubscription); + } else { + subscription = getPubsub().createSubscription(shortTopic, shortSubscription); + } + + // Setup the sink for the publisher. + switch (configuration.pubSubMode) { + case SUBSCRIBE_ONLY: + // Nothing to publish. + break; + case PUBLISH_ONLY: + // Send synthesized events to pub/sub in this job. + createSyntheticSource(p).apply(NexmarkUtils.snoop(queryName)).apply(pubsubEventSink()); + break; + case COMBINED: + if (options.getRunner() != DataflowPipelineRunner.class || !monitorJobs) { + throw new RuntimeException( + "if --pubSubMode=COMBINED then you must use --runner=DataflowPipelineRunner " + + "and --monitorJobs=true so that the publisher job can be shutdown cleanly"); + } + // Send synthesized events to pub/sub 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. + String jobName = options.getJobName(); + String appName = options.getAppName(); + options.setJobName("p-" + jobName); + options.setAppName("p-" + appName); + int eventGeneratorWorkers = + (configuration.numEventGenerators + configuration.coresPerWorker() - 1) + / configuration.coresPerWorker(); + options.setMaxNumWorkers(Math.min(configuration.maxNumWorkers, eventGeneratorWorkers)); + options.setNumWorkers(Math.min(configuration.numWorkers, eventGeneratorWorkers)); + publisherMonitor = new Monitor(queryName, "publisher"); + Pipeline q = Pipeline.create(options); + createSyntheticSource(q) + .apply(publisherMonitor.getTransform()) + .apply(pubsubEventSink()); + PrintStream stdout = System.out; + try { + // Suppress output of publisher job; it makes the output of the command harder to + // interpret. + System.setOut(new PrintStream(new OutputStream() { + @Override + public void write(int b) {} + })); + publisherResult = q.run(); + } finally { + System.setOut(stdout); + } + System.out.println( + "Publisher job running as " + ((DataflowPipelineJob) publisherResult).getJobId()); + options.setJobName(jobName); + options.setAppName(appName); + options.setMaxNumWorkers(configuration.maxNumWorkers); + options.setNumWorkers(configuration.numWorkers); + waitForPublisherPreload(); + 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 = p.apply(pubsubEventSource(subscription)); + break; + } + break; + } + return source; + } + + /** + * Consume the given results. + * + * @throws IOException + * @throws InterruptedException + */ + private void createSink(PCollection> results, long now) + throws IOException, InterruptedException { + + if (NexmarkUtils.SinkType.COUNT_ONLY == configuration.sinkType) { + results.apply(NexmarkUtils.devNull(queryName)); + return; + } + PCollection formattedResults = results.apply(NexmarkUtils.format(queryName)); + + if (configuration.logResults) { + formattedResults = formattedResults.apply(NexmarkUtils.log(queryName)); + } + + switch (configuration.sinkType) { + case DEVNULL: + // Discard all results + formattedResults.apply(NexmarkUtils.devNull(queryName)); + break; + case PUBSUB: + // Publish results to pubsub. Don't bother consuming them, the published + // results will be discarded when we delete the 'sink' topic. + if (pubsubTopic == null) { + throw new RuntimeException("must supply a --pubsubTopic if using --sinkType=PUBSUB"); + } + String shortTopic; + if (uniqify) { + shortTopic = String.format("%s_%d_%s_sink", pubsubTopic, now, queryName); + } else { + shortTopic = String.format("%s_%s_sink", pubsubTopic, queryName); + } + String topic; + if (!manageResources) { + topic = getPubsub().reuseTopic(shortTopic); + } else { + topic = getPubsub().createTopic(shortTopic); + } + formattedResults.apply(pubsubResultSink(topic)); + break; + case TEXT: + // Write results to text. Only works in batch mode. + if (options.isStreaming()) { + throw new RuntimeException("can only use --sinkType=TEXT with --streaming=false"); + } + if (outputPath == null) { + throw new RuntimeException("must supply an --outputPath if using --sinkType=TEXT"); + } + String fullFilename; + if (uniqify) { + fullFilename = String.format("%s/nexmark_%d_%s.txt", outputPath, now, queryName); + } else { + fullFilename = String.format("%s/nexmark_%s.txt", outputPath, queryName); + } + formattedResults.apply(TextIO.Write.to(fullFilename).named(queryName + ".Text")); + break; + case AVRO: + NexmarkUtils.console(null, "WARNING: with --sinkType=AVRO, actual query results will be " + + "discarded.", outputPath); + break; + case BIGQUERY: + NexmarkUtils.console(null, "Writing events to BigQuery table %s", outputPath); + formattedResults.apply(ParDo.of(new StringToTableRow())) + .apply(BigQueryIO.Write.named("WriteBigQuery(Events)") + .to(options.getProject() + ":nexmark.table_" + + new Random().nextInt(Integer.MAX_VALUE)) + .withSchema(new TableSchema().setFields(new ArrayList() { + { add(new TableFieldSchema().setName("event").setType("STRING")); } + }))); + break; + case COUNT_ONLY: + // Short-circuited above. + throw new RuntimeException(); + } + } + + private static class StringToTableRow extends DoFn { + @Override + public void processElement(ProcessContext c) { + TableRow row = new TableRow(); + row.set("event", c.element()); + c.output(row); + } + } + + /** + * Sink all raw Events in {@code source} to {@code this.outputPath}. + * + * 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 sinkToAvro(final PCollection source) { + if (options.isStreaming()) { + throw new RuntimeException("can only use Avro SinkType with --streaming=false"); + } + if (outputPath == null) { + throw new RuntimeException("Must supply an --outputPath if using --sinkType=AVRO"); + } + NexmarkUtils.console(null, "Writing events in Avro to %s", outputPath); + source.apply(AvroIO.Write.named("WriteAvro(Events)") + .to(outputPath + "/event") + .withSuffix(".avro") + .withSchema(Event.class)); + source.apply(NexmarkQuery.JUST_BIDS) + .apply(AvroIO.Write.named("WriteAvro(Bids)") + .to(outputPath + "/bid") + .withSuffix(".avro") + .withSchema(Bid.class)); + source.apply(NexmarkQuery.JUST_NEW_AUCTIONS) + .apply(AvroIO.Write.named("WriteAvro(Auctions)") + .to(outputPath + "/auction") + .withSuffix(".avro") + .withSchema(Auction.class)); + source.apply(NexmarkQuery.JUST_NEW_PERSONS) + .apply(AvroIO.Write.named("WriteAvro(People)") + .to(outputPath + "/person") + .withSuffix(".avro") + .withSchema(Person.class)); + } + + /** + * Run {@code configuration} and return its performance (if using DataflowPipelineRunner + * and {@link #monitorJobs} is true). + * + * @throws IOException + * @throws InterruptedException + */ + @Nullable + public NexmarkPerf run(NexmarkConfiguration runConfiguration) + throws IOException, InterruptedException { + // + // Setup per-run state. + // + Preconditions.checkState(configuration == null); + Preconditions.checkState(pubsub == null); + Preconditions.checkState(publisherResult == null); + Preconditions.checkState(mainResult == null); + Preconditions.checkState(queryName == null); + configuration = runConfiguration; + + // GCS URI patterns to delete on exit. + List pathsToDelete = new ArrayList<>(); + + try { + NexmarkUtils.console(null, "running %s", configuration.toShortString()); + + if (configuration.numEvents < 0) { + NexmarkUtils.console(null, "skipping since configuration is disabled"); + return null; + } + + List queries = Arrays.asList(new NexmarkQuery[] {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)}); + NexmarkQuery query = queries.get(configuration.query); + queryName = query.getName(); + + List models = Arrays.asList( + new NexmarkQueryModel[] {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}); + NexmarkQueryModel model = models.get(configuration.query); + if (configuration.justModelResultRate) { + if (model == null) { + throw new RuntimeException(String.format("No model for %s", queryName)); + } + modelResultRates(model); + return null; + } + + // Copy configuration into option for the few parameters which are shared. + // (If these have been set on the command line then those values will have + // been copied into the configuration, and so these assignments will have no effect.) + options.setWorkerMachineType(configuration.workerMachineType); + options.setNumWorkers(configuration.numWorkers); + options.setMaxNumWorkers(configuration.maxNumWorkers); + options.setAutoscalingAlgorithm(configuration.autoscalingAlgorithm); + options.setStreaming(configuration.streaming); + options.setExperiments(configuration.experiments); + long now = System.currentTimeMillis(); + Pipeline p = Pipeline.create(options); + NexmarkUtils.setupPipeline(configuration.coderStrategy, p); + + // Generate events. + PCollection source = createSource(p, now); + + // 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) { + if (monitorJobs && options.getRunner() != DataflowPipelineRunner.class) { + throw new RuntimeException("can only use --monitorJobs=true if also " + + "using --runner=DataflowPipelineRunner"); + } + + // Optionally sink events in Avro format + if (configuration.sinkType == NexmarkUtils.SinkType.AVRO) { + this.sinkToAvro(source); + } + + // Special hacks for Query 10 (big logger). + if (configuration.query == 10) { + String path = null; + if (outputPath != null) { + if (uniqify) { + path = String.format("%s/%d_logs", outputPath, now); + } else { + path = String.format("%s/logs", outputPath); + } + } + ((Query10) query).setOutputPath(path); + if (path != null && manageResources) { + pathsToDelete.add(path + "/**"); + } + } + + // Apply query. + PCollection> results = source.apply(query); + + if (configuration.assertCorrectness) { + 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(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. + DataflowAssert.that(results).satisfies(model.assertionFor()); + } + + // Output results. + createSink(results, now); + } + + mainResult = p.run(); + return monitor(query); + } finally { + // + // Cleanup per-run state. + // + if (pubsub != null) { + // Delete any subscriptions and topics we created. + pubsub.cleanup(); + pubsub = null; + } + // TODO: Find a way to cleanup pathsToDelete robustly without depending on gsutil. + configuration = null; + publisherResult = null; + mainResult = null; + queryName = null; + } + } + + /** + * 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(null, "Query%d: only %d samples", model.configuration.query, n); + } else { + NexmarkUtils.console(null, "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)); + } + } + + /** + * Monitor the progress of the publisher job. Return when it has produced at + * least {@code configuration.numPreloadEvents}. + * + * @throws IOException + * @throws InterruptedException + */ + private void waitForPublisherPreload() throws IOException, InterruptedException { + if (!monitorJobs) { + return; + } + if (options.getRunner() != DataflowPipelineRunner.class) { + return; + } + if (!(publisherResult instanceof DataflowPipelineJob)) { + return; + } + if (configuration.numPreloadEvents <= 0) { + return; + } + + NexmarkUtils.console(null, "waiting for publisher to pre-load"); + + DataflowPipelineJob job = (DataflowPipelineJob) publisherResult; + + while (true) { + PipelineResult.State state = job.getState(); + long numEvents = getLong(job, publisherMonitor.getElementCounter()); + NexmarkUtils.console(null, "%s publisher (waiting for %d events, seen %d so far)", state, + configuration.numPreloadEvents, numEvents); + + if (numEvents >= 0 && numEvents >= configuration.numPreloadEvents) { + NexmarkUtils.console(null, "publisher preload done"); + return; + } + + switch (state) { + case UNKNOWN: + case STOPPED: + case RUNNING: + // Keep waiting. + break; + case DONE: + // All done. + NexmarkUtils.console(null, "publisher pipeline done"); + return; + case CANCELLED: + case FAILED: + case UPDATED: + // Something went wrong. + NexmarkUtils.console(null, "publisher pipeline failed", state, numEvents); + return; + } + + Thread.sleep(PERF_DELAY.getMillis()); + } + } + + /** + * Monitor the performance and progress of a running job. Return final performance if + * it was measured. + * + * @throws IOException + * @throws InterruptedException + */ + @Nullable + private NexmarkPerf monitor(NexmarkQuery query) throws IOException, InterruptedException { + if (!monitorJobs) { + return null; + } + if (options.getRunner() != DataflowPipelineRunner.class) { + return null; + } + if (!(mainResult instanceof DataflowPipelineJob)) { + return null; + } + if (!configuration.debug) { + return null; + } + + NexmarkUtils.console(null, "waiting for main pipeline to 'finish'"); + + DataflowPipelineJob job = (DataflowPipelineJob) mainResult; + DataflowPipelineJob publisherJob = (DataflowPipelineJob) publisherResult; + List snapshots = new ArrayList<>(); + Instant start = Instant.now(); + Instant end = + options.getRunningTimeMinutes() != null + ? start.plus(Duration.standardMinutes(options.getRunningTimeMinutes())) + : new Instant(Long.MAX_VALUE); + Instant lastActivity = null; + NexmarkPerf perf = null; + boolean waitingForShutdown = false; + boolean publisherCancelled = false; + List errors = new ArrayList<>(); + + while (true) { + Instant now = Instant.now(); + if (now.isAfter(end) && !waitingForShutdown) { + NexmarkUtils.console(null, "Reached end of test, cancelling job"); + job.cancel(); + if (publisherResult != null) { + publisherJob.cancel(); + publisherCancelled = true; + } + waitingForShutdown = true; + } + + PipelineResult.State state = job.getState(); + NexmarkUtils.console( + now, "%s %s%s", state, queryName, waitingForShutdown ? " (waiting for shutdown)" : ""); + + NexmarkPerf currPerf = + currentPerf(start, now, job, snapshots, query.eventMonitor, query.resultMonitor); + + if (perf == null || perf.anyActivity(currPerf)) { + lastActivity = now; + } + + if (configuration.streaming && !waitingForShutdown) { + Duration quietFor = new Duration(lastActivity, now); + if (query.getFatalCount() != null && getLong(job, query.getFatalCount()) > 0) { + NexmarkUtils.console(now, "job has fatal errors, cancelling."); + errors.add(String.format("Pipeline reported %s fatal errors", query.getFatalCount())); + waitingForShutdown = true; + } else if (configuration.numEvents > 0 && currPerf.numEvents == configuration.numEvents + && currPerf.numResults >= 0 && quietFor.isLongerThan(DONE_DELAY)) { + NexmarkUtils.console(now, "streaming query appears to have finished, cancelling job."); + waitingForShutdown = true; + } else if (quietFor.isLongerThan(STUCK_TERMINATE_DELAY)) { + NexmarkUtils.console( + now, "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( + now, "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, now.isAfter(start.plus(Duration.standardMinutes(5))))); + + if (waitingForShutdown) { + job.cancel(); + } + } + + 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 (lastActivity.equals(now)) { + NexmarkUtils.console(now, "new perf %s", perf); + } else { + NexmarkUtils.console(now, "no activity"); + } + + Thread.sleep(PERF_DELAY.getMillis()); + } + + perf.errors = errors; + perf.snapshots = snapshots; + NexmarkUtils.console(null, "final perf %s", perf); + + if (publisherResult != null) { + if (publisherCancelled) { + publisherJob.waitToFinish(5, TimeUnit.MINUTES, null); + } else { + publisherJob.cancel(); + } + } + + 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, boolean watermarksExpected) { + 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()); + + 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(null, msg); + } + } + if (!foundWatermarks) { + NexmarkUtils.console(null, "No known watermarks in update: " + metrics); + if (watermarksExpected) { + errors.add("No known watermarks found. Metrics were " + metrics); + } + } + } + } catch (IOException e) { + NexmarkUtils.console(null, "Warning: failed to get JobMetrics: " + e); + } + + return errors; + } + + /** + * Return the current performance given {@code eventMonitor} and {@code resultMonitor}. + */ + private NexmarkPerf currentPerf(Instant start, Instant 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 - start.getMillis()) / 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.getMillis() - 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.getMillis() - start.getMillis()) / 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 (!configuration.streaming) { + 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( + null, "%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(null, + "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( + null, "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( + Instant 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.getMillis()) > Duration.standardDays(10000).getMillis()) { + return -1; + } + return value; + } catch (AggregatorRetrievalException e) { + return -1; + } + } +} diff --git a/integration/java/src/main/java/com/google/cloud/dataflow/integration/nexmark/NexmarkSuite.java b/integration/java/src/main/java/com/google/cloud/dataflow/integration/nexmark/NexmarkSuite.java new file mode 100644 index 000000000000..346e6ecb6595 --- /dev/null +++ b/integration/java/src/main/java/com/google/cloud/dataflow/integration/nexmark/NexmarkSuite.java @@ -0,0 +1,372 @@ +/* + * Copyright (C) 2015 Google Inc. + * + * Licensed under the Apache License, Version 2.0 (the "License"); you may not + * use this file except in compliance with the License. You may obtain a copy of + * the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT + * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the + * License for the specific language governing permissions and limitations under + * the License. + */ + +package com.google.cloud.dataflow.integration.nexmark; + +import com.google.cloud.dataflow.integration.nexmark.NexmarkDriver.Options; +import com.google.cloud.dataflow.integration.nexmark.NexmarkUtils.SinkType; +import com.google.cloud.dataflow.integration.nexmark.NexmarkUtils.SourceType; +import com.google.cloud.dataflow.sdk.options.DataflowPipelineWorkerPoolOptions; + +import java.util.ArrayList; +import java.util.List; + +/** + * A set of {@link NexmarkConfiguration}s. + */ +public enum NexmarkSuite { + /** The default. isStreaming controlled from command line. */ + DEFAULT(true, defaultConf()), + + /** Sweep through all 10 queries with both streaming and batch using the default + * configuration. That means one worker, 100k/10k events (depending on query). */ + SMOKE(false, smoke()), + + /** + * As for SMOKE, but with 10m/1m events and 5 workers. + */ + STRESS(false, stress()), + + /** + * As for SMOKE, but with 1b/100m events and 25 workers. + */ + FULL_THROTTLE(false, fullThrottle()), + + /** + * Assert correspondence of actual against model for all queries in both + * batch and streaming. Only 2000 events. + */ + CORRECT(false, correct()), + + /** + * Calculate expected distribution of number of results per minute for all queries. + * Does not run any pipelines. + */ + JUST_MODEL_RESULT_RATE(false, justModelResultRate()), + + /** + * Compare coders on query 0 in batch and streaming. 1m events and 1 worker. + */ + CODERS(false, coders()), + + /** + * Query 10. + */ + BIG_LOGGER(false, bigLogger()), + + /** + * Query 10, but slow and small for debugging. + */ + SMALL_LOGGER(false, smallLogger()), + + /** + * Query 10, at high volume with no autoscaling. + */ + LONG_RUNNING_LOGGER(false, longRunningLogger()), + + /** + * Query 11 in streaming for 100b events to check for progressive slowdown. + */ + LONG_RUNNING_SESSIONS(false, longRunningSessions()), + + /** + * High-volume BigQuery output. + **/ + LONG_RUNNING_BIGQUERY(false, longRunningBigQuery()); + + 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 <= 10; query++) { + for (int streaming = 0; streaming <= 1; streaming++) { + NexmarkConfiguration configuration = NexmarkConfiguration.DEFAULT.clone(); + configuration.query = query; + configuration.streaming = streaming > 0; + configuration.numEvents = 100_000; + if (query == 7) { + // ###################################################################### + // Currently broken. b/22932773. + // ###################################################################### + configuration.numEvents = -1; + } else 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; + } + configuration.numWorkers *= 5; + configuration.maxNumWorkers *= 5; + } + return configurations; + } + + private static List fullThrottle() { + List configurations = smoke(); + for (NexmarkConfiguration configuration : configurations) { + if (configuration.numEvents >= 0) { + configuration.numEvents *= 1000; + } + configuration.numWorkers *= 25; + configuration.maxNumWorkers *= 25; + } + return configurations; + } + + private static List correct() { + List configurations = smoke(); + for (NexmarkConfiguration configuration : configurations) { + if (configuration.numEvents >= 0) { + configuration.numEvents = 2_000L; + } + configuration.assertCorrectness = true; + configuration.logResults = true; + } + return configurations; + } + + private static List justModelResultRate() { + List configurations = new ArrayList<>(); + for (int query = 0; query <= 10; query++) { + if (query == 10) { + // No model for Query 10. + continue; + } + NexmarkConfiguration configuration = NexmarkConfiguration.DEFAULT.clone(); + configuration.query = query; + configuration.justModelResultRate = true; + configurations.add(configuration); + } + return configurations; + } + + private static List coders() { + List configurations = new ArrayList<>(); + for (int streaming = 0; streaming <= 1; streaming++) { + for (NexmarkUtils.CoderStrategy coderStrategy : NexmarkUtils.CoderStrategy.values()) { + NexmarkConfiguration configuration = NexmarkConfiguration.DEFAULT.clone(); + configuration.query = 0; + configuration.streaming = streaming > 0; + configuration.numEvents = 1_000_000L; + configuration.coderStrategy = coderStrategy; + configurations.add(configuration); + } + } + return configurations; + } + + private static List bigLogger() { + NexmarkConfiguration configuration = NexmarkConfiguration.DEFAULT.clone(); + int totalCores = 400; + int generatorCores = 24; + int queryCores = totalCores - generatorCores; + int maxDisksPerWorker = 15; + configuration.maxNumWorkers = queryCores / configuration.coresPerWorker(); + configuration.numWorkers = + Math.max(1, (configuration.maxNumWorkers + maxDisksPerWorker - 1) / maxDisksPerWorker); + configuration.numEventGenerators = generatorCores; + + configuration.query = 10; + configuration.streaming = true; + configuration.isRateLimited = true; + configuration.experiments.add("enable_ppm"); + configuration.experiments.add("enable_streaming_scaling"); + configuration.sourceType = SourceType.PUBSUB; + configuration.autoscalingAlgorithm = + DataflowPipelineWorkerPoolOptions.AutoscalingAlgorithmType.THROUGHPUT_BASED; + configuration.numEvents = 0; // as many as possible without overflow. + configuration.numPreloadEvents = 1_000_000L; + configuration.avgPersonByteSize = 500; + configuration.avgAuctionByteSize = 500; + configuration.avgBidByteSize = 500; + configuration.windowSizeSec = 60 * 20; + configuration.occasionalDelaySec = 60 * 60; + configuration.probDelayedEvent = 0.000001; + // Due to poor interaction of pub/sub and custom timestamps we need to make + // event timestamps as close as possible to wallclock time. + configuration.useWallclockEventTime = true; + configuration.qpsShape = NexmarkUtils.QpsShape.SINE; + configuration.firstEventQps = 100_000; + configuration.nextEventQps = 5_000; + configuration.qpsPeriodSec = 60 * 60; + configuration.preloadEventQps = 100_000; + // At 100k qps and 94 workers (ie 188 log shards) a pane will fill every 16 mins. + configuration.maxLogEvents = 500_000; + + configuration.usePubsubPublishTime = false; + + List configurations = new ArrayList<>(); + configurations.add(configuration); + return configurations; + } + + private static List smallLogger() { + NexmarkConfiguration configuration = NexmarkConfiguration.DEFAULT.clone(); + configuration.maxNumWorkers = 1; + configuration.numWorkers = 1; + configuration.numEventGenerators = 1; + + configuration.query = 10; + configuration.streaming = true; + configuration.isRateLimited = true; + configuration.experiments.add("enable_ppm"); + configuration.experiments.add("enable_streaming_scaling"); + configuration.sourceType = SourceType.PUBSUB; + configuration.autoscalingAlgorithm = + DataflowPipelineWorkerPoolOptions.AutoscalingAlgorithmType.NONE; + configuration.numEvents = 0; // as many as possible without overflow. + configuration.numPreloadEvents = 0L; + configuration.avgPersonByteSize = 500; + configuration.avgAuctionByteSize = 500; + configuration.avgBidByteSize = 500; + configuration.windowSizeSec = 30; + configuration.occasionalDelaySec = 360; + configuration.probDelayedEvent = 0.001; + configuration.useWallclockEventTime = true; + configuration.firstEventQps = 100; + configuration.nextEventQps = 100; + configuration.maxLogEvents = 15000; + + List configurations = new ArrayList<>(); + configurations.add(configuration); + return configurations; + } + + private static List longRunningSessions() { + NexmarkConfiguration configuration = NexmarkConfiguration.DEFAULT.clone(); + configuration.maxNumWorkers = 25; + configuration.numWorkers = 25; + configuration.numEventGenerators = 25; + + configuration.query = 11; + configuration.sourceType = SourceType.PUBSUB; + configuration.autoscalingAlgorithm = + DataflowPipelineWorkerPoolOptions.AutoscalingAlgorithmType.NONE; + configuration.streaming = true; + configuration.isRateLimited = true; + configuration.firstEventQps = 50_000; + configuration.nextEventQps = 25_000; + configuration.numEvents = 0L; + configuration.outOfOrderGroupSize = 10_000L; + configuration.occasionalDelaySec = 60; + configuration.probDelayedEvent = 0.00001; + configuration.numInFlightAuctions = 1000; + configuration.windowSizeSec = 5; // 5sec session gap. + configuration.maxLogEvents = 1000; // break sessions at around 1000 events. + + List configurations = new ArrayList<>(); + configurations.add(configuration); + return configurations; + } + + private static List longRunningLogger() { + NexmarkConfiguration configuration = NexmarkConfiguration.DEFAULT.clone(); + configuration.maxNumWorkers = 25; + configuration.numWorkers = 25; + configuration.numEventGenerators = 10; + + configuration.query = 10; + configuration.streaming = true; + configuration.isRateLimited = true; + configuration.sourceType = SourceType.PUBSUB; + configuration.numEvents = 0; // as many as possible without overflow. + configuration.numPreloadEvents = 0L; + configuration.avgPersonByteSize = 500; + configuration.avgAuctionByteSize = 500; + configuration.avgBidByteSize = 500; + configuration.windowSizeSec = 300; + configuration.occasionalDelaySec = 360; + configuration.probDelayedEvent = 0.001; + configuration.useWallclockEventTime = true; + configuration.firstEventQps = 30000; + configuration.nextEventQps = 30000; + configuration.maxLogEvents = 15000; + + List configurations = new ArrayList<>(); + configurations.add(configuration); + return configurations; + } + + private static List longRunningBigQuery() { + NexmarkConfiguration configuration = NexmarkConfiguration.DEFAULT.clone(); + configuration.maxNumWorkers = 25; + configuration.numWorkers = 25; + configuration.numEventGenerators = 10; + + configuration.query = 0; + configuration.streaming = true; + configuration.isRateLimited = true; + configuration.sourceType = SourceType.PUBSUB; + configuration.sinkType = SinkType.BIGQUERY; + configuration.numEvents = 0; // as many as possible without overflow. + configuration.numPreloadEvents = 0L; + configuration.avgPersonByteSize = 500; + configuration.avgAuctionByteSize = 500; + configuration.avgBidByteSize = 500; + configuration.windowSizeSec = 300; + configuration.probDelayedEvent = 0; + configuration.useWallclockEventTime = true; + configuration.firstEventQps = 60000; + configuration.nextEventQps = 60000; + + List configurations = new ArrayList<>(); + configurations.add(configuration); + return configurations; + } + + private final boolean takeIsStreamingFromCommandLine; + private final List configurations; + + private NexmarkSuite( + boolean takeIsStreamingFromCommandLine, List configurations) { + this.takeIsStreamingFromCommandLine = takeIsStreamingFromCommandLine; + 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) { + List results = new ArrayList<>(); + for (NexmarkConfiguration configuration : configurations) { + NexmarkConfiguration result = configuration.clone(); + if (takeIsStreamingFromCommandLine) { + // The --isStreaming flag is the ONLY one we can't distinguish between not-set + // and set to false. So only copy its value in a few special cases. + result.streaming = options.isStreaming(); + } + result.overrideFromOptions(options); + results.add(result); + } + return results; + } +} diff --git a/integration/java/src/main/java/com/google/cloud/dataflow/integration/nexmark/NexmarkUtils.java b/integration/java/src/main/java/com/google/cloud/dataflow/integration/nexmark/NexmarkUtils.java new file mode 100644 index 000000000000..524f1052d1ca --- /dev/null +++ b/integration/java/src/main/java/com/google/cloud/dataflow/integration/nexmark/NexmarkUtils.java @@ -0,0 +1,605 @@ +/* + * Copyright (C) 2015 Google Inc. + * + * Licensed under the Apache License, Version 2.0 (the "License"); you may not + * use this file except in compliance with the License. You may obtain a copy of + * the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT + * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the + * License for the specific language governing permissions and limitations under + * the License. + */ + +package com.google.cloud.dataflow.integration.nexmark; + +import com.google.cloud.dataflow.sdk.Pipeline; +import com.google.cloud.dataflow.sdk.coders.AvroCoder; +import com.google.cloud.dataflow.sdk.coders.ByteArrayCoder; +import com.google.cloud.dataflow.sdk.coders.Coder; +import com.google.cloud.dataflow.sdk.coders.CoderException; +import com.google.cloud.dataflow.sdk.coders.CoderRegistry; +import com.google.cloud.dataflow.sdk.coders.CustomCoder; +import com.google.cloud.dataflow.sdk.coders.SerializableCoder; +import com.google.cloud.dataflow.sdk.io.Read; +import com.google.cloud.dataflow.sdk.runners.DirectPipelineRunner; +import com.google.cloud.dataflow.sdk.runners.PipelineRunner; +import com.google.cloud.dataflow.sdk.transforms.Aggregator; +import com.google.cloud.dataflow.sdk.transforms.Combine; +import com.google.cloud.dataflow.sdk.transforms.DoFn; +import com.google.cloud.dataflow.sdk.transforms.PTransform; +import com.google.cloud.dataflow.sdk.transforms.ParDo; +import com.google.cloud.dataflow.sdk.transforms.Sum.SumLongFn; +import com.google.cloud.dataflow.sdk.transforms.windowing.AfterPane; +import com.google.cloud.dataflow.sdk.transforms.windowing.BoundedWindow; +import com.google.cloud.dataflow.sdk.transforms.windowing.GlobalWindows; +import com.google.cloud.dataflow.sdk.transforms.windowing.Window; +import com.google.cloud.dataflow.sdk.util.state.StateNamespaces; +import com.google.cloud.dataflow.sdk.util.state.StateTag; +import com.google.cloud.dataflow.sdk.util.state.StateTags; +import com.google.cloud.dataflow.sdk.util.state.ValueState; +import com.google.cloud.dataflow.sdk.values.PCollection; +import com.google.cloud.dataflow.sdk.values.PInput; +import com.google.cloud.dataflow.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; + +import javax.annotation.Nullable; + +/** + * Odd's 'n Ends used throughout queries and driver. + */ +public class NexmarkUtils { + private static final Logger LOG = LoggerFactory.getLogger(NexmarkDriver.class.getName()); + + /** Mapper for (de)serializing JSON. */ + static final ObjectMapper MAPPER = new ObjectMapper(); + + /** Possible sources for events. */ + public static 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 static 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 static 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 static enum CoderStrategy { + /** Hand-written. */ + HAND, + /** Avro. */ + AVRO, + /** Java serialization. */ + JAVA + } + + /** Shape of event qps. */ + public static enum QpsShape { + 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 qps} using {@code numGenerators}. + */ + public long interEventDelayUs(int qps, int numGenerators) { + return (1_000_000L * numGenerators) / qps; + } + + /** + * Return array of successive inter-event delays, in microseconds, for each generator + * to follow in order to achieve this shape with {@code firstQps/nextQps} using + * {@code numGenerators}. + */ + public long[] interEventDelayUs(int firstQps, int nextQps, int numGenerators) { + if (firstQps == nextQps) { + long[] interEventDelayUs = new long[1]; + interEventDelayUs[0] = (1_000_000L * numGenerators) / firstQps; + return interEventDelayUs; + } + + switch (this) { + case SQUARE: + { + long[] interEventDelayUs = new long[2]; + interEventDelayUs[0] = (1_000_000L * numGenerators) / firstQps; + interEventDelayUs[1] = (1_000_000L * numGenerators) / nextQps; + return interEventDelayUs; + } + case SINE: + { + double mid = (firstQps + nextQps) / 2.0; + double amp = (firstQps - nextQps) / 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 qps = mid + amp * Math.cos(r); + interEventDelayUs[i] = (1_000_000L * numGenerators) / Math.round(qps); + } + 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 qpsPeriodSec}. + */ + public int stepLengthSec(int qpsPeriodSec) { + int n = 0; + switch (this) { + case SQUARE: + n = 2; + break; + case SINE: + n = N; + break; + } + return (qpsPeriodSec + 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(@Nullable Instant now, String format, Object... args) { + if (now == null) { + now = Instant.now(); + } + System.out.printf("%s %s\n", 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) { + GeneratorConfig config = new GeneratorConfig( + configuration, BASE_TIME, 0, configuration.numEvents, configuration.numPreloadEvents, 0); + if (configuration.useWallclockEventTime) { + // We want events to have a timestamp matching wallclock time. + long now = System.currentTimeMillis(); + // How long, in wallclock time, do we expect it to take to pre-load events? + long preloadInterEventDelayUs = QpsShape.SQUARE.interEventDelayUs( + configuration.preloadEventQps, configuration.numEventGenerators); + long wallclockPreloadMs = + (configuration.numPreloadEvents * preloadInterEventDelayUs) + / (1000L * configuration.numEventGenerators); + // How long, in event time, does the event stream take to generate the pre-load events? + long firstPreloadEventTime = config.timestampAndInterEventDelayUsForEvent(0).getKey(); + long firstPostloadEventTime = + config.timestampAndInterEventDelayUsForEvent( + configuration.numPreloadEvents / configuration.numEventGenerators).getKey(); + long eventPreloadMs = firstPostloadEventTime - firstPreloadEventTime; + // Shift base time s.t. firstPostloadEventTime = now + wallclockPreLoadMs + config = new GeneratorConfig(configuration, now + wallclockPreloadMs - eventPreloadMs, 0, + configuration.numEvents, configuration.numPreloadEvents, 0); + } + return config; + } + + /** + * 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 result: %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("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/com/google/cloud/dataflow/integration/nexmark/Person.java b/integration/java/src/main/java/com/google/cloud/dataflow/integration/nexmark/Person.java new file mode 100644 index 000000000000..89ffe67e8c4e --- /dev/null +++ b/integration/java/src/main/java/com/google/cloud/dataflow/integration/nexmark/Person.java @@ -0,0 +1,164 @@ +/* + * Copyright (C) 2015 Google Inc. + * + * Licensed under the Apache License, Version 2.0 (the "License"); you may not + * use this file except in compliance with the License. You may obtain a copy of + * the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT + * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the + * License for the specific language governing permissions and limitations under + * the License. + */ + +package com.google.cloud.dataflow.integration.nexmark; + +import com.google.cloud.dataflow.sdk.coders.AtomicCoder; +import com.google.cloud.dataflow.sdk.coders.Coder; +import com.google.cloud.dataflow.sdk.coders.CoderException; +import com.google.cloud.dataflow.sdk.coders.StringUtf8Coder; +import com.google.cloud.dataflow.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, + com.google.cloud.dataflow.sdk.coders.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, com.google.cloud.dataflow.sdk.coders.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/com/google/cloud/dataflow/integration/nexmark/PubsubHelper.java b/integration/java/src/main/java/com/google/cloud/dataflow/integration/nexmark/PubsubHelper.java new file mode 100644 index 000000000000..a6518d336ce6 --- /dev/null +++ b/integration/java/src/main/java/com/google/cloud/dataflow/integration/nexmark/PubsubHelper.java @@ -0,0 +1,237 @@ +/* + * Copyright (C) 2015 Google Inc. + * + * Licensed under the Apache License, Version 2.0 (the "License"); you may not + * use this file except in compliance with the License. You may obtain a copy of + * the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT + * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the + * License for the specific language governing permissions and limitations under + * the License. + */ + +package com.google.cloud.dataflow.integration.nexmark; + +import com.google.api.client.googleapis.json.GoogleJsonResponseException; +import com.google.api.client.util.BackOff; +import com.google.api.client.util.BackOffUtils; +import com.google.api.client.util.Sleeper; +import com.google.api.services.pubsub.Pubsub; +import com.google.api.services.pubsub.model.Subscription; +import com.google.api.services.pubsub.model.Topic; +import com.google.cloud.dataflow.sdk.util.AttemptBoundedExponentialBackOff; +import com.google.cloud.dataflow.sdk.util.Transport; +import java.io.IOException; +import java.util.ArrayList; +import java.util.List; + +/** + * Helper for working with pubsub and gcs. + */ +public class PubsubHelper { + /** Underlying pub/sub client. */ + private final Pubsub pubsubClient; + + /** Project id. */ + private final String project; + + /** Topics we should delete on close. */ + private final List createdTopics; + + /** Subscriptions we should delete on close. */ + private final List createdSubscriptions; + + /** How to sleep in retry loops. */ + private final Sleeper sleeper; + + /** How to backoff in retry loops. */ + private final BackOff backOff; + + private PubsubHelper(Pubsub pubsubClient, String project) { + this.pubsubClient = pubsubClient; + this.project = project; + createdTopics = new ArrayList<>(); + createdSubscriptions = new ArrayList<>(); + sleeper = Sleeper.DEFAULT; + backOff = new AttemptBoundedExponentialBackOff(3, 500); + } + + /** Create a helper. */ + public static PubsubHelper create(NexmarkDriver.Options options) throws IOException { + return new PubsubHelper(Transport.newPubsubClient(options).build(), + options.getProject()); + } + + /** Return full topic name corresponding to short topic name. */ + private String fullTopic(String shortTopic) { + return String.format("projects/%s/topics/%s", project, shortTopic); + } + + /** Return full subscription name corresponding to short subscription name. */ + private String fullSubscription(String shortSubscription) { + return String.format("projects/%s/subscriptions/%s", project, shortSubscription); + } + + /** + * Create a topic from short name. Delete it if it already exists. Ensure the topic will be + * deleted on cleanup. Return full topic name. + * @throws InterruptedException + */ + public String createTopic(String shortTopic) throws IOException, InterruptedException { + String topic = fullTopic(shortTopic); + while (true) { + try { + NexmarkUtils.console(null, "create topic %s", topic); + pubsubClient.projects().topics().create(topic, new Topic()).execute(); + createdTopics.add(topic); + return topic; + } catch (GoogleJsonResponseException ex) { + NexmarkUtils.console(null, "attempting to cleanup topic %s", topic); + pubsubClient.projects().topics().delete(topic).execute(); + if (!BackOffUtils.next(sleeper, backOff)) { + NexmarkUtils.console(null, "too many retries for creating topic %s", topic); + throw ex; + } + } + } + } + + /** Create a topic from short name if it does not already exist. The topic will not be + * deleted on cleanup. Return full topic name. + * @throws InterruptedException */ + public String createOrReuseTopic(String shortTopic) throws IOException, InterruptedException { + String topic = fullTopic(shortTopic); + while (true) { + try { + NexmarkUtils.console(null, "create topic %s", topic); + pubsubClient.projects().topics().create(topic, new Topic()).execute(); + return topic; + } catch (GoogleJsonResponseException ex) { + if (topicExists(shortTopic)) { + NexmarkUtils.console(null, "topic %s already exists", topic); + return topic; + } + if (!BackOffUtils.next(sleeper, backOff)) { + NexmarkUtils.console(null, "too many retries for creating/reusing topic %s", topic); + throw ex; + } + } + } + } + + /** + * 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 String reuseTopic(String shortTopic) throws IOException { + String topic = fullTopic(shortTopic); + if (topicExists(shortTopic)) { + NexmarkUtils.console(null, "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) throws IOException { + String topic = fullTopic(shortTopic); + List existingTopics = + pubsubClient.projects().topics().list("projects/" + project).execute().getTopics(); + if (existingTopics != null) { + for (Topic existingTopic : existingTopics) { + if (existingTopic.getName().equals(topic)) { + return true; + } + } + } + return false; + } + + /** + * Create subscription from short name. Ensure the subscription will be deleted + * on cleanup. Return full subscription name. + * @throws InterruptedException + */ + public String createSubscription(String shortTopic, String shortSubscription) + throws IOException, InterruptedException { + String topic = fullTopic(shortTopic); + String subscription = fullSubscription(shortSubscription); + while (true) { + try { + NexmarkUtils.console(null, "create subscription %s", subscription); + pubsubClient.projects() + .subscriptions() + .create(subscription, new Subscription().setTopic(topic).setAckDeadlineSeconds(60)) + .execute(); + createdSubscriptions.add(subscription); + return subscription; + } catch (GoogleJsonResponseException ex) { + NexmarkUtils.console(null, "attempting to cleanup subscription %s", subscription); + pubsubClient.projects().subscriptions().delete(subscription).execute(); + if (!BackOffUtils.next(sleeper, backOff)) { + NexmarkUtils.console(null, "too many retries for creating subscription %s", subscription); + throw ex; + } + } + } + } + + /** + * 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 String reuseSubscription(String shortTopic, String shortSubscription) throws IOException { + String subscription = fullSubscription(shortSubscription); + if (subscriptionExists(shortTopic, shortSubscription)) { + NexmarkUtils.console(null, "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) + throws IOException { + String topic = fullTopic(shortTopic); + String subscription = fullSubscription(shortSubscription); + List existingSubscriptions = + pubsubClient.projects() + .subscriptions() + .list("projects/" + project) + .execute() + .getSubscriptions(); + if (existingSubscriptions != null) { + for (Subscription existingSubscription : existingSubscriptions) { + if (existingSubscription.getTopic().equals(topic) + && existingSubscription.getName().equals(subscription)) { + return true; + } + } + } + return false; + } + + /** Delete all the subscriptions and topics we created. */ + public void cleanup() { + for (String subscription : createdSubscriptions) { + try { + NexmarkUtils.console(null, "delete subscription %s", subscription); + pubsubClient.projects().subscriptions().delete(subscription).execute(); + } catch (IOException ex) { + NexmarkUtils.console(null, "could not delete subscription %s", subscription); + } + } + for (String topic : createdTopics) { + try { + NexmarkUtils.console(null, "delete topic %s", topic); + pubsubClient.projects().topics().delete(topic).execute(); + } catch (IOException ex) { + NexmarkUtils.console(null, "could not delete topic %s", topic); + } + } + } +} diff --git a/integration/java/src/main/java/com/google/cloud/dataflow/integration/nexmark/Query0.java b/integration/java/src/main/java/com/google/cloud/dataflow/integration/nexmark/Query0.java new file mode 100644 index 000000000000..90f97c9807ac --- /dev/null +++ b/integration/java/src/main/java/com/google/cloud/dataflow/integration/nexmark/Query0.java @@ -0,0 +1,70 @@ +/* + * Copyright (C) 2015 Google Inc. + * + * Licensed under the Apache License, Version 2.0 (the "License"); you may not + * use this file except in compliance with the License. You may obtain a copy of + * the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT + * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the + * License for the specific language governing permissions and limitations under + * the License. + */ + +package com.google.cloud.dataflow.integration.nexmark; + +import com.google.cloud.dataflow.sdk.coders.Coder; +import com.google.cloud.dataflow.sdk.coders.CoderException; +import com.google.cloud.dataflow.sdk.transforms.Aggregator; +import com.google.cloud.dataflow.sdk.transforms.DoFn; +import com.google.cloud.dataflow.sdk.transforms.ParDo; +import com.google.cloud.dataflow.sdk.transforms.Sum.SumLongFn; +import com.google.cloud.dataflow.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/com/google/cloud/dataflow/integration/nexmark/Query0Model.java b/integration/java/src/main/java/com/google/cloud/dataflow/integration/nexmark/Query0Model.java new file mode 100644 index 000000000000..2afe6b09351c --- /dev/null +++ b/integration/java/src/main/java/com/google/cloud/dataflow/integration/nexmark/Query0Model.java @@ -0,0 +1,60 @@ +/* + * Copyright (C) 2015 Google Inc. + * + * Licensed under the Apache License, Version 2.0 (the "License"); you may not + * use this file except in compliance with the License. You may obtain a copy of + * the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT + * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the + * License for the specific language governing permissions and limitations under + * the License. + */ + +package com.google.cloud.dataflow.integration.nexmark; + +import com.google.cloud.dataflow.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/com/google/cloud/dataflow/integration/nexmark/Query1.java b/integration/java/src/main/java/com/google/cloud/dataflow/integration/nexmark/Query1.java new file mode 100644 index 000000000000..b1a096a124d4 --- /dev/null +++ b/integration/java/src/main/java/com/google/cloud/dataflow/integration/nexmark/Query1.java @@ -0,0 +1,62 @@ +/* + * Copyright (C) 2015 Google Inc. + * + * Licensed under the Apache License, Version 2.0 (the "License"); you may not + * use this file except in compliance with the License. You may obtain a copy of + * the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT + * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the + * License for the specific language governing permissions and limitations under + * the License. + */ + +package com.google.cloud.dataflow.integration.nexmark; + +import com.google.cloud.dataflow.sdk.transforms.DoFn; +import com.google.cloud.dataflow.sdk.transforms.ParDo; +import com.google.cloud.dataflow.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/com/google/cloud/dataflow/integration/nexmark/Query10.java b/integration/java/src/main/java/com/google/cloud/dataflow/integration/nexmark/Query10.java new file mode 100644 index 000000000000..c0d17b7f2629 --- /dev/null +++ b/integration/java/src/main/java/com/google/cloud/dataflow/integration/nexmark/Query10.java @@ -0,0 +1,307 @@ +/* + * Copyright (C) 2015 Google Inc. + * + * Licensed under the Apache License, Version 2.0 (the "License"); you may not + * use this file except in compliance with the License. You may obtain a copy of + * the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT + * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the + * License for the specific language governing permissions and limitations under + * the License. + */ + +package com.google.cloud.dataflow.integration.nexmark; + +import com.google.cloud.dataflow.sdk.coders.Coder; +import com.google.cloud.dataflow.sdk.options.GcsOptions; +import com.google.cloud.dataflow.sdk.transforms.Aggregator; +import com.google.cloud.dataflow.sdk.transforms.DoFn; +import com.google.cloud.dataflow.sdk.transforms.DoFnWithContext; +import com.google.cloud.dataflow.sdk.transforms.GroupByKey; +import com.google.cloud.dataflow.sdk.transforms.Keys; +import com.google.cloud.dataflow.sdk.transforms.ParDo; +import com.google.cloud.dataflow.sdk.transforms.Sum.SumLongFn; +import com.google.cloud.dataflow.sdk.transforms.windowing.AfterEach; +import com.google.cloud.dataflow.sdk.transforms.windowing.AfterFirst; +import com.google.cloud.dataflow.sdk.transforms.windowing.AfterPane; +import com.google.cloud.dataflow.sdk.transforms.windowing.AfterProcessingTime; +import com.google.cloud.dataflow.sdk.transforms.windowing.AfterWatermark; +import com.google.cloud.dataflow.sdk.transforms.windowing.BoundedWindow; +import com.google.cloud.dataflow.sdk.transforms.windowing.FixedWindows; +import com.google.cloud.dataflow.sdk.transforms.windowing.PaneInfo; +import com.google.cloud.dataflow.sdk.transforms.windowing.PaneInfo.Timing; +import com.google.cloud.dataflow.sdk.transforms.windowing.Repeatedly; +import com.google.cloud.dataflow.sdk.transforms.windowing.Window; +import com.google.cloud.dataflow.sdk.util.GcsIOChannelFactory; +import com.google.cloud.dataflow.sdk.values.KV; +import com.google.cloud.dataflow.sdk.values.PCollection; +import com.google.cloud.hadoop.gcsio.GoogleCloudStorageWriteChannel; +import com.google.common.base.Preconditions; + +import org.joda.time.Duration; + +import java.io.IOException; +import java.io.OutputStream; +import java.nio.ByteBuffer; +import java.nio.channels.Channels; +import java.nio.channels.WritableByteChannel; +import javax.annotation.Nullable; + +/** + * Query "10", 'Log to sharded files' (Not in original suite.) + * + *

Every windowSizeSec, save all events from the last our 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 TEN_SECONDS = Duration.standardSeconds(10); + + /** + * GCS uri prefix for all log and 'finished' files. If null they won't be written. + */ + @Nullable + private String outputPath; + + public Query10(NexmarkConfiguration configuration) { + super(configuration, "Query10"); + } + + public void setOutputPath(@Nullable String outputPath) { + this.outputPath = outputPath; + } + + private WritableByteChannel openGcsFile(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; + } + + @Nullable + private String buildOutputPath( + String key, BoundedWindow window, PaneInfo.Timing timing, long paneIndex) { + if (outputPath == null) { + return null; + } + String which = null; + switch (timing) { + case EARLY: + which = "E"; + break; + case ON_TIME: + which = "O"; + break; + case LATE: + which = "L"; + break; + } + return String.format("%s/%s-%s-%03d%s", outputPath, window.maxTimestamp(), key, + paneIndex, which); + } + + @Nullable + private String buildFinalizeOutputPath(BoundedWindow window) { + if (outputPath == null) { + return null; + } + return String.format("%s/_SUCCESS-%s", outputPath, window.maxTimestamp()); + } + + private PCollection applyTyped(PCollection events) { + final int numLogShards = + Math.max(configuration.maxNumWorkers, configuration.numWorkers) * 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 shard = (int) Math.abs((long) c.element().hashCode() % numLogShards); + c.output(KV.of("dummy-" + 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 + (TEN_SECONDS))))) + .discardingFiredPanes() + // Use a 1 day allowed lateness so that any forgotten hold will stall the + // pipeline for that period and be very noticable. + .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 key = c.element().getKey(); + NexmarkUtils.error( + "key %s with timestamp %s has %d actually late and %d on-time " + + "elements in pane %s for window %s", + key, 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 key %s", key); + unexpectedLatePaneCounter.addValue(1L); + } + if (numOnTime > 0) { + NexmarkUtils.error( + "ERROR! Have %d on-time events in late pane for key %s", + numOnTime, key); + 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 key %s", + numOnTime + numLate, key); + } + 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 key = c.element().getKey(); + GcsOptions options = c.getPipelineOptions().as(GcsOptions.class); + NexmarkUtils.error( + "Writing key %s with record timestamp %s, window timestamp %s, pane %s", + key, c.timestamp(), window.maxTimestamp(), c.pane()); + String path = buildOutputPath(key, window, c.pane().getTiming(), c.pane() + .getIndex()); + if (path != null) { + NexmarkUtils.error("Beginning write for '%s'", path); + int n = 0; + try (OutputStream output = Channels.newOutputStream(openGcsFile(options, + path))) { + 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, path); + } + } + } + NexmarkUtils.error("Written all %d records to '%s'", n, path); + } + savedFileCounter.addValue(1L); + c.output(KV.of(null, null)); + } + })) + // 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(Keys.create()) + .apply( + ParDo.named(name + ".Finalize") + .of(new DoFnWithContext() { + 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("finalized", 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( + "Finalize with record timestamp %s, window timestamp %s, pane %s", + c.timestamp(), window.maxTimestamp(), c.pane()); + String path = buildFinalizeOutputPath(window); + if (path != null) { + NexmarkUtils.error("Beginning write for '%s'", path); + try (WritableByteChannel output = openGcsFile(options, path)) { + output.write(ByteBuffer.wrap("FINISHED".getBytes())); + } + NexmarkUtils.error("Written '%s'", path); + } + c.output( + new Done("written for timestamp " + window.maxTimestamp().getMillis())); + finalizedCounter.addValue(1L); + } + } + })); + } + + @Override + protected PCollection applyPrim(PCollection events) { + return NexmarkUtils.castToKnownSize(name, applyTyped(events)); + } +} diff --git a/integration/java/src/main/java/com/google/cloud/dataflow/integration/nexmark/Query11.java b/integration/java/src/main/java/com/google/cloud/dataflow/integration/nexmark/Query11.java new file mode 100644 index 000000000000..90f61ffb3dae --- /dev/null +++ b/integration/java/src/main/java/com/google/cloud/dataflow/integration/nexmark/Query11.java @@ -0,0 +1,73 @@ +/* + * Copyright (C) 2015 Google Inc. + * + * Licensed under the Apache License, Version 2.0 (the "License"); you may not + * use this file except in compliance with the License. You may obtain a copy of + * the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT + * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the + * License for the specific language governing permissions and limitations under + * the License. + */ + +package com.google.cloud.dataflow.integration.nexmark; + +import com.google.cloud.dataflow.sdk.transforms.Count; +import com.google.cloud.dataflow.sdk.transforms.DoFn; +import com.google.cloud.dataflow.sdk.transforms.ParDo; +import com.google.cloud.dataflow.sdk.transforms.windowing.AfterPane; +import com.google.cloud.dataflow.sdk.transforms.windowing.Sessions; +import com.google.cloud.dataflow.sdk.transforms.windowing.Window; +import com.google.cloud.dataflow.sdk.values.KV; +import com.google.cloud.dataflow.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(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/com/google/cloud/dataflow/integration/nexmark/Query1Model.java b/integration/java/src/main/java/com/google/cloud/dataflow/integration/nexmark/Query1Model.java new file mode 100644 index 000000000000..47dc098de600 --- /dev/null +++ b/integration/java/src/main/java/com/google/cloud/dataflow/integration/nexmark/Query1Model.java @@ -0,0 +1,71 @@ +/* + * Copyright (C) 2015 Google Inc. + * + * Licensed under the Apache License, Version 2.0 (the "License"); you may not + * use this file except in compliance with the License. You may obtain a copy of + * the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT + * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the + * License for the specific language governing permissions and limitations under + * the License. + */ + +package com.google.cloud.dataflow.integration.nexmark; + +import com.google.cloud.dataflow.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/com/google/cloud/dataflow/integration/nexmark/Query2.java b/integration/java/src/main/java/com/google/cloud/dataflow/integration/nexmark/Query2.java new file mode 100644 index 000000000000..47b521980863 --- /dev/null +++ b/integration/java/src/main/java/com/google/cloud/dataflow/integration/nexmark/Query2.java @@ -0,0 +1,73 @@ +/* + * Copyright (C) 2015 Google Inc. + * + * Licensed under the Apache License, Version 2.0 (the "License"); you may not + * use this file except in compliance with the License. You may obtain a copy of + * the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT + * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the + * License for the specific language governing permissions and limitations under + * the License. + */ + +package com.google.cloud.dataflow.integration.nexmark; + +import com.google.cloud.dataflow.sdk.transforms.DoFn; +import com.google.cloud.dataflow.sdk.transforms.Filter; +import com.google.cloud.dataflow.sdk.transforms.ParDo; +import com.google.cloud.dataflow.sdk.transforms.SerializableFunction; +import com.google.cloud.dataflow.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/com/google/cloud/dataflow/integration/nexmark/Query2Model.java b/integration/java/src/main/java/com/google/cloud/dataflow/integration/nexmark/Query2Model.java new file mode 100644 index 000000000000..bf30b01d1516 --- /dev/null +++ b/integration/java/src/main/java/com/google/cloud/dataflow/integration/nexmark/Query2Model.java @@ -0,0 +1,74 @@ +/* + * Copyright (C) 2015 Google Inc. + * + * Licensed under the Apache License, Version 2.0 (the "License"); you may not + * use this file except in compliance with the License. You may obtain a copy of + * the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT + * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the + * License for the specific language governing permissions and limitations under + * the License. + */ + +package com.google.cloud.dataflow.integration.nexmark; + +import com.google.cloud.dataflow.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/com/google/cloud/dataflow/integration/nexmark/Query3.java b/integration/java/src/main/java/com/google/cloud/dataflow/integration/nexmark/Query3.java new file mode 100644 index 000000000000..cbb2fa161582 --- /dev/null +++ b/integration/java/src/main/java/com/google/cloud/dataflow/integration/nexmark/Query3.java @@ -0,0 +1,246 @@ +/* + * Copyright (C) 2015 Google Inc. + * + * Licensed under the Apache License, Version 2.0 (the "License"); you may not + * use this file except in compliance with the License. You may obtain a copy of + * the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT + * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the + * License for the specific language governing permissions and limitations under + * the License. + */ + +package com.google.cloud.dataflow.integration.nexmark; + +import com.google.cloud.dataflow.sdk.coders.ListCoder; +import com.google.cloud.dataflow.sdk.transforms.Aggregator; +import com.google.cloud.dataflow.sdk.transforms.DoFn; +import com.google.cloud.dataflow.sdk.transforms.Filter; +import com.google.cloud.dataflow.sdk.transforms.ParDo; +import com.google.cloud.dataflow.sdk.transforms.SerializableFunction; +import com.google.cloud.dataflow.sdk.transforms.Sum.SumLongFn; +import com.google.cloud.dataflow.sdk.transforms.join.CoGbkResult; +import com.google.cloud.dataflow.sdk.transforms.join.CoGroupByKey; +import com.google.cloud.dataflow.sdk.transforms.join.KeyedPCollectionTuple; +import com.google.cloud.dataflow.sdk.transforms.windowing.FixedWindows; +import com.google.cloud.dataflow.sdk.transforms.windowing.Window; +import com.google.cloud.dataflow.sdk.util.state.StateInternals; +import com.google.cloud.dataflow.sdk.util.state.StateNamespace; +import com.google.cloud.dataflow.sdk.util.state.StateNamespaces; +import com.google.cloud.dataflow.sdk.util.state.StateTag; +import com.google.cloud.dataflow.sdk.util.state.StateTags; +import com.google.cloud.dataflow.sdk.util.state.ValueState; +import com.google.cloud.dataflow.sdk.values.KV; +import com.google.cloud.dataflow.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/com/google/cloud/dataflow/integration/nexmark/Query3Model.java b/integration/java/src/main/java/com/google/cloud/dataflow/integration/nexmark/Query3Model.java new file mode 100644 index 000000000000..6ea152a1e79d --- /dev/null +++ b/integration/java/src/main/java/com/google/cloud/dataflow/integration/nexmark/Query3Model.java @@ -0,0 +1,117 @@ +/* + * Copyright (C) 2015 Google Inc. + * + * Licensed under the Apache License, Version 2.0 (the "License"); you may not + * use this file except in compliance with the License. You may obtain a copy of + * the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT + * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the + * License for the specific language governing permissions and limitations under + * the License. + */ + +package com.google.cloud.dataflow.integration.nexmark; + +import com.google.cloud.dataflow.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/com/google/cloud/dataflow/integration/nexmark/Query4.java b/integration/java/src/main/java/com/google/cloud/dataflow/integration/nexmark/Query4.java new file mode 100644 index 000000000000..332be51db079 --- /dev/null +++ b/integration/java/src/main/java/com/google/cloud/dataflow/integration/nexmark/Query4.java @@ -0,0 +1,108 @@ +/* + * Copyright (C) 2015 Google Inc. + * + * Licensed under the Apache License, Version 2.0 (the "License"); you may not + * use this file except in compliance with the License. You may obtain a copy of + * the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT + * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the + * License for the specific language governing permissions and limitations under + * the License. + */ + +package com.google.cloud.dataflow.integration.nexmark; + +import com.google.cloud.dataflow.sdk.transforms.DoFn; +import com.google.cloud.dataflow.sdk.transforms.Mean; +import com.google.cloud.dataflow.sdk.transforms.ParDo; +import com.google.cloud.dataflow.sdk.transforms.windowing.SlidingWindows; +import com.google.cloud.dataflow.sdk.transforms.windowing.Window; +import com.google.cloud.dataflow.sdk.values.KV; +import com.google.cloud.dataflow.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/com/google/cloud/dataflow/integration/nexmark/Query4Model.java b/integration/java/src/main/java/com/google/cloud/dataflow/integration/nexmark/Query4Model.java new file mode 100644 index 000000000000..3f3359ed2a8f --- /dev/null +++ b/integration/java/src/main/java/com/google/cloud/dataflow/integration/nexmark/Query4Model.java @@ -0,0 +1,179 @@ +/* + * Copyright (C) 2015 Google Inc. + * + * Licensed under the Apache License, Version 2.0 (the "License"); you may not + * use this file except in compliance with the License. You may obtain a copy of + * the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT + * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the + * License for the specific language governing permissions and limitations under + * the License. + */ + +package com.google.cloud.dataflow.integration.nexmark; + +import com.google.cloud.dataflow.sdk.transforms.windowing.BoundedWindow; +import com.google.cloud.dataflow.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/com/google/cloud/dataflow/integration/nexmark/Query5.java b/integration/java/src/main/java/com/google/cloud/dataflow/integration/nexmark/Query5.java new file mode 100644 index 000000000000..f42221e5cc5e --- /dev/null +++ b/integration/java/src/main/java/com/google/cloud/dataflow/integration/nexmark/Query5.java @@ -0,0 +1,125 @@ +/* + * Copyright (C) 2015 Google Inc. + * + * Licensed under the Apache License, Version 2.0 (the "License"); you may not + * use this file except in compliance with the License. You may obtain a copy of + * the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT + * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the + * License for the specific language governing permissions and limitations under + * the License. + */ + +package com.google.cloud.dataflow.integration.nexmark; + +import com.google.cloud.dataflow.sdk.transforms.Combine; +import com.google.cloud.dataflow.sdk.transforms.Count; +import com.google.cloud.dataflow.sdk.transforms.DoFn; +import com.google.cloud.dataflow.sdk.transforms.ParDo; +import com.google.cloud.dataflow.sdk.transforms.windowing.SlidingWindows; +import com.google.cloud.dataflow.sdk.transforms.windowing.Window; +import com.google.cloud.dataflow.sdk.values.KV; +import com.google.cloud.dataflow.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/com/google/cloud/dataflow/integration/nexmark/Query5Model.java b/integration/java/src/main/java/com/google/cloud/dataflow/integration/nexmark/Query5Model.java new file mode 100644 index 000000000000..4265c31067aa --- /dev/null +++ b/integration/java/src/main/java/com/google/cloud/dataflow/integration/nexmark/Query5Model.java @@ -0,0 +1,172 @@ +/* + * Copyright (C) 2015 Google Inc. + * + * Licensed under the Apache License, Version 2.0 (the "License"); you may not + * use this file except in compliance with the License. You may obtain a copy of + * the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT + * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the + * License for the specific language governing permissions and limitations under + * the License. + */ + +package com.google.cloud.dataflow.integration.nexmark; + +import com.google.cloud.dataflow.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/com/google/cloud/dataflow/integration/nexmark/Query6.java b/integration/java/src/main/java/com/google/cloud/dataflow/integration/nexmark/Query6.java new file mode 100644 index 000000000000..4706b346d362 --- /dev/null +++ b/integration/java/src/main/java/com/google/cloud/dataflow/integration/nexmark/Query6.java @@ -0,0 +1,152 @@ +/* + * Copyright (C) 2015 Google Inc. + * + * Licensed under the Apache License, Version 2.0 (the "License"); you may not + * use this file except in compliance with the License. You may obtain a copy of + * the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT + * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the + * License for the specific language governing permissions and limitations under + * the License. + */ + +package com.google.cloud.dataflow.integration.nexmark; + +import com.google.cloud.dataflow.sdk.transforms.Combine; +import com.google.cloud.dataflow.sdk.transforms.DoFn; +import com.google.cloud.dataflow.sdk.transforms.ParDo; +import com.google.cloud.dataflow.sdk.transforms.windowing.AfterPane; +import com.google.cloud.dataflow.sdk.transforms.windowing.GlobalWindows; +import com.google.cloud.dataflow.sdk.transforms.windowing.Repeatedly; +import com.google.cloud.dataflow.sdk.transforms.windowing.Window; +import com.google.cloud.dataflow.sdk.values.KV; +import com.google.cloud.dataflow.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/com/google/cloud/dataflow/integration/nexmark/Query6Model.java b/integration/java/src/main/java/com/google/cloud/dataflow/integration/nexmark/Query6Model.java new file mode 100644 index 000000000000..98493fdfe42e --- /dev/null +++ b/integration/java/src/main/java/com/google/cloud/dataflow/integration/nexmark/Query6Model.java @@ -0,0 +1,126 @@ +/* + * Copyright (C) 2015 Google Inc. + * + * Licensed under the Apache License, Version 2.0 (the "License"); you may not + * use this file except in compliance with the License. You may obtain a copy of + * the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT + * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the + * License for the specific language governing permissions and limitations under + * the License. + */ + +package com.google.cloud.dataflow.integration.nexmark; + +import com.google.cloud.dataflow.sdk.transforms.windowing.BoundedWindow; +import com.google.cloud.dataflow.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/com/google/cloud/dataflow/integration/nexmark/Query7.java b/integration/java/src/main/java/com/google/cloud/dataflow/integration/nexmark/Query7.java new file mode 100644 index 000000000000..12b958bde998 --- /dev/null +++ b/integration/java/src/main/java/com/google/cloud/dataflow/integration/nexmark/Query7.java @@ -0,0 +1,85 @@ +/* + * Copyright (C) 2015 Google Inc. + * + * Licensed under the Apache License, Version 2.0 (the "License"); you may not + * use this file except in compliance with the License. You may obtain a copy of + * the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT + * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the + * License for the specific language governing permissions and limitations under + * the License. + */ + +package com.google.cloud.dataflow.integration.nexmark; + +import com.google.cloud.dataflow.sdk.transforms.DoFn; +import com.google.cloud.dataflow.sdk.transforms.Max; +import com.google.cloud.dataflow.sdk.transforms.ParDo; +import com.google.cloud.dataflow.sdk.transforms.windowing.FixedWindows; +import com.google.cloud.dataflow.sdk.transforms.windowing.Window; +import com.google.cloud.dataflow.sdk.values.PCollection; +import com.google.cloud.dataflow.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/com/google/cloud/dataflow/integration/nexmark/Query7Model.java b/integration/java/src/main/java/com/google/cloud/dataflow/integration/nexmark/Query7Model.java new file mode 100644 index 000000000000..7437e0130c70 --- /dev/null +++ b/integration/java/src/main/java/com/google/cloud/dataflow/integration/nexmark/Query7Model.java @@ -0,0 +1,126 @@ +/* + * Copyright (C) 2015 Google Inc. + * + * Licensed under the Apache License, Version 2.0 (the "License"); you may not + * use this file except in compliance with the License. You may obtain a copy of + * the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT + * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the + * License for the specific language governing permissions and limitations under + * the License. + */ + +package com.google.cloud.dataflow.integration.nexmark; + +import com.google.cloud.dataflow.sdk.transforms.windowing.BoundedWindow; +import com.google.cloud.dataflow.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/com/google/cloud/dataflow/integration/nexmark/Query8.java b/integration/java/src/main/java/com/google/cloud/dataflow/integration/nexmark/Query8.java new file mode 100644 index 000000000000..7b5acb3ec6b1 --- /dev/null +++ b/integration/java/src/main/java/com/google/cloud/dataflow/integration/nexmark/Query8.java @@ -0,0 +1,90 @@ +/* + * Copyright (C) 2015 Google Inc. + * + * Licensed under the Apache License, Version 2.0 (the "License"); you may not + * use this file except in compliance with the License. You may obtain a copy of + * the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT + * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the + * License for the specific language governing permissions and limitations under + * the License. + */ + +package com.google.cloud.dataflow.integration.nexmark; + +import com.google.cloud.dataflow.sdk.transforms.DoFn; +import com.google.cloud.dataflow.sdk.transforms.ParDo; +import com.google.cloud.dataflow.sdk.transforms.join.CoGbkResult; +import com.google.cloud.dataflow.sdk.transforms.join.CoGroupByKey; +import com.google.cloud.dataflow.sdk.transforms.join.KeyedPCollectionTuple; +import com.google.cloud.dataflow.sdk.transforms.windowing.FixedWindows; +import com.google.cloud.dataflow.sdk.transforms.windowing.Window; +import com.google.cloud.dataflow.sdk.values.KV; +import com.google.cloud.dataflow.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/com/google/cloud/dataflow/integration/nexmark/Query8Model.java b/integration/java/src/main/java/com/google/cloud/dataflow/integration/nexmark/Query8Model.java new file mode 100644 index 000000000000..9fd110fe1ddd --- /dev/null +++ b/integration/java/src/main/java/com/google/cloud/dataflow/integration/nexmark/Query8Model.java @@ -0,0 +1,143 @@ +/* + * Copyright (C) 2015 Google Inc. + * + * Licensed under the Apache License, Version 2.0 (the "License"); you may not + * use this file except in compliance with the License. You may obtain a copy of + * the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT + * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the + * License for the specific language governing permissions and limitations under + * the License. + */ + +package com.google.cloud.dataflow.integration.nexmark; + +import com.google.cloud.dataflow.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/com/google/cloud/dataflow/integration/nexmark/Query9.java b/integration/java/src/main/java/com/google/cloud/dataflow/integration/nexmark/Query9.java new file mode 100644 index 000000000000..a497951c5a3c --- /dev/null +++ b/integration/java/src/main/java/com/google/cloud/dataflow/integration/nexmark/Query9.java @@ -0,0 +1,38 @@ +/* + * Copyright (C) 2015 Google Inc. + * + * Licensed under the Apache License, Version 2.0 (the "License"); you may not + * use this file except in compliance with the License. You may obtain a copy of + * the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT + * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the + * License for the specific language governing permissions and limitations under + * the License. + */ + +package com.google.cloud.dataflow.integration.nexmark; + +import com.google.cloud.dataflow.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/com/google/cloud/dataflow/integration/nexmark/Query9Model.java b/integration/java/src/main/java/com/google/cloud/dataflow/integration/nexmark/Query9Model.java new file mode 100644 index 000000000000..c64f5a70cf8e --- /dev/null +++ b/integration/java/src/main/java/com/google/cloud/dataflow/integration/nexmark/Query9Model.java @@ -0,0 +1,42 @@ +/* + * Copyright (C) 2015 Google Inc. + * + * Licensed under the Apache License, Version 2.0 (the "License"); you may not + * use this file except in compliance with the License. You may obtain a copy of + * the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT + * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the + * License for the specific language governing permissions and limitations under + * the License. + */ + +package com.google.cloud.dataflow.integration.nexmark; + +import com.google.cloud.dataflow.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/com/google/cloud/dataflow/integration/nexmark/SellerPrice.java b/integration/java/src/main/java/com/google/cloud/dataflow/integration/nexmark/SellerPrice.java new file mode 100644 index 000000000000..bf32275b7c68 --- /dev/null +++ b/integration/java/src/main/java/com/google/cloud/dataflow/integration/nexmark/SellerPrice.java @@ -0,0 +1,89 @@ +/* + * Copyright (C) 2015 Google Inc. + * + * Licensed under the Apache License, Version 2.0 (the "License"); you may not + * use this file except in compliance with the License. You may obtain a copy of + * the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT + * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the + * License for the specific language governing permissions and limitations under + * the License. + */ + +package com.google.cloud.dataflow.integration.nexmark; + +import com.google.cloud.dataflow.sdk.coders.AtomicCoder; +import com.google.cloud.dataflow.sdk.coders.Coder; +import com.google.cloud.dataflow.sdk.coders.CoderException; +import com.google.cloud.dataflow.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, + com.google.cloud.dataflow.sdk.coders.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, com.google.cloud.dataflow.sdk.coders.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/com/google/cloud/dataflow/integration/nexmark/UnboundedEventSource.java b/integration/java/src/main/java/com/google/cloud/dataflow/integration/nexmark/UnboundedEventSource.java new file mode 100644 index 000000000000..be88f0712d83 --- /dev/null +++ b/integration/java/src/main/java/com/google/cloud/dataflow/integration/nexmark/UnboundedEventSource.java @@ -0,0 +1,320 @@ +/* + * Copyright (C) 2015 Google Inc. + * + * Licensed under the Apache License, Version 2.0 (the "License"); you may not + * use this file except in compliance with the License. You may obtain a copy of + * the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT + * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the + * License for the specific language governing permissions and limitations under + * the License. + */ + +package com.google.cloud.dataflow.integration.nexmark; + +import com.google.cloud.dataflow.sdk.coders.Coder; +import com.google.cloud.dataflow.sdk.io.UnboundedSource; +import com.google.cloud.dataflow.sdk.options.PipelineOptions; +import com.google.cloud.dataflow.sdk.transforms.windowing.BoundedWindow; +import com.google.cloud.dataflow.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(); + } 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; + 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 (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; + 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/com/google/cloud/dataflow/integration/nexmark/WinningBids.java b/integration/java/src/main/java/com/google/cloud/dataflow/integration/nexmark/WinningBids.java new file mode 100644 index 000000000000..c8e327038784 --- /dev/null +++ b/integration/java/src/main/java/com/google/cloud/dataflow/integration/nexmark/WinningBids.java @@ -0,0 +1,383 @@ +/* + * Copyright (C) 2015 Google Inc. + * + * Licensed under the Apache License, Version 2.0 (the "License"); you may not + * use this file except in compliance with the License. You may obtain a copy of + * the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT + * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the + * License for the specific language governing permissions and limitations under + * the License. + */ + +package com.google.cloud.dataflow.integration.nexmark; + +import com.google.cloud.dataflow.sdk.coders.AtomicCoder; +import com.google.cloud.dataflow.sdk.coders.Coder; +import com.google.cloud.dataflow.sdk.coders.CoderException; +import com.google.cloud.dataflow.sdk.coders.VarIntCoder; +import com.google.cloud.dataflow.sdk.coders.VarLongCoder; +import com.google.cloud.dataflow.sdk.transforms.Aggregator; +import com.google.cloud.dataflow.sdk.transforms.DoFn; +import com.google.cloud.dataflow.sdk.transforms.PTransform; +import com.google.cloud.dataflow.sdk.transforms.ParDo; +import com.google.cloud.dataflow.sdk.transforms.Sum.SumLongFn; +import com.google.cloud.dataflow.sdk.transforms.join.CoGbkResult; +import com.google.cloud.dataflow.sdk.transforms.join.CoGroupByKey; +import com.google.cloud.dataflow.sdk.transforms.join.KeyedPCollectionTuple; +import com.google.cloud.dataflow.sdk.transforms.windowing.BoundedWindow; +import com.google.cloud.dataflow.sdk.transforms.windowing.IntervalWindow; +import com.google.cloud.dataflow.sdk.transforms.windowing.OutputTimeFn; +import com.google.cloud.dataflow.sdk.transforms.windowing.Window; +import com.google.cloud.dataflow.sdk.transforms.windowing.WindowFn; +import com.google.cloud.dataflow.sdk.values.KV; +import com.google.cloud.dataflow.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 qps (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"); + } + + private static final OutputTimeFn.DependsOnlyOnWindow OUTPUT_TIME_FN = + new OutputTimeFn.DependsOnlyOnWindow() { + @Override + protected Instant assignOutputTime(AuctionOrBidWindow window) { + return window.maxTimestamp(); + } + }; + + /** + * 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 OutputTimeFn getOutputTimeFn() { + return OUTPUT_TIME_FN; + } + } + + 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 qps). + long[] interEventDelayUs = configuration.qpsShape.interEventDelayUs( + configuration.firstEventQps, configuration.nextEventQps, 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(null, "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/com/google/cloud/dataflow/integration/nexmark/WinningBidsSimulator.java b/integration/java/src/main/java/com/google/cloud/dataflow/integration/nexmark/WinningBidsSimulator.java new file mode 100644 index 000000000000..489c82ace374 --- /dev/null +++ b/integration/java/src/main/java/com/google/cloud/dataflow/integration/nexmark/WinningBidsSimulator.java @@ -0,0 +1,201 @@ +/* + * Copyright (C) 2015 Google Inc. + * + * Licensed under the Apache License, Version 2.0 (the "License"); you may not + * use this file except in compliance with the License. You may obtain a copy of + * the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT + * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the + * License for the specific language governing permissions and limitations under + * the License. + */ + +package com.google.cloud.dataflow.integration.nexmark; + +import com.google.cloud.dataflow.sdk.transforms.windowing.BoundedWindow; +import com.google.cloud.dataflow.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 6a1206751a7e..173a453a5f30 100644 --- a/pom.xml +++ b/pom.xml @@ -126,49 +126,40 @@ pom - sdks + sdks/java/core runners - examples + examples/java + integration/java + sdks/java/maven-archetypes - doclint-java8-disable + java8-tests [1.8,) - - -Xdoclint:-missing - + + sdks/java/java8tests + - src + java8-examples - false + [1.8,) - - - - org.apache.maven.plugins - maven-assembly-plugin - - - src - package - - single - - - apache-beam-${project.version} - - sdks/java/src.xml - - - - - - - + + examples/java8 + + + + doclint-java8-disable + + [1.8,) + + + -Xdoclint:-missing + @@ -270,13 +261,6 @@ - - - org.eluder.coveralls - coveralls-maven-plugin - 4.1.0 - - org.apache.maven.plugins maven-surefire-plugin @@ -374,18 +358,6 @@ build-helper-maven-plugin 1.10 - - - org.apache.maven.plugins - maven-release-plugin - 2.5.3 - - src - clean install - deploy - true - - From 15d4b54ac9c2f3b52a4ce51acc70898e668934b0 Mon Sep 17 00:00:00 2001 From: Mark Shields Date: Wed, 30 Mar 2016 13:24:57 -0700 Subject: [PATCH 06/14] Limit dataflow dependencies --- .../dataflow/integration/nexmark/Auction.java | 40 +- .../integration/nexmark/AuctionBid.java | 8 +- .../integration/nexmark/AuctionCount.java | 8 +- .../integration/nexmark/AuctionPrice.java | 8 +- .../dataflow/integration/nexmark/Bid.java | 20 +- .../integration/nexmark/BidsPerSession.java | 8 +- .../integration/nexmark/CategoryPrice.java | 10 +- .../dataflow/integration/nexmark/Done.java | 4 +- .../dataflow/integration/nexmark/Event.java | 18 +- .../integration/nexmark/Generator.java | 8 +- .../integration/nexmark/GeneratorConfig.java | 1 - .../integration/nexmark/IdNameReserve.java | 12 +- .../integration/nexmark/KnownSize.java | 2 +- .../dataflow/integration/nexmark/Monitor.java | 2 +- .../integration/nexmark/NameCityStateId.java | 16 +- .../nexmark/NexmarkConfiguration.java | 111 +--- .../integration/nexmark/NexmarkDriver.java | 625 ------------------ .../nexmark/NexmarkGoogleDriver.java | 301 +++++++++ ...rkRunner.java => NexmarkGoogleRunner.java} | 253 ++++--- .../integration/nexmark/NexmarkSuite.java | 289 +------- .../integration/nexmark/NexmarkUtils.java | 2 +- .../dataflow/integration/nexmark/Options.java | 355 ++++++++++ .../dataflow/integration/nexmark/Person.java | 32 +- .../integration/nexmark/PubsubHelper.java | 184 +++--- .../dataflow/integration/nexmark/Query10.java | 341 +++++----- .../integration/nexmark/SellerPrice.java | 8 +- .../integration/nexmark/WinningBids.java | 12 +- 27 files changed, 1191 insertions(+), 1487 deletions(-) delete mode 100644 integration/java/src/main/java/com/google/cloud/dataflow/integration/nexmark/NexmarkDriver.java create mode 100644 integration/java/src/main/java/com/google/cloud/dataflow/integration/nexmark/NexmarkGoogleDriver.java rename integration/java/src/main/java/com/google/cloud/dataflow/integration/nexmark/{NexmarkRunner.java => NexmarkGoogleRunner.java} (82%) create mode 100644 integration/java/src/main/java/com/google/cloud/dataflow/integration/nexmark/Options.java diff --git a/integration/java/src/main/java/com/google/cloud/dataflow/integration/nexmark/Auction.java b/integration/java/src/main/java/com/google/cloud/dataflow/integration/nexmark/Auction.java index 1ff5847bbaed..f4c97767fe44 100644 --- a/integration/java/src/main/java/com/google/cloud/dataflow/integration/nexmark/Auction.java +++ b/integration/java/src/main/java/com/google/cloud/dataflow/integration/nexmark/Auction.java @@ -42,32 +42,32 @@ public class Auction implements KnownSize, Serializable { public void encode(Auction value, OutputStream outStream, com.google.cloud.dataflow.sdk.coders.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()); + 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, com.google.cloud.dataflow.sdk.coders.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()); + 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); diff --git a/integration/java/src/main/java/com/google/cloud/dataflow/integration/nexmark/AuctionBid.java b/integration/java/src/main/java/com/google/cloud/dataflow/integration/nexmark/AuctionBid.java index b1b505d06c7f..805925c98278 100644 --- a/integration/java/src/main/java/com/google/cloud/dataflow/integration/nexmark/AuctionBid.java +++ b/integration/java/src/main/java/com/google/cloud/dataflow/integration/nexmark/AuctionBid.java @@ -37,16 +37,16 @@ public class AuctionBid implements KnownSize, Serializable { public void encode(AuctionBid value, OutputStream outStream, com.google.cloud.dataflow.sdk.coders.Coder.Context context) throws CoderException, IOException { - Auction.CODER.encode(value.auction, outStream, context.nested()); - Bid.CODER.encode(value.bid, outStream, context.nested()); + Auction.CODER.encode(value.auction, outStream, Context.NESTED); + Bid.CODER.encode(value.bid, outStream, Context.NESTED); } @Override public AuctionBid decode( InputStream inStream, com.google.cloud.dataflow.sdk.coders.Coder.Context context) throws CoderException, IOException { - Auction auction = Auction.CODER.decode(inStream, context.nested()); - Bid bid = Bid.CODER.decode(inStream, context.nested()); + Auction auction = Auction.CODER.decode(inStream, Context.NESTED); + Bid bid = Bid.CODER.decode(inStream, Context.NESTED); return new AuctionBid(auction, bid); } }; diff --git a/integration/java/src/main/java/com/google/cloud/dataflow/integration/nexmark/AuctionCount.java b/integration/java/src/main/java/com/google/cloud/dataflow/integration/nexmark/AuctionCount.java index 663ab3aa1817..61984659e51c 100644 --- a/integration/java/src/main/java/com/google/cloud/dataflow/integration/nexmark/AuctionCount.java +++ b/integration/java/src/main/java/com/google/cloud/dataflow/integration/nexmark/AuctionCount.java @@ -40,16 +40,16 @@ public class AuctionCount implements KnownSize, Serializable { public void encode(AuctionCount value, OutputStream outStream, com.google.cloud.dataflow.sdk.coders.Coder.Context context) throws CoderException, IOException { - LONG_CODER.encode(value.auction, outStream, context.nested()); - LONG_CODER.encode(value.count, outStream, context.nested()); + LONG_CODER.encode(value.auction, outStream, Context.NESTED); + LONG_CODER.encode(value.count, outStream, Context.NESTED); } @Override public AuctionCount decode( InputStream inStream, com.google.cloud.dataflow.sdk.coders.Coder.Context context) throws CoderException, IOException { - long auction = LONG_CODER.decode(inStream, context.nested()); - long count = LONG_CODER.decode(inStream, context.nested()); + long auction = LONG_CODER.decode(inStream, Context.NESTED); + long count = LONG_CODER.decode(inStream, Context.NESTED); return new AuctionCount(auction, count); } }; diff --git a/integration/java/src/main/java/com/google/cloud/dataflow/integration/nexmark/AuctionPrice.java b/integration/java/src/main/java/com/google/cloud/dataflow/integration/nexmark/AuctionPrice.java index e3a5678cf9a0..5a5b73d309e6 100644 --- a/integration/java/src/main/java/com/google/cloud/dataflow/integration/nexmark/AuctionPrice.java +++ b/integration/java/src/main/java/com/google/cloud/dataflow/integration/nexmark/AuctionPrice.java @@ -40,16 +40,16 @@ public class AuctionPrice implements KnownSize, Serializable { public void encode(AuctionPrice value, OutputStream outStream, com.google.cloud.dataflow.sdk.coders.Coder.Context context) throws CoderException, IOException { - LONG_CODER.encode(value.auction, outStream, context.nested()); - LONG_CODER.encode(value.price, outStream, context.nested()); + LONG_CODER.encode(value.auction, outStream, Context.NESTED); + LONG_CODER.encode(value.price, outStream, Context.NESTED); } @Override public AuctionPrice decode( InputStream inStream, com.google.cloud.dataflow.sdk.coders.Coder.Context context) throws CoderException, IOException { - long auction = LONG_CODER.decode(inStream, context.nested()); - long price = LONG_CODER.decode(inStream, context.nested()); + long auction = LONG_CODER.decode(inStream, Context.NESTED); + long price = LONG_CODER.decode(inStream, Context.NESTED); return new AuctionPrice(auction, price); } }; diff --git a/integration/java/src/main/java/com/google/cloud/dataflow/integration/nexmark/Bid.java b/integration/java/src/main/java/com/google/cloud/dataflow/integration/nexmark/Bid.java index bb654c9ae170..543f66506a76 100644 --- a/integration/java/src/main/java/com/google/cloud/dataflow/integration/nexmark/Bid.java +++ b/integration/java/src/main/java/com/google/cloud/dataflow/integration/nexmark/Bid.java @@ -43,22 +43,22 @@ public class Bid implements KnownSize, Serializable { public void encode(Bid value, OutputStream outStream, com.google.cloud.dataflow.sdk.coders.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()); + 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, com.google.cloud.dataflow.sdk.coders.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()); + 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); } }; diff --git a/integration/java/src/main/java/com/google/cloud/dataflow/integration/nexmark/BidsPerSession.java b/integration/java/src/main/java/com/google/cloud/dataflow/integration/nexmark/BidsPerSession.java index 41223b0e56e7..fd9b42f521e3 100644 --- a/integration/java/src/main/java/com/google/cloud/dataflow/integration/nexmark/BidsPerSession.java +++ b/integration/java/src/main/java/com/google/cloud/dataflow/integration/nexmark/BidsPerSession.java @@ -40,16 +40,16 @@ public class BidsPerSession implements KnownSize, Serializable { public void encode(BidsPerSession value, OutputStream outStream, com.google.cloud.dataflow.sdk.coders.Coder.Context context) throws CoderException, IOException { - LONG_CODER.encode(value.personId, outStream, context.nested()); - LONG_CODER.encode(value.bidsPerSession, outStream, context.nested()); + LONG_CODER.encode(value.personId, outStream, Context.NESTED); + LONG_CODER.encode(value.bidsPerSession, outStream, Context.NESTED); } @Override public BidsPerSession decode( InputStream inStream, com.google.cloud.dataflow.sdk.coders.Coder.Context context) throws CoderException, IOException { - long personId = LONG_CODER.decode(inStream, context.nested()); - long bidsPerSession = LONG_CODER.decode(inStream, context.nested()); + long personId = LONG_CODER.decode(inStream, Context.NESTED); + long bidsPerSession = LONG_CODER.decode(inStream, Context.NESTED); return new BidsPerSession(personId, bidsPerSession); } }; diff --git a/integration/java/src/main/java/com/google/cloud/dataflow/integration/nexmark/CategoryPrice.java b/integration/java/src/main/java/com/google/cloud/dataflow/integration/nexmark/CategoryPrice.java index db50535119c6..4439b48ab11a 100644 --- a/integration/java/src/main/java/com/google/cloud/dataflow/integration/nexmark/CategoryPrice.java +++ b/integration/java/src/main/java/com/google/cloud/dataflow/integration/nexmark/CategoryPrice.java @@ -42,17 +42,17 @@ public class CategoryPrice implements KnownSize, Serializable { public void encode(CategoryPrice value, OutputStream outStream, com.google.cloud.dataflow.sdk.coders.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()); + 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, com.google.cloud.dataflow.sdk.coders.Coder.Context context) throws CoderException, IOException { - long category = LONG_CODER.decode(inStream, context.nested()); - long price = LONG_CODER.decode(inStream, context.nested()); + 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); } diff --git a/integration/java/src/main/java/com/google/cloud/dataflow/integration/nexmark/Done.java b/integration/java/src/main/java/com/google/cloud/dataflow/integration/nexmark/Done.java index 7467cf95dec4..af3e01485717 100644 --- a/integration/java/src/main/java/com/google/cloud/dataflow/integration/nexmark/Done.java +++ b/integration/java/src/main/java/com/google/cloud/dataflow/integration/nexmark/Done.java @@ -40,14 +40,14 @@ public class Done implements KnownSize, Serializable { public void encode(Done value, OutputStream outStream, com.google.cloud.dataflow.sdk.coders.Coder.Context context) throws CoderException, IOException { - STRING_CODER.encode(value.message, outStream, context.nested()); + STRING_CODER.encode(value.message, outStream, Context.NESTED); } @Override public Done decode( InputStream inStream, com.google.cloud.dataflow.sdk.coders.Coder.Context context) throws CoderException, IOException { - String message = STRING_CODER.decode(inStream, context.nested()); + String message = STRING_CODER.decode(inStream, Context.NESTED); return new Done(message); } }; diff --git a/integration/java/src/main/java/com/google/cloud/dataflow/integration/nexmark/Event.java b/integration/java/src/main/java/com/google/cloud/dataflow/integration/nexmark/Event.java index 1c4ffec4273f..da569e49361d 100644 --- a/integration/java/src/main/java/com/google/cloud/dataflow/integration/nexmark/Event.java +++ b/integration/java/src/main/java/com/google/cloud/dataflow/integration/nexmark/Event.java @@ -41,14 +41,14 @@ public void encode(Event value, OutputStream outStream, com.google.cloud.dataflow.sdk.coders.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()); + 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()); + 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()); + INT_CODER.encode(2, outStream, Context.NESTED); + Bid.CODER.encode(value.bid, outStream, Context.NESTED); } else { throw new RuntimeException("invalid event"); } @@ -60,13 +60,13 @@ public Event decode( throws CoderException, IOException { int tag = INT_CODER.decode(inStream, context); if (tag == 0) { - Person person = Person.CODER.decode(inStream, context.nested()); + Person person = Person.CODER.decode(inStream, Context.NESTED); return new Event(person); } else if (tag == 1) { - Auction auction = Auction.CODER.decode(inStream, context.nested()); + Auction auction = Auction.CODER.decode(inStream, Context.NESTED); return new Event(auction); } else if (tag == 2) { - Bid bid = Bid.CODER.decode(inStream, context.nested()); + Bid bid = Bid.CODER.decode(inStream, Context.NESTED); return new Event(bid); } else { throw new RuntimeException("invalid event encoding"); diff --git a/integration/java/src/main/java/com/google/cloud/dataflow/integration/nexmark/Generator.java b/integration/java/src/main/java/com/google/cloud/dataflow/integration/nexmark/Generator.java index 6afa0c6df04b..3f63af214d6e 100644 --- a/integration/java/src/main/java/com/google/cloud/dataflow/integration/nexmark/Generator.java +++ b/integration/java/src/main/java/com/google/cloud/dataflow/integration/nexmark/Generator.java @@ -104,16 +104,16 @@ public void encode( OutputStream outStream, com.google.cloud.dataflow.sdk.coders.Coder.Context context) throws CoderException, IOException { - LONG_CODER.encode(value.numEvents, outStream, context.nested()); - LONG_CODER.encode(value.wallclockBaseTime, outStream, context.nested()); + LONG_CODER.encode(value.numEvents, outStream, Context.NESTED); + LONG_CODER.encode(value.wallclockBaseTime, outStream, Context.NESTED); } @Override public Checkpoint decode( InputStream inStream, com.google.cloud.dataflow.sdk.coders.Coder.Context context) throws CoderException, IOException { - long numEvents = LONG_CODER.decode(inStream, context.nested()); - long wallclockBaseTime = LONG_CODER.decode(inStream, context.nested()); + long numEvents = LONG_CODER.decode(inStream, Context.NESTED); + long wallclockBaseTime = LONG_CODER.decode(inStream, Context.NESTED); return new Checkpoint(numEvents, wallclockBaseTime); } }; diff --git a/integration/java/src/main/java/com/google/cloud/dataflow/integration/nexmark/GeneratorConfig.java b/integration/java/src/main/java/com/google/cloud/dataflow/integration/nexmark/GeneratorConfig.java index 6f6f2572f035..d4045f4aa11d 100644 --- a/integration/java/src/main/java/com/google/cloud/dataflow/integration/nexmark/GeneratorConfig.java +++ b/integration/java/src/main/java/com/google/cloud/dataflow/integration/nexmark/GeneratorConfig.java @@ -32,7 +32,6 @@ class GeneratorConfig implements Serializable { */ public static final long FIRST_AUCTION_ID = 1000L; public static final long FIRST_PERSON_ID = 1000L; - public static final long FIRST_BID_ID = 0L; public static final long FIRST_CATEGORY_ID = 10L; /** diff --git a/integration/java/src/main/java/com/google/cloud/dataflow/integration/nexmark/IdNameReserve.java b/integration/java/src/main/java/com/google/cloud/dataflow/integration/nexmark/IdNameReserve.java index d15c97caab7e..70fcf019fd28 100644 --- a/integration/java/src/main/java/com/google/cloud/dataflow/integration/nexmark/IdNameReserve.java +++ b/integration/java/src/main/java/com/google/cloud/dataflow/integration/nexmark/IdNameReserve.java @@ -42,18 +42,18 @@ public class IdNameReserve implements KnownSize, Serializable { public void encode(IdNameReserve value, OutputStream outStream, com.google.cloud.dataflow.sdk.coders.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()); + 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, com.google.cloud.dataflow.sdk.coders.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()); + 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); } }; diff --git a/integration/java/src/main/java/com/google/cloud/dataflow/integration/nexmark/KnownSize.java b/integration/java/src/main/java/com/google/cloud/dataflow/integration/nexmark/KnownSize.java index 89abe7a2df21..1d32a8377c1a 100644 --- a/integration/java/src/main/java/com/google/cloud/dataflow/integration/nexmark/KnownSize.java +++ b/integration/java/src/main/java/com/google/cloud/dataflow/integration/nexmark/KnownSize.java @@ -17,7 +17,7 @@ package com.google.cloud.dataflow.integration.nexmark; /** - * Interface for elements which know their encoded byte size. + * Interface for elements which can quickly estimate their encoded byte size. */ public interface KnownSize { long sizeInBytes(); diff --git a/integration/java/src/main/java/com/google/cloud/dataflow/integration/nexmark/Monitor.java b/integration/java/src/main/java/com/google/cloud/dataflow/integration/nexmark/Monitor.java index 064fe3570a2b..b499e408ba91 100644 --- a/integration/java/src/main/java/com/google/cloud/dataflow/integration/nexmark/Monitor.java +++ b/integration/java/src/main/java/com/google/cloud/dataflow/integration/nexmark/Monitor.java @@ -32,7 +32,7 @@ * * @param Type of element we are monitoring. */ -class Monitor implements Serializable { +public class Monitor implements Serializable { private class MonitorDoFn extends DoFn { public final Aggregator elementCounter = createAggregator(counterNamePrefix + "_elements", new SumLongFn()); diff --git a/integration/java/src/main/java/com/google/cloud/dataflow/integration/nexmark/NameCityStateId.java b/integration/java/src/main/java/com/google/cloud/dataflow/integration/nexmark/NameCityStateId.java index e007240d6b5e..bf5a27d2d628 100644 --- a/integration/java/src/main/java/com/google/cloud/dataflow/integration/nexmark/NameCityStateId.java +++ b/integration/java/src/main/java/com/google/cloud/dataflow/integration/nexmark/NameCityStateId.java @@ -42,20 +42,20 @@ public class NameCityStateId implements KnownSize, Serializable { public void encode(NameCityStateId value, OutputStream outStream, com.google.cloud.dataflow.sdk.coders.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()); + 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, com.google.cloud.dataflow.sdk.coders.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()); + 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); } }; diff --git a/integration/java/src/main/java/com/google/cloud/dataflow/integration/nexmark/NexmarkConfiguration.java b/integration/java/src/main/java/com/google/cloud/dataflow/integration/nexmark/NexmarkConfiguration.java index d79622b02122..3dd0004dc331 100644 --- a/integration/java/src/main/java/com/google/cloud/dataflow/integration/nexmark/NexmarkConfiguration.java +++ b/integration/java/src/main/java/com/google/cloud/dataflow/integration/nexmark/NexmarkConfiguration.java @@ -17,8 +17,6 @@ package com.google.cloud.dataflow.integration.nexmark; import com.google.cloud.dataflow.sdk.options.DataflowPipelineOptions; -import com.google.cloud.dataflow.sdk.options.DataflowPipelineWorkerPoolOptions; -import com.google.cloud.dataflow.sdk.options.StreamingOptions; import com.google.cloud.dataflow.sdk.testing.DataflowAssert; import com.fasterxml.jackson.annotation.JsonProperty; @@ -26,8 +24,6 @@ import java.io.IOException; import java.io.Serializable; -import java.util.ArrayList; -import java.util.List; import java.util.Objects; /** @@ -39,27 +35,6 @@ class NexmarkConfiguration implements Serializable { public static final NexmarkConfiguration DEFAULT = new NexmarkConfiguration(); - /** Worker machine type. */ - @JsonProperty - public String workerMachineType = "n1-standard-4"; - - /** Same as {@link DataflowPipelineWorkerPoolOptions#setNumWorkers}. */ - @JsonProperty - public int numWorkers = 1; - - /** Same as {@link DataflowPipelineWorkerPoolOptions#setMaxNumWorkers}. */ - @JsonProperty - public int maxNumWorkers = 1; - - /** Same as {@link DataflowPipelineWorkerPoolOptions#setAutoscalingAlgorithm}. */ - @JsonProperty - public DataflowPipelineWorkerPoolOptions.AutoscalingAlgorithmType autoscalingAlgorithm = - DataflowPipelineWorkerPoolOptions.AutoscalingAlgorithmType.NONE; - - /** Same as {@link StreamingOptions#setStreaming}. */ - @JsonProperty - public boolean streaming = false; - /** Which query to run, in [0,9]. */ @JsonProperty public int query = 0; @@ -236,12 +211,6 @@ class NexmarkConfiguration implements Serializable { @JsonProperty public int fanout = 5; - /** - * Experiments to enable. - */ - @JsonProperty - public List experiments = new ArrayList<>(); - /** * Length of occasional delay to impose on events (in seconds). */ @@ -279,37 +248,12 @@ class NexmarkConfiguration implements Serializable { @JsonProperty public boolean debug = true; - /** Return number of cores for configuration's machine type. */ - public int coresPerWorker() { - String[] split = workerMachineType.split("-"); - if (split.length != 3) { - return 1; - } - try { - return Integer.parseInt(split[2]); - } catch (NumberFormatException ex) { - return 1; - } - } - /** * Replace any properties of this configuration which have been supplied by the command line. * However *never replace* isStreaming since we can't tell if it was supplied by the command line * or merely has its default false value. */ - public void overrideFromOptions(NexmarkDriver.Options options) { - if (options.getWorkerMachineType() != null) { - workerMachineType = options.getWorkerMachineType(); - } - if (options.getNumWorkers() > 0) { - numWorkers = options.getNumWorkers(); - } - if (options.getMaxNumWorkers() > 0) { - maxNumWorkers = options.getMaxNumWorkers(); - } - if (options.getAutoscalingAlgorithm() != null) { - autoscalingAlgorithm = options.getAutoscalingAlgorithm(); - } + public void overrideFromOptions(Options options) { if (options.getQuery() != null) { query = options.getQuery(); } @@ -409,10 +353,6 @@ public void overrideFromOptions(NexmarkDriver.Options options) { if (options.getFanout() != null) { fanout = options.getFanout(); } - if (options.getExperiments() != null) { - experiments.clear(); - experiments.addAll(options.getExperiments()); - } if (options.getOccasionalDelaySec() != null) { occasionalDelaySec = options.getOccasionalDelaySec(); } @@ -439,11 +379,6 @@ public void overrideFromOptions(NexmarkDriver.Options options) { @Override public NexmarkConfiguration clone() { NexmarkConfiguration result = new NexmarkConfiguration(); - result.workerMachineType = workerMachineType; - result.numWorkers = numWorkers; - result.maxNumWorkers = maxNumWorkers; - result.autoscalingAlgorithm = autoscalingAlgorithm; - result.streaming = streaming; result.query = query; result.logResults = logResults; result.assertCorrectness = assertCorrectness; @@ -477,7 +412,6 @@ public NexmarkConfiguration clone() { result.diskBusyBytes = diskBusyBytes; result.auctionSkip = auctionSkip; result.fanout = fanout; - result.experiments.addAll(experiments); result.occasionalDelaySec = occasionalDelaySec; result.probDelayedEvent = probDelayedEvent; result.maxLogEvents = maxLogEvents; @@ -493,19 +427,6 @@ public NexmarkConfiguration clone() { public String toShortString() { StringBuilder sb = new StringBuilder(); sb.append(String.format("query:%d", query)); - sb.append(String.format("; streaming:%s", streaming)); - if (!workerMachineType.equals(DEFAULT.workerMachineType)) { - sb.append(String.format("; workerMachineType:%s", workerMachineType)); - } - if (numWorkers != DEFAULT.numWorkers) { - sb.append(String.format("; numWorkers:%d", numWorkers)); - } - if (maxNumWorkers != DEFAULT.maxNumWorkers) { - sb.append(String.format("; maxNumWorkers:%d", maxNumWorkers)); - } - if (autoscalingAlgorithm != DEFAULT.autoscalingAlgorithm) { - sb.append(String.format("; autosalingAlgorithms:%s", autoscalingAlgorithm)); - } if (sourceType != DEFAULT.sourceType) { sb.append(String.format("; sourceType:%s", sourceType)); } @@ -594,9 +515,6 @@ public String toShortString() { if (fanout != DEFAULT.fanout) { sb.append(String.format("; fanout:%d", fanout)); } - if (!experiments.equals(DEFAULT.experiments)) { - sb.append(String.format("; experiments:%s", experiments)); - } if (occasionalDelaySec != DEFAULT.occasionalDelaySec) { sb.append(String.format("; occasionalDelaySec:%d", occasionalDelaySec)); } @@ -638,14 +556,13 @@ public static NexmarkConfiguration fromString(String string) throws IOException @Override public int hashCode() { - return Objects.hash(workerMachineType, numWorkers, maxNumWorkers, autoscalingAlgorithm, - streaming, query, logResults, assertCorrectness, sourceType, sinkType, pubSubMode, + return Objects.hash(query, logResults, assertCorrectness, sourceType, sinkType, pubSubMode, numEvents, numPreloadEvents, numEventGenerators, qpsShape, firstEventQps, nextEventQps, qpsPeriodSec, preloadEventQps, isRateLimited, useWallclockEventTime, avgPersonByteSize, avgAuctionByteSize, avgBidByteSize, hotAuctionRatio, hotSellersRatio, hotBiddersRatio, windowSizeSec, windowPeriodSec, watermarkHoldbackSec, numInFlightAuctions, numActivePeople, justModelResultRate, coderStrategy, cpuDelayMs, diskBusyBytes, auctionSkip, fanout, - experiments, occasionalDelaySec, probDelayedEvent, maxLogEvents, usePubsubPublishTime, + occasionalDelaySec, probDelayedEvent, maxLogEvents, usePubsubPublishTime, outOfOrderGroupSize); } @@ -661,18 +578,12 @@ public boolean equals(Object obj) { return false; } NexmarkConfiguration other = (NexmarkConfiguration) obj; - if (!workerMachineType.equals(other.workerMachineType)) { - return false; - } if (assertCorrectness != other.assertCorrectness) { return false; } if (auctionSkip != other.auctionSkip) { return false; } - if (autoscalingAlgorithm != other.autoscalingAlgorithm) { - return false; - } if (avgAuctionByteSize != other.avgAuctionByteSize) { return false; } @@ -691,13 +602,6 @@ public boolean equals(Object obj) { if (diskBusyBytes != other.diskBusyBytes) { return false; } - if (experiments == null) { - if (other.experiments != null) { - return false; - } - } else if (!experiments.equals(other.experiments)) { - return false; - } if (fanout != other.fanout) { return false; } @@ -716,9 +620,6 @@ public boolean equals(Object obj) { if (isRateLimited != other.isRateLimited) { return false; } - if (streaming != other.streaming) { - return false; - } if (justModelResultRate != other.justModelResultRate) { return false; } @@ -728,9 +629,6 @@ public boolean equals(Object obj) { if (maxLogEvents != other.maxLogEvents) { return false; } - if (maxNumWorkers != other.maxNumWorkers) { - return false; - } if (nextEventQps != other.nextEventQps) { return false; } @@ -749,9 +647,6 @@ public boolean equals(Object obj) { if (numPreloadEvents != other.numPreloadEvents) { return false; } - if (numWorkers != other.numWorkers) { - return false; - } if (occasionalDelaySec != other.occasionalDelaySec) { return false; } diff --git a/integration/java/src/main/java/com/google/cloud/dataflow/integration/nexmark/NexmarkDriver.java b/integration/java/src/main/java/com/google/cloud/dataflow/integration/nexmark/NexmarkDriver.java deleted file mode 100644 index e2289a6c838c..000000000000 --- a/integration/java/src/main/java/com/google/cloud/dataflow/integration/nexmark/NexmarkDriver.java +++ /dev/null @@ -1,625 +0,0 @@ -/* - * Copyright (C) 2015 Google Inc. - * - * Licensed under the Apache License, Version 2.0 (the "License"); you may not - * use this file except in compliance with the License. You may obtain a copy of - * the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT - * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the - * License for the specific language governing permissions and limitations under - * the License. - */ - -package com.google.cloud.dataflow.integration.nexmark; - -import com.google.cloud.dataflow.sdk.options.DataflowPipelineOptions; -import com.google.cloud.dataflow.sdk.options.Default; -import com.google.cloud.dataflow.sdk.options.Description; -import com.google.cloud.dataflow.sdk.options.PipelineOptionsFactory; -import com.google.common.collect.Sets; - -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.' These are 8 queries in Oracle's 'Continuous Query - * Language' (CQL) 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 NexmarkDriver { - /** - * Command line flags. - */ - public interface Options extends DataflowPipelineOptions { - @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("Avro input file pattern; only valid if source type is avro") - @Nullable - String getInputFilePrefix(); - - void setInputFilePrefix(String filepattern); - - @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("Approximate number of events to generate." - + "Zero for effectively unlimited in streaming mode.") - @Nullable - Long getNumEvents(); - - void setNumEvents(Long numEvents); - - @Description("Number of events to generate at the special pre-load rate. This is useful " - + "for generating a backlog of events on pub/sub before the main query begins.") - @Nullable - Long getNumPreloadEvents(); - - void setNumPreloadEvents(Long numPreloadEvents); - - @Description("Number of unbounded sources to create events.") - @Nullable - Integer getNumEventGenerators(); - - void setNumEventGenerators(Integer numEventGenerators); - - @Description("Shape of event qps curve.") - @Nullable - NexmarkUtils.QpsShape getQpsShape(); - - void setQpsShape(NexmarkUtils.QpsShape qpsShape); - - @Description("Initial overall event qps.") - @Nullable - Integer getFirstEventQps(); - - void setFirstEventQps(Integer firstEventQps); - - @Description("Next overall event qps.") - @Nullable - Integer getNextEventQps(); - - void setNextEventQps(Integer nextEventQps); - - @Description("Overall period of qps shape, in seconds.") - @Nullable - Integer getQpsPeriodSec(); - - void setQpsPeriodSec(Integer qpsPeriodSec); - - @Description("Overall event qps while pre-loading. " - + "Typcially as large as possible for given pub/sub quota.") - @Nullable - Integer getPreloadEventQps(); - - void setPreloadEventQps(Integer preloadEventQps); - - @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 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("If true, make sure all topics, subscriptions and gcs filenames are unique.") - @Default.Boolean(true) - boolean getUniqify(); - - void setUniqify(boolean uniqify); - - @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 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("If false, do not add the Monitor and Snoop transforms.") - @Nullable - Boolean getDebug(); - void setDebug(Boolean value); - } - - /** - * Entry point. - * - * @throws IOException - * @throws InterruptedException - */ - public static void main(String[] args) throws IOException, InterruptedException { - // Gather command line args, baseline, configurations, etc. - Options options = PipelineOptionsFactory.fromArgs(args).withValidation().as(Options.class); - NexmarkRunner runner = - new NexmarkRunner(options, options.getOutputPath(), options.getPubsubTopic(), - options.getMonitorJobs(), options.getUniqify(), options.getManageResources()); - Instant start = Instant.now(); - Map baseline = loadBaseline(options.getBaselineFilename()); - Map actual = new LinkedHashMap<>(); - Iterable configurations = - Sets.newLinkedHashSet(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); - } - } - - System.exit(successful ? 0 : 1); - } - - /** - * Append the pair of {@code configuration} and {@code perf} to perf file. - * - * @throws IOException - */ - private static void appendPerf(@Nullable String perfFilename, NexmarkConfiguration configuration, - NexmarkPerf perf) throws IOException { - 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()); - Files.write(Paths.get(perfFilename), lines, StandardCharsets.UTF_8, StandardOpenOption.CREATE, - StandardOpenOption.APPEND); - NexmarkUtils.console(null, "appended results to perf file %s.", perfFilename); - } - - /** - * Load the baseline perf. - * - * @throws IOException - */ - @Nullable - private static Map loadBaseline( - @Nullable String baselineFilename) throws IOException { - if (baselineFilename == null) { - return null; - } - Map baseline = new LinkedHashMap<>(); - List lines = Files.readAllLines(Paths.get(baselineFilename), StandardCharsets.UTF_8); - 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( - null, "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) throws IOException { - 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("NexmarkRunner 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) { - Files.write(Paths.get(summaryFilename), lines, StandardCharsets.UTF_8, - StandardOpenOption.CREATE, StandardOpenOption.APPEND); - NexmarkUtils.console(null, "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) throws IOException { - 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("];"); - - Files.write(Paths.get(javascriptFilename), lines, StandardCharsets.UTF_8, - StandardOpenOption.CREATE, StandardOpenOption.TRUNCATE_EXISTING); - NexmarkUtils.console(null, "saved javascript to file %s.", javascriptFilename); - } -} - diff --git a/integration/java/src/main/java/com/google/cloud/dataflow/integration/nexmark/NexmarkGoogleDriver.java b/integration/java/src/main/java/com/google/cloud/dataflow/integration/nexmark/NexmarkGoogleDriver.java new file mode 100644 index 000000000000..7da5774eb27a --- /dev/null +++ b/integration/java/src/main/java/com/google/cloud/dataflow/integration/nexmark/NexmarkGoogleDriver.java @@ -0,0 +1,301 @@ +/* + * Copyright (C) 2015 Google Inc. + * + * Licensed under the Apache License, Version 2.0 (the "License"); you may not + * use this file except in compliance with the License. You may obtain a copy of + * the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT + * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the + * License for the specific language governing permissions and limitations under + * the License. + */ + +package com.google.cloud.dataflow.integration.nexmark; + +import com.google.cloud.dataflow.sdk.options.DataflowPipelineOptions; +import com.google.cloud.dataflow.sdk.options.PipelineOptionsFactory; +import com.google.common.collect.Sets; + +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/ + */ +class NexmarkGoogleDriver { + /** + * Command line flags. + */ + public interface NexmarkGoogleOptions extends Options, DataflowPipelineOptions { + } + + /** + * Entry point. + * + * @throws IOException + * @throws InterruptedException + */ + public static void main(String[] args) throws IOException, InterruptedException { + // Gather command line args, baseline, configurations, etc. + NexmarkGoogleOptions options = PipelineOptionsFactory.fromArgs(args) + .withValidation() + .as(NexmarkGoogleOptions.class); + NexmarkGoogleRunner runner = + new NexmarkGoogleRunner(options, options.getOutputPath(), options.getPubsubTopic(), + options.getMonitorJobs(), options.getUniqify(), + options.getManageResources()); + 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); + } + } + + System.exit(successful ? 0 : 1); + } + + /** + * Append the pair of {@code configuration} and {@code perf} to perf file. + * + * @throws IOException + */ + private static void appendPerf( + @Nullable String perfFilename, NexmarkConfiguration configuration, + NexmarkPerf perf) throws IOException { + 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()); + Files.write(Paths.get(perfFilename), lines, StandardCharsets.UTF_8, StandardOpenOption.CREATE, + StandardOpenOption.APPEND); + NexmarkUtils.console(null, "appended results to perf file %s.", perfFilename); + } + + /** + * Load the baseline perf. + * + * @throws IOException + */ + @Nullable + private static Map loadBaseline( + @Nullable String baselineFilename) throws IOException { + if (baselineFilename == null) { + return null; + } + Map baseline = new LinkedHashMap<>(); + List lines = Files.readAllLines(Paths.get(baselineFilename), StandardCharsets.UTF_8); + 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( + null, "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) throws IOException { + 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) { + Files.write(Paths.get(summaryFilename), lines, StandardCharsets.UTF_8, + StandardOpenOption.CREATE, StandardOpenOption.APPEND); + NexmarkUtils.console(null, "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) throws IOException { + 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("];"); + + Files.write(Paths.get(javascriptFilename), lines, StandardCharsets.UTF_8, + StandardOpenOption.CREATE, StandardOpenOption.TRUNCATE_EXISTING); + NexmarkUtils.console(null, "saved javascript to file %s.", javascriptFilename); + } +} + diff --git a/integration/java/src/main/java/com/google/cloud/dataflow/integration/nexmark/NexmarkRunner.java b/integration/java/src/main/java/com/google/cloud/dataflow/integration/nexmark/NexmarkGoogleRunner.java similarity index 82% rename from integration/java/src/main/java/com/google/cloud/dataflow/integration/nexmark/NexmarkRunner.java rename to integration/java/src/main/java/com/google/cloud/dataflow/integration/nexmark/NexmarkGoogleRunner.java index bcda95d1fa4e..f8bc3968268e 100644 --- a/integration/java/src/main/java/com/google/cloud/dataflow/integration/nexmark/NexmarkRunner.java +++ b/integration/java/src/main/java/com/google/cloud/dataflow/integration/nexmark/NexmarkGoogleRunner.java @@ -28,6 +28,7 @@ import com.google.cloud.dataflow.sdk.io.BigQueryIO; import com.google.cloud.dataflow.sdk.io.PubsubIO; import com.google.cloud.dataflow.sdk.io.TextIO; +import com.google.cloud.dataflow.sdk.options.DataflowPipelineOptions; import com.google.cloud.dataflow.sdk.runners.AggregatorRetrievalException; import com.google.cloud.dataflow.sdk.runners.BlockingDataflowPipelineRunner; import com.google.cloud.dataflow.sdk.runners.DataflowPipelineJob; @@ -62,9 +63,9 @@ import javax.annotation.Nullable; /** - * Run a query according to specific set of options. + * Run a query according to specific set of options on Google Dataflow. */ -class NexmarkRunner { +class NexmarkGoogleRunner { /** * How long to let streaming pipeline run after all events have been generated and we've * seen no activity. @@ -99,7 +100,7 @@ class NexmarkRunner { // Following is valid over all runs. - private final NexmarkDriver.Options options; + private final NexmarkGoogleDriver.NexmarkGoogleOptions options; @Nullable private final String outputPath; @@ -110,7 +111,9 @@ class NexmarkRunner { // Following is valid per-run only. - /** Which configuration should we run. */ + /** + * Which configuration should we run. + */ @Nullable private NexmarkConfiguration configuration; @@ -120,33 +123,48 @@ class NexmarkRunner { @Nullable private PubsubHelper pubsub; - /** Pipeline 'result' for the publishing pipeline if in pub/sub COMBINED mode. */ + /** + * Pipeline 'result' for the publishing pipeline if in pub/sub COMBINED mode. + */ @Nullable private PipelineResult publisherResult; - /** Result for the main query pipeline. */ + /** + * Result for the main query pipeline. + */ @Nullable private PipelineResult mainResult; - /** Monitor for published events if in pub/sub COMBINED mode. */ + /** + * Monitor for published events if in pub/sub COMBINED mode. + */ @Nullable private Monitor publisherMonitor; - /** Query name. */ + /** + * Query name. + */ @Nullable private String queryName; - /** If sending events via pub/sub, the full topic name to use. */ + /** + * If sending events via pub/sub, the full topic name to use. + */ @Nullable private String inputTopic; - /** If true, make sure all topic, subscription and gcs file names are unique. */ + /** + * If true, make sure all topic, subscription and gcs file names are unique. + */ private final boolean uniqify; - /** If true, manage the creation and cleanup of topics, subscriptions and gcs files. */ + /** + * If true, manage the creation and cleanup of topics, subscriptions and gcs files. + */ private final boolean manageResources; - public NexmarkRunner(NexmarkDriver.Options options, @Nullable String outputPath, + public NexmarkGoogleRunner( + NexmarkGoogleDriver.NexmarkGoogleOptions options, @Nullable String outputPath, String pubsubTopic, boolean monitorJobs, boolean uniqify, boolean manageResources) { this.options = options; this.outputPath = outputPath != null && outputPath.isEmpty() ? null : outputPath; @@ -163,7 +181,7 @@ public NexmarkRunner(NexmarkDriver.Options options, @Nullable String outputPath, */ private PubsubHelper getPubsub() throws IOException { if (pubsub == null) { - pubsub = PubsubHelper.create(options); + pubsub = PubsubHelper.create(options, options.getProject()); } return pubsub; } @@ -185,10 +203,10 @@ private PCollection createSyntheticSource(Pipeline p) { private PubsubIO.Write.Bound pubsubEventSink() { PubsubIO.Write.Bound io = PubsubIO.Write.named(queryName + ".Write(" + inputTopic + ")") - .topic(inputTopic) - .idLabel(NexmarkUtils.PUBSUB_ID) - .withCoder(Event.CODER) - .named(queryName + ".PubsubSourceWrite"); + .topic(inputTopic) + .idLabel(NexmarkUtils.PUBSUB_ID) + .withCoder(Event.CODER) + .named(queryName + ".PubsubSourceWrite"); if (!configuration.usePubsubPublishTime) { io = io.timestampLabel(NexmarkUtils.PUBSUB_TIMESTAMP); } @@ -201,10 +219,10 @@ private PubsubIO.Write.Bound pubsubEventSink() { private PubsubIO.Read.Bound pubsubEventSource(String subscription) { PubsubIO.Read.Bound io = PubsubIO.Read.named(queryName + ".Read(" + subscription + ")") - .subscription(subscription) - .idLabel(NexmarkUtils.PUBSUB_ID) - .withCoder(Event.CODER) - .named(queryName + ".PubsubSourceRead"); + .subscription(subscription) + .idLabel(NexmarkUtils.PUBSUB_ID) + .withCoder(Event.CODER) + .named(queryName + ".PubsubSourceRead"); if (!configuration.usePubsubPublishTime) { io = io.timestampLabel(NexmarkUtils.PUBSUB_TIMESTAMP); } @@ -217,15 +235,34 @@ private PubsubIO.Read.Bound pubsubEventSource(String subscription) { private PubsubIO.Write.Bound pubsubResultSink(String topic) { PubsubIO.Write.Bound io = PubsubIO.Write.named(queryName + ".Write(" + topic + ")") - .topic(topic) - .idLabel(NexmarkUtils.PUBSUB_ID) - .named(queryName + ".PubsubSinkWrite"); + .topic(topic) + .idLabel(NexmarkUtils.PUBSUB_ID) + .named(queryName + ".PubsubSinkWrite"); if (!configuration.usePubsubPublishTime) { io = io.timestampLabel(NexmarkUtils.PUBSUB_TIMESTAMP); } return io; } + /** + * Return number of cores for configuration's machine type. + */ + private int coresPerWorker(DataflowPipelineOptions options) { + 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; + } + } + /** * Return source of events for this run, or null if we are simply publishing events * to pub/sub. @@ -246,10 +283,11 @@ private PCollection createSource(Pipeline p, long now) "If sourceType is AVRO, --inputFilePrefix must be specified."); } PCollection preTimestamp = p.apply(AvroIO.Read.named("ReadFromAvro") - .from(options.getInputFilePrefix() + "*.avro") - .withSchema(Event.class)); + .from(options.getInputFilePrefix() + + "*.avro") + .withSchema(Event.class)); source = preTimestamp.apply("adjust timestamp", - ParDo.of(NexmarkQuery.EVENT_TIMESTAMP_FROM_DATA)); + ParDo.of(NexmarkQuery.EVENT_TIMESTAMP_FROM_DATA)); break; case PUBSUB: if (pubsubTopic == null) { @@ -332,11 +370,12 @@ private PCollection createSource(Pipeline p, long now) String appName = options.getAppName(); options.setJobName("p-" + jobName); options.setAppName("p-" + appName); + int coresPerWorker = coresPerWorker(options); int eventGeneratorWorkers = - (configuration.numEventGenerators + configuration.coresPerWorker() - 1) - / configuration.coresPerWorker(); - options.setMaxNumWorkers(Math.min(configuration.maxNumWorkers, eventGeneratorWorkers)); - options.setNumWorkers(Math.min(configuration.numWorkers, 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"); Pipeline q = Pipeline.create(options); createSyntheticSource(q) @@ -347,9 +386,10 @@ private PCollection createSource(Pipeline p, long now) // Suppress output of publisher job; it makes the output of the command harder to // interpret. System.setOut(new PrintStream(new OutputStream() { - @Override - public void write(int b) {} - })); + @Override + public void write(int b) { + } + })); publisherResult = q.run(); } finally { System.setOut(stdout); @@ -358,8 +398,8 @@ public void write(int b) {} "Publisher job running as " + ((DataflowPipelineJob) publisherResult).getJobId()); options.setJobName(jobName); options.setAppName(appName); - options.setMaxNumWorkers(configuration.maxNumWorkers); - options.setNumWorkers(configuration.numWorkers); + options.setMaxNumWorkers(options.getMaxNumWorkers()); + options.setNumWorkers(options.getNumWorkers()); waitForPublisherPreload(); break; } @@ -442,17 +482,22 @@ private void createSink(PCollection> results, long n break; case AVRO: NexmarkUtils.console(null, "WARNING: with --sinkType=AVRO, actual query results will be " - + "discarded.", outputPath); + + "discarded.", outputPath); break; case BIGQUERY: NexmarkUtils.console(null, "Writing events to BigQuery table %s", outputPath); - formattedResults.apply(ParDo.of(new StringToTableRow())) + formattedResults + .apply(ParDo.of(new StringToTableRow())) .apply(BigQueryIO.Write.named("WriteBigQuery(Events)") - .to(options.getProject() + ":nexmark.table_" - + new Random().nextInt(Integer.MAX_VALUE)) - .withSchema(new TableSchema().setFields(new ArrayList() { - { add(new TableFieldSchema().setName("event").setType("STRING")); } - }))); + .to(options.getProject() + ":nexmark.table_" + + new Random().nextInt(Integer.MAX_VALUE)) + .withSchema(new TableSchema().setFields( + new ArrayList() { + { + add(new TableFieldSchema().setName("event") + .setType("STRING")); + } + }))); break; case COUNT_ONLY: // Short-circuited above. @@ -471,14 +516,14 @@ public void processElement(ProcessContext c) { /** * Sink all raw Events in {@code source} to {@code this.outputPath}. - * + *

* 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. + *
  • {@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. @@ -488,28 +533,28 @@ private void sinkToAvro(final PCollection source) { throw new RuntimeException("can only use Avro SinkType with --streaming=false"); } if (outputPath == null) { - throw new RuntimeException("Must supply an --outputPath if using --sinkType=AVRO"); + throw new RuntimeException("Must supply an --outputPath if using --sinkType=AVRO"); } NexmarkUtils.console(null, "Writing events in Avro to %s", outputPath); source.apply(AvroIO.Write.named("WriteAvro(Events)") - .to(outputPath + "/event") - .withSuffix(".avro") - .withSchema(Event.class)); + .to(outputPath + "/event") + .withSuffix(".avro") + .withSchema(Event.class)); source.apply(NexmarkQuery.JUST_BIDS) - .apply(AvroIO.Write.named("WriteAvro(Bids)") - .to(outputPath + "/bid") - .withSuffix(".avro") - .withSchema(Bid.class)); + .apply(AvroIO.Write.named("WriteAvro(Bids)") + .to(outputPath + "/bid") + .withSuffix(".avro") + .withSchema(Bid.class)); source.apply(NexmarkQuery.JUST_NEW_AUCTIONS) - .apply(AvroIO.Write.named("WriteAvro(Auctions)") - .to(outputPath + "/auction") - .withSuffix(".avro") - .withSchema(Auction.class)); + .apply(AvroIO.Write.named("WriteAvro(Auctions)") + .to(outputPath + "/auction") + .withSuffix(".avro") + .withSchema(Auction.class)); source.apply(NexmarkQuery.JUST_NEW_PERSONS) - .apply(AvroIO.Write.named("WriteAvro(People)") - .to(outputPath + "/person") - .withSuffix(".avro") - .withSchema(Person.class)); + .apply(AvroIO.Write.named("WriteAvro(People)") + .to(outputPath + "/person") + .withSuffix(".avro") + .withSchema(Person.class)); } /** @@ -543,20 +588,28 @@ public NexmarkPerf run(NexmarkConfiguration runConfiguration) return null; } - List queries = Arrays.asList(new NexmarkQuery[] {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)}); + 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)); NexmarkQuery query = queries.get(configuration.query); queryName = query.getName(); List models = Arrays.asList( - new NexmarkQueryModel[] {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}); + 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); NexmarkQueryModel model = models.get(configuration.query); if (configuration.justModelResultRate) { if (model == null) { @@ -566,15 +619,6 @@ public NexmarkPerf run(NexmarkConfiguration runConfiguration) return null; } - // Copy configuration into option for the few parameters which are shared. - // (If these have been set on the command line then those values will have - // been copied into the configuration, and so these assignments will have no effect.) - options.setWorkerMachineType(configuration.workerMachineType); - options.setNumWorkers(configuration.numWorkers); - options.setMaxNumWorkers(configuration.maxNumWorkers); - options.setAutoscalingAlgorithm(configuration.autoscalingAlgorithm); - options.setStreaming(configuration.streaming); - options.setExperiments(configuration.experiments); long now = System.currentTimeMillis(); Pipeline p = Pipeline.create(options); NexmarkUtils.setupPipeline(configuration.coderStrategy, p); @@ -587,7 +631,7 @@ public NexmarkPerf run(NexmarkConfiguration runConfiguration) if (source != null) { if (monitorJobs && options.getRunner() != DataflowPipelineRunner.class) { throw new RuntimeException("can only use --monitorJobs=true if also " - + "using --runner=DataflowPipelineRunner"); + + "using --runner=DataflowPipelineRunner"); } // Optionally sink events in Avro format @@ -606,6 +650,8 @@ public NexmarkPerf run(NexmarkConfiguration runConfiguration) } } ((Query10) query).setOutputPath(path); + ((Query10) query).setMaxNumWorkers( + Math.max(options.getMaxNumWorkers(), options.getNumWorkers())); if (path != null && manageResources) { pathsToDelete.add(path + "/**"); } @@ -660,8 +706,9 @@ private void modelResultRates(NexmarkQueryModel model) { NexmarkUtils.console(null, "Query%d: only %d samples", model.configuration.query, n); } else { NexmarkUtils.console(null, "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)); + 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)); } } @@ -694,7 +741,7 @@ private void waitForPublisherPreload() throws IOException, InterruptedException PipelineResult.State state = job.getState(); long numEvents = getLong(job, publisherMonitor.getElementCounter()); NexmarkUtils.console(null, "%s publisher (waiting for %d events, seen %d so far)", state, - configuration.numPreloadEvents, numEvents); + configuration.numPreloadEvents, numEvents); if (numEvents >= 0 && numEvents >= configuration.numPreloadEvents) { NexmarkUtils.console(null, "publisher preload done"); @@ -753,8 +800,8 @@ private NexmarkPerf monitor(NexmarkQuery query) throws IOException, InterruptedE Instant start = Instant.now(); Instant end = options.getRunningTimeMinutes() != null - ? start.plus(Duration.standardMinutes(options.getRunningTimeMinutes())) - : new Instant(Long.MAX_VALUE); + ? start.plus(Duration.standardMinutes(options.getRunningTimeMinutes())) + : new Instant(Long.MAX_VALUE); Instant lastActivity = null; NexmarkPerf perf = null; boolean waitingForShutdown = false; @@ -784,14 +831,14 @@ private NexmarkPerf monitor(NexmarkQuery query) throws IOException, InterruptedE lastActivity = now; } - if (configuration.streaming && !waitingForShutdown) { + if (options.isStreaming() && !waitingForShutdown) { Duration quietFor = new Duration(lastActivity, now); if (query.getFatalCount() != null && getLong(job, query.getFatalCount()) > 0) { NexmarkUtils.console(now, "job has fatal errors, cancelling."); errors.add(String.format("Pipeline reported %s fatal errors", query.getFatalCount())); waitingForShutdown = true; } else if (configuration.numEvents > 0 && currPerf.numEvents == configuration.numEvents - && currPerf.numResults >= 0 && quietFor.isLongerThan(DONE_DELAY)) { + && currPerf.numResults >= 0 && quietFor.isLongerThan(DONE_DELAY)) { NexmarkUtils.console(now, "streaming query appears to have finished, cancelling job."); waitingForShutdown = true; } else if (quietFor.isLongerThan(STUCK_TERMINATE_DELAY)) { @@ -868,6 +915,7 @@ private NexmarkPerf monitor(NexmarkQuery query) throws IOException, InterruptedE return perf; } + enum MetricType { SYSTEM_WATERMARK, DATA_WATERMARK, @@ -887,17 +935,17 @@ private MetricType getMetricType(MetricUpdate metric) { /** * Check that watermarks are not too far behind. - * + *

*

Returns a list of errors detected. */ private List checkWatermarks(DataflowPipelineJob job, boolean watermarksExpected) { List errors = new ArrayList<>(); try { JobMetrics metricResponse = job.getDataflowClient() - .projects() - .jobs() - .getMetrics(job.getProjectId(), job.getJobId()) - .execute(); + .projects() + .jobs() + .getMetrics(job.getProjectId(), job.getJobId()) + .execute(); List metrics = metricResponse.getMetrics(); if (metrics != null) { boolean foundWatermarks = false; @@ -925,7 +973,7 @@ private List checkWatermarks(DataflowPipelineJob job, boolean watermarks 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); + metric.getName().getName(), value, updateTime, threshold); errors.add(msg); NexmarkUtils.console(null, msg); } @@ -947,7 +995,8 @@ private List checkWatermarks(DataflowPipelineJob job, boolean watermarks /** * Return the current performance given {@code eventMonitor} and {@code resultMonitor}. */ - private NexmarkPerf currentPerf(Instant start, Instant now, DataflowPipelineJob job, + private NexmarkPerf currentPerf( + Instant start, Instant now, DataflowPipelineJob job, List snapshots, Monitor eventMonitor, Monitor resultMonitor) { NexmarkPerf perf = new NexmarkPerf(); @@ -1045,7 +1094,7 @@ private NexmarkPerf currentPerf(Instant start, Instant now, DataflowPipelineJob * store it in {@code perf} if found. */ private void captureSteadyState(NexmarkPerf perf, List snapshots) { - if (!configuration.streaming) { + if (!options.isStreaming()) { return; } @@ -1082,7 +1131,9 @@ private void captureSteadyState(NexmarkPerf perf, List defaultConf() { List configurations = new ArrayList<>(); @@ -96,22 +60,14 @@ private static List defaultConf() { private static List smoke() { List configurations = new ArrayList<>(); for (int query = 0; query <= 10; query++) { - for (int streaming = 0; streaming <= 1; streaming++) { - NexmarkConfiguration configuration = NexmarkConfiguration.DEFAULT.clone(); - configuration.query = query; - configuration.streaming = streaming > 0; - configuration.numEvents = 100_000; - if (query == 7) { - // ###################################################################### - // Currently broken. b/22932773. - // ###################################################################### - configuration.numEvents = -1; - } else if (query == 4 || query == 6 || query == 9) { - // Scale back so overall runtimes are reasonably close across all queries. - configuration.numEvents /= 10; - } - configurations.add(configuration); + 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; } @@ -122,8 +78,6 @@ private static List stress() { if (configuration.numEvents >= 0) { configuration.numEvents *= 1000; } - configuration.numWorkers *= 5; - configuration.maxNumWorkers *= 5; } return configurations; } @@ -134,8 +88,6 @@ private static List fullThrottle() { if (configuration.numEvents >= 0) { configuration.numEvents *= 1000; } - configuration.numWorkers *= 25; - configuration.maxNumWorkers *= 25; } return configurations; } @@ -152,201 +104,9 @@ private static List correct() { return configurations; } - private static List justModelResultRate() { - List configurations = new ArrayList<>(); - for (int query = 0; query <= 10; query++) { - if (query == 10) { - // No model for Query 10. - continue; - } - NexmarkConfiguration configuration = NexmarkConfiguration.DEFAULT.clone(); - configuration.query = query; - configuration.justModelResultRate = true; - configurations.add(configuration); - } - return configurations; - } - - private static List coders() { - List configurations = new ArrayList<>(); - for (int streaming = 0; streaming <= 1; streaming++) { - for (NexmarkUtils.CoderStrategy coderStrategy : NexmarkUtils.CoderStrategy.values()) { - NexmarkConfiguration configuration = NexmarkConfiguration.DEFAULT.clone(); - configuration.query = 0; - configuration.streaming = streaming > 0; - configuration.numEvents = 1_000_000L; - configuration.coderStrategy = coderStrategy; - configurations.add(configuration); - } - } - return configurations; - } - - private static List bigLogger() { - NexmarkConfiguration configuration = NexmarkConfiguration.DEFAULT.clone(); - int totalCores = 400; - int generatorCores = 24; - int queryCores = totalCores - generatorCores; - int maxDisksPerWorker = 15; - configuration.maxNumWorkers = queryCores / configuration.coresPerWorker(); - configuration.numWorkers = - Math.max(1, (configuration.maxNumWorkers + maxDisksPerWorker - 1) / maxDisksPerWorker); - configuration.numEventGenerators = generatorCores; - - configuration.query = 10; - configuration.streaming = true; - configuration.isRateLimited = true; - configuration.experiments.add("enable_ppm"); - configuration.experiments.add("enable_streaming_scaling"); - configuration.sourceType = SourceType.PUBSUB; - configuration.autoscalingAlgorithm = - DataflowPipelineWorkerPoolOptions.AutoscalingAlgorithmType.THROUGHPUT_BASED; - configuration.numEvents = 0; // as many as possible without overflow. - configuration.numPreloadEvents = 1_000_000L; - configuration.avgPersonByteSize = 500; - configuration.avgAuctionByteSize = 500; - configuration.avgBidByteSize = 500; - configuration.windowSizeSec = 60 * 20; - configuration.occasionalDelaySec = 60 * 60; - configuration.probDelayedEvent = 0.000001; - // Due to poor interaction of pub/sub and custom timestamps we need to make - // event timestamps as close as possible to wallclock time. - configuration.useWallclockEventTime = true; - configuration.qpsShape = NexmarkUtils.QpsShape.SINE; - configuration.firstEventQps = 100_000; - configuration.nextEventQps = 5_000; - configuration.qpsPeriodSec = 60 * 60; - configuration.preloadEventQps = 100_000; - // At 100k qps and 94 workers (ie 188 log shards) a pane will fill every 16 mins. - configuration.maxLogEvents = 500_000; - - configuration.usePubsubPublishTime = false; - - List configurations = new ArrayList<>(); - configurations.add(configuration); - return configurations; - } - - private static List smallLogger() { - NexmarkConfiguration configuration = NexmarkConfiguration.DEFAULT.clone(); - configuration.maxNumWorkers = 1; - configuration.numWorkers = 1; - configuration.numEventGenerators = 1; - - configuration.query = 10; - configuration.streaming = true; - configuration.isRateLimited = true; - configuration.experiments.add("enable_ppm"); - configuration.experiments.add("enable_streaming_scaling"); - configuration.sourceType = SourceType.PUBSUB; - configuration.autoscalingAlgorithm = - DataflowPipelineWorkerPoolOptions.AutoscalingAlgorithmType.NONE; - configuration.numEvents = 0; // as many as possible without overflow. - configuration.numPreloadEvents = 0L; - configuration.avgPersonByteSize = 500; - configuration.avgAuctionByteSize = 500; - configuration.avgBidByteSize = 500; - configuration.windowSizeSec = 30; - configuration.occasionalDelaySec = 360; - configuration.probDelayedEvent = 0.001; - configuration.useWallclockEventTime = true; - configuration.firstEventQps = 100; - configuration.nextEventQps = 100; - configuration.maxLogEvents = 15000; - - List configurations = new ArrayList<>(); - configurations.add(configuration); - return configurations; - } - - private static List longRunningSessions() { - NexmarkConfiguration configuration = NexmarkConfiguration.DEFAULT.clone(); - configuration.maxNumWorkers = 25; - configuration.numWorkers = 25; - configuration.numEventGenerators = 25; - - configuration.query = 11; - configuration.sourceType = SourceType.PUBSUB; - configuration.autoscalingAlgorithm = - DataflowPipelineWorkerPoolOptions.AutoscalingAlgorithmType.NONE; - configuration.streaming = true; - configuration.isRateLimited = true; - configuration.firstEventQps = 50_000; - configuration.nextEventQps = 25_000; - configuration.numEvents = 0L; - configuration.outOfOrderGroupSize = 10_000L; - configuration.occasionalDelaySec = 60; - configuration.probDelayedEvent = 0.00001; - configuration.numInFlightAuctions = 1000; - configuration.windowSizeSec = 5; // 5sec session gap. - configuration.maxLogEvents = 1000; // break sessions at around 1000 events. - - List configurations = new ArrayList<>(); - configurations.add(configuration); - return configurations; - } - - private static List longRunningLogger() { - NexmarkConfiguration configuration = NexmarkConfiguration.DEFAULT.clone(); - configuration.maxNumWorkers = 25; - configuration.numWorkers = 25; - configuration.numEventGenerators = 10; - - configuration.query = 10; - configuration.streaming = true; - configuration.isRateLimited = true; - configuration.sourceType = SourceType.PUBSUB; - configuration.numEvents = 0; // as many as possible without overflow. - configuration.numPreloadEvents = 0L; - configuration.avgPersonByteSize = 500; - configuration.avgAuctionByteSize = 500; - configuration.avgBidByteSize = 500; - configuration.windowSizeSec = 300; - configuration.occasionalDelaySec = 360; - configuration.probDelayedEvent = 0.001; - configuration.useWallclockEventTime = true; - configuration.firstEventQps = 30000; - configuration.nextEventQps = 30000; - configuration.maxLogEvents = 15000; - - List configurations = new ArrayList<>(); - configurations.add(configuration); - return configurations; - } - - private static List longRunningBigQuery() { - NexmarkConfiguration configuration = NexmarkConfiguration.DEFAULT.clone(); - configuration.maxNumWorkers = 25; - configuration.numWorkers = 25; - configuration.numEventGenerators = 10; - - configuration.query = 0; - configuration.streaming = true; - configuration.isRateLimited = true; - configuration.sourceType = SourceType.PUBSUB; - configuration.sinkType = SinkType.BIGQUERY; - configuration.numEvents = 0; // as many as possible without overflow. - configuration.numPreloadEvents = 0L; - configuration.avgPersonByteSize = 500; - configuration.avgAuctionByteSize = 500; - configuration.avgBidByteSize = 500; - configuration.windowSizeSec = 300; - configuration.probDelayedEvent = 0; - configuration.useWallclockEventTime = true; - configuration.firstEventQps = 60000; - configuration.nextEventQps = 60000; - - List configurations = new ArrayList<>(); - configurations.add(configuration); - return configurations; - } - - private final boolean takeIsStreamingFromCommandLine; private final List configurations; - private NexmarkSuite( - boolean takeIsStreamingFromCommandLine, List configurations) { - this.takeIsStreamingFromCommandLine = takeIsStreamingFromCommandLine; + NexmarkSuite(List configurations) { this.configurations = configurations; } @@ -356,14 +116,9 @@ private NexmarkSuite( * the {@link #DEFAULT} suite. */ public Iterable getConfigurations(Options options) { - List results = new ArrayList<>(); + Set results = new LinkedHashSet<>(); for (NexmarkConfiguration configuration : configurations) { NexmarkConfiguration result = configuration.clone(); - if (takeIsStreamingFromCommandLine) { - // The --isStreaming flag is the ONLY one we can't distinguish between not-set - // and set to false. So only copy its value in a few special cases. - result.streaming = options.isStreaming(); - } result.overrideFromOptions(options); results.add(result); } diff --git a/integration/java/src/main/java/com/google/cloud/dataflow/integration/nexmark/NexmarkUtils.java b/integration/java/src/main/java/com/google/cloud/dataflow/integration/nexmark/NexmarkUtils.java index 524f1052d1ca..77dd3c6af44c 100644 --- a/integration/java/src/main/java/com/google/cloud/dataflow/integration/nexmark/NexmarkUtils.java +++ b/integration/java/src/main/java/com/google/cloud/dataflow/integration/nexmark/NexmarkUtils.java @@ -67,7 +67,7 @@ * Odd's 'n Ends used throughout queries and driver. */ public class NexmarkUtils { - private static final Logger LOG = LoggerFactory.getLogger(NexmarkDriver.class.getName()); + private static final Logger LOG = LoggerFactory.getLogger(NexmarkGoogleDriver.class.getName()); /** Mapper for (de)serializing JSON. */ static final ObjectMapper MAPPER = new ObjectMapper(); diff --git a/integration/java/src/main/java/com/google/cloud/dataflow/integration/nexmark/Options.java b/integration/java/src/main/java/com/google/cloud/dataflow/integration/nexmark/Options.java new file mode 100644 index 000000000000..0f1c4b023ef5 --- /dev/null +++ b/integration/java/src/main/java/com/google/cloud/dataflow/integration/nexmark/Options.java @@ -0,0 +1,355 @@ +/* + * Copyright (C) 2016 Google Inc. + * + * Licensed under the Apache License, Version 2.0 (the "License"); you may not + * use this file except in compliance with the License. You may obtain a copy of + * the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT + * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the + * License for the specific language governing permissions and limitations under + * the License. + */ + +package com.google.cloud.dataflow.integration.nexmark; + +import com.google.cloud.dataflow.sdk.options.Default; +import com.google.cloud.dataflow.sdk.options.Description; +import javax.annotation.Nullable; + +/** + * Command line flags. + */ +public interface Options { + @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("Avro input file pattern; only valid if source type is avro") + @Nullable + String getInputFilePrefix(); + + void setInputFilePrefix(String filepattern); + + @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("Approximate number of events to generate. " + + "Zero for effectively unlimited in streaming mode.") + @Nullable + Long getNumEvents(); + + void setNumEvents(Long numEvents); + + @Description("Number of events to generate at the special pre-load rate. This is useful " + + "for generating a backlog of events on pub/sub before the main query begins.") + @Nullable + Long getNumPreloadEvents(); + + void setNumPreloadEvents(Long numPreloadEvents); + + @Description("Number of unbounded sources to create events.") + @Nullable + Integer getNumEventGenerators(); + + void setNumEventGenerators(Integer numEventGenerators); + + @Description("Shape of event qps curve.") + @Nullable + NexmarkUtils.QpsShape getQpsShape(); + + void setQpsShape(NexmarkUtils.QpsShape qpsShape); + + @Description("Initial overall event qps.") + @Nullable + Integer getFirstEventQps(); + + void setFirstEventQps(Integer firstEventQps); + + @Description("Next overall event qps.") + @Nullable + Integer getNextEventQps(); + + void setNextEventQps(Integer nextEventQps); + + @Description("Overall period of qps shape, in seconds.") + @Nullable + Integer getQpsPeriodSec(); + + void setQpsPeriodSec(Integer qpsPeriodSec); + + @Description("Overall event qps while pre-loading. " + + "Typcially as large as possible for given pub/sub quota.") + @Nullable + Integer getPreloadEventQps(); + + void setPreloadEventQps(Integer preloadEventQps); + + @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 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("If true, make sure all topics, subscriptions and gcs filenames are unique.") + @Default.Boolean(true) + boolean getUniqify(); + + void setUniqify(boolean uniqify); + + @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 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("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/com/google/cloud/dataflow/integration/nexmark/Person.java b/integration/java/src/main/java/com/google/cloud/dataflow/integration/nexmark/Person.java index 89ffe67e8c4e..1f53f7d45823 100644 --- a/integration/java/src/main/java/com/google/cloud/dataflow/integration/nexmark/Person.java +++ b/integration/java/src/main/java/com/google/cloud/dataflow/integration/nexmark/Person.java @@ -41,28 +41,28 @@ public class Person implements KnownSize, Serializable { public void encode(Person value, OutputStream outStream, com.google.cloud.dataflow.sdk.coders.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()); + 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, com.google.cloud.dataflow.sdk.coders.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()); + 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); } }; diff --git a/integration/java/src/main/java/com/google/cloud/dataflow/integration/nexmark/PubsubHelper.java b/integration/java/src/main/java/com/google/cloud/dataflow/integration/nexmark/PubsubHelper.java index a6518d336ce6..ac342cd4c6e2 100644 --- a/integration/java/src/main/java/com/google/cloud/dataflow/integration/nexmark/PubsubHelper.java +++ b/integration/java/src/main/java/com/google/cloud/dataflow/integration/nexmark/PubsubHelper.java @@ -16,62 +16,61 @@ package com.google.cloud.dataflow.integration.nexmark; -import com.google.api.client.googleapis.json.GoogleJsonResponseException; -import com.google.api.client.util.BackOff; -import com.google.api.client.util.BackOffUtils; -import com.google.api.client.util.Sleeper; -import com.google.api.services.pubsub.Pubsub; -import com.google.api.services.pubsub.model.Subscription; -import com.google.api.services.pubsub.model.Topic; -import com.google.cloud.dataflow.sdk.util.AttemptBoundedExponentialBackOff; -import com.google.cloud.dataflow.sdk.util.Transport; +import com.google.cloud.dataflow.sdk.io.PubsubGrpcClient; +import com.google.cloud.dataflow.sdk.options.GcpOptions; import java.io.IOException; import java.util.ArrayList; +import java.util.Collection; import java.util.List; /** - * Helper for working with pubsub and gcs. + * Helper for working with pubsub. */ public class PubsubHelper { - /** Underlying pub/sub client. */ - private final Pubsub pubsubClient; + /** + * Underlying pub/sub client. + */ + private final PubsubGrpcClient pubsubClient; - /** Project id. */ + /** + * Project id. + */ private final String project; - /** Topics we should delete on close. */ + /** + * Topics we should delete on close. + */ private final List createdTopics; - /** Subscriptions we should delete on close. */ + /** + * Subscriptions we should delete on close. + */ private final List createdSubscriptions; - /** How to sleep in retry loops. */ - private final Sleeper sleeper; - - /** How to backoff in retry loops. */ - private final BackOff backOff; - - private PubsubHelper(Pubsub pubsubClient, String project) { + private PubsubHelper(PubsubGrpcClient pubsubClient, String project) { this.pubsubClient = pubsubClient; this.project = project; createdTopics = new ArrayList<>(); createdSubscriptions = new ArrayList<>(); - sleeper = Sleeper.DEFAULT; - backOff = new AttemptBoundedExponentialBackOff(3, 500); } - /** Create a helper. */ - public static PubsubHelper create(NexmarkDriver.Options options) throws IOException { - return new PubsubHelper(Transport.newPubsubClient(options).build(), - options.getProject()); + /** + * Create a helper. + */ + public static PubsubHelper create(GcpOptions options, String project) throws IOException { + return new PubsubHelper(PubsubGrpcClient.newClient(null, null, options), project); } - /** Return full topic name corresponding to short topic name. */ + /** + * Return full topic name corresponding to short topic name. + */ private String fullTopic(String shortTopic) { return String.format("projects/%s/topics/%s", project, shortTopic); } - /** Return full subscription name corresponding to short subscription name. */ + /** + * Return full subscription name corresponding to short subscription name. + */ private String fullSubscription(String shortSubscription) { return String.format("projects/%s/subscriptions/%s", project, shortSubscription); } @@ -79,48 +78,36 @@ private String fullSubscription(String shortSubscription) { /** * Create a topic from short name. Delete it if it already exists. Ensure the topic will be * deleted on cleanup. Return full topic name. + * * @throws InterruptedException */ public String createTopic(String shortTopic) throws IOException, InterruptedException { String topic = fullTopic(shortTopic); - while (true) { - try { - NexmarkUtils.console(null, "create topic %s", topic); - pubsubClient.projects().topics().create(topic, new Topic()).execute(); - createdTopics.add(topic); - return topic; - } catch (GoogleJsonResponseException ex) { - NexmarkUtils.console(null, "attempting to cleanup topic %s", topic); - pubsubClient.projects().topics().delete(topic).execute(); - if (!BackOffUtils.next(sleeper, backOff)) { - NexmarkUtils.console(null, "too many retries for creating topic %s", topic); - throw ex; - } - } + if (topicExists(topic)) { + NexmarkUtils.console(null, "attempting to cleanup topic %s", topic); + pubsubClient.deleteTopic(topic); } + NexmarkUtils.console(null, "create topic %s", topic); + pubsubClient.createTopic(topic); + createdTopics.add(topic); + return topic; } - /** Create a topic from short name if it does not already exist. The topic will not be + /** + * Create a topic from short name if it does not already exist. The topic will not be * deleted on cleanup. Return full topic name. - * @throws InterruptedException */ + * + * @throws InterruptedException + */ public String createOrReuseTopic(String shortTopic) throws IOException, InterruptedException { String topic = fullTopic(shortTopic); - while (true) { - try { - NexmarkUtils.console(null, "create topic %s", topic); - pubsubClient.projects().topics().create(topic, new Topic()).execute(); - return topic; - } catch (GoogleJsonResponseException ex) { - if (topicExists(shortTopic)) { - NexmarkUtils.console(null, "topic %s already exists", topic); - return topic; - } - if (!BackOffUtils.next(sleeper, backOff)) { - NexmarkUtils.console(null, "too many retries for creating/reusing topic %s", topic); - throw ex; - } - } + if (topicExists(topic)) { + NexmarkUtils.console(null, "topic %s already exists", topic); + return topic; } + NexmarkUtils.console(null, "create topic %s", topic); + pubsubClient.createTopic(topic); + return topic; } /** @@ -136,48 +123,33 @@ public String reuseTopic(String shortTopic) throws IOException { throw new RuntimeException("topic '" + topic + "' does not already exist"); } - /** Does topic corresponding to short name exist? */ + /** + * Does topic corresponding to short name exist? + */ public boolean topicExists(String shortTopic) throws IOException { String topic = fullTopic(shortTopic); - List existingTopics = - pubsubClient.projects().topics().list("projects/" + project).execute().getTopics(); - if (existingTopics != null) { - for (Topic existingTopic : existingTopics) { - if (existingTopic.getName().equals(topic)) { - return true; - } - } - } - return false; + Collection existingTopics = pubsubClient.listTopics("projects/" + project); + return existingTopics.contains(topic); } /** - * Create subscription from short name. Ensure the subscription will be deleted - * on cleanup. Return full subscription name. + * Create subscription from short name. Delete subscription if it already exists. Ensure the + * subscription will be deleted on cleanup. Return full subscription name. + * * @throws InterruptedException */ public String createSubscription(String shortTopic, String shortSubscription) throws IOException, InterruptedException { String topic = fullTopic(shortTopic); String subscription = fullSubscription(shortSubscription); - while (true) { - try { - NexmarkUtils.console(null, "create subscription %s", subscription); - pubsubClient.projects() - .subscriptions() - .create(subscription, new Subscription().setTopic(topic).setAckDeadlineSeconds(60)) - .execute(); - createdSubscriptions.add(subscription); - return subscription; - } catch (GoogleJsonResponseException ex) { - NexmarkUtils.console(null, "attempting to cleanup subscription %s", subscription); - pubsubClient.projects().subscriptions().delete(subscription).execute(); - if (!BackOffUtils.next(sleeper, backOff)) { - NexmarkUtils.console(null, "too many retries for creating subscription %s", subscription); - throw ex; - } - } + if (subscriptionExists(shortTopic, shortSubscription)) { + NexmarkUtils.console(null, "attempting to cleanup subscription %s", subscription); + pubsubClient.deleteSubscription(subscription); } + NexmarkUtils.console(null, "create subscription %s", subscription); + pubsubClient.createSubscription(subscription, topic, 60); + createdSubscriptions.add(subscription); + return subscription; } /** @@ -193,34 +165,26 @@ public String reuseSubscription(String shortTopic, String shortSubscription) thr throw new RuntimeException("subscription'" + subscription + "' does not already exist"); } - /** Does subscription corresponding to short name exist? */ + /** + * Does subscription corresponding to short name exist? + */ public boolean subscriptionExists(String shortTopic, String shortSubscription) throws IOException { String topic = fullTopic(shortTopic); String subscription = fullSubscription(shortSubscription); - List existingSubscriptions = - pubsubClient.projects() - .subscriptions() - .list("projects/" + project) - .execute() - .getSubscriptions(); - if (existingSubscriptions != null) { - for (Subscription existingSubscription : existingSubscriptions) { - if (existingSubscription.getTopic().equals(topic) - && existingSubscription.getName().equals(subscription)) { - return true; - } - } - } - return false; + Collection existingSubscriptions = + pubsubClient.listSubscriptions("projects/" + project, topic); + return existingSubscriptions.contains(subscription); } - /** Delete all the subscriptions and topics we created. */ + /** + * Delete all the subscriptions and topics we created. + */ public void cleanup() { for (String subscription : createdSubscriptions) { try { NexmarkUtils.console(null, "delete subscription %s", subscription); - pubsubClient.projects().subscriptions().delete(subscription).execute(); + pubsubClient.deleteSubscription(subscription); } catch (IOException ex) { NexmarkUtils.console(null, "could not delete subscription %s", subscription); } @@ -228,7 +192,7 @@ public void cleanup() { for (String topic : createdTopics) { try { NexmarkUtils.console(null, "delete topic %s", topic); - pubsubClient.projects().topics().delete(topic).execute(); + pubsubClient.deleteTopic(topic); } catch (IOException ex) { NexmarkUtils.console(null, "could not delete topic %s", topic); } diff --git a/integration/java/src/main/java/com/google/cloud/dataflow/integration/nexmark/Query10.java b/integration/java/src/main/java/com/google/cloud/dataflow/integration/nexmark/Query10.java index c0d17b7f2629..eb098c2b1e18 100644 --- a/integration/java/src/main/java/com/google/cloud/dataflow/integration/nexmark/Query10.java +++ b/integration/java/src/main/java/com/google/cloud/dataflow/integration/nexmark/Query10.java @@ -53,7 +53,7 @@ /** * Query "10", 'Log to sharded files' (Not in original suite.) - * + *

*

Every windowSizeSec, save all events from the last our into 2*maxWorkers log files. */ class Query10 extends NexmarkQuery { @@ -67,6 +67,11 @@ class Query10 extends NexmarkQuery { @Nullable private String outputPath; + /** + * Maximum number of workers, used to determine log sharding factor. + */ + private int maxNumWorkers; + public Query10(NexmarkConfiguration configuration) { super(configuration, "Query10"); } @@ -75,6 +80,10 @@ public void setOutputPath(@Nullable String outputPath) { this.outputPath = outputPath; } + public void setMaxNumWorkers(int maxNumWorkers) { + this.maxNumWorkers = maxNumWorkers; + } + private WritableByteChannel openGcsFile(GcsOptions options, String filename) throws IOException { WritableByteChannel channel = new GcsIOChannelFactory(options).create(filename, "text/plain"); Preconditions.checkState(channel instanceof GoogleCloudStorageWriteChannel); @@ -101,7 +110,7 @@ private String buildOutputPath( break; } return String.format("%s/%s-%s-%03d%s", outputPath, window.maxTimestamp(), key, - paneIndex, which); + paneIndex, which); } @Nullable @@ -113,191 +122,191 @@ private String buildFinalizeOutputPath(BoundedWindow window) { } private PCollection applyTyped(PCollection events) { - final int numLogShards = - Math.max(configuration.maxNumWorkers, configuration.numWorkers) * NUM_SHARDS_PER_WORKER; + 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()); + .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 shard = (int) Math.abs((long) c.element().hashCode() % numLogShards); - c.output(KV.of("dummy-" + shard, c.element())); - } - })) + @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 shard = (int) Math.abs((long) c.element().hashCode() % numLogShards); + c.output(KV.of("dummy-" + 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 - (TEN_SECONDS))))) - .discardingFiredPanes() - // Use a 1 day allowed lateness so that any forgotten hold will stall the - // pipeline for that period and be very noticable. - .withAllowedLateness(Duration.standardDays(1))) + .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 + (TEN_SECONDS))))) + .discardingFiredPanes() + // Use a 1 day allowed lateness so that any forgotten hold will stall the + // pipeline for that period and be very noticable. + .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()); + .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 key = c.element().getKey(); - NexmarkUtils.error( - "key %s with timestamp %s has %d actually late and %d on-time " - + "elements in pane %s for window %s", - key, 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 key %s", key); - unexpectedLatePaneCounter.addValue(1L); - } - if (numOnTime > 0) { - NexmarkUtils.error( - "ERROR! Have %d on-time events in late pane for key %s", - numOnTime, key); - 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 key %s", - numOnTime + numLate, key); - } - earlyCounter.addValue(1L); - } else { - onTimeCounter.addValue(1L); - } - c.output(c.element()); - } - })) + @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 key = c.element().getKey(); + NexmarkUtils.error( + "key %s with timestamp %s has %d actually late and %d on-time " + + "elements in pane %s for window %s", + key, 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 key %s", key); + unexpectedLatePaneCounter.addValue(1L); + } + if (numOnTime > 0) { + NexmarkUtils.error( + "ERROR! Have %d on-time events in late pane for key %s", + numOnTime, key); + 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 key %s", + numOnTime + numLate, key); + } + 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()); + .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 key = c.element().getKey(); - GcsOptions options = c.getPipelineOptions().as(GcsOptions.class); - NexmarkUtils.error( - "Writing key %s with record timestamp %s, window timestamp %s, pane %s", - key, c.timestamp(), window.maxTimestamp(), c.pane()); - String path = buildOutputPath(key, window, c.pane().getTiming(), c.pane() - .getIndex()); - if (path != null) { - NexmarkUtils.error("Beginning write for '%s'", path); - int n = 0; - try (OutputStream output = Channels.newOutputStream(openGcsFile(options, - path))) { - 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, path); - } - } - } - NexmarkUtils.error("Written all %d records to '%s'", n, path); - } - savedFileCounter.addValue(1L); - c.output(KV.of(null, null)); - } - })) + @ProcessElement + public void process(ProcessContext c, BoundedWindow window) throws IOException { + String key = c.element().getKey(); + GcsOptions options = c.getPipelineOptions().as(GcsOptions.class); + NexmarkUtils.error( + "Writing key %s with record timestamp %s, window timestamp %s, pane %s", + key, c.timestamp(), window.maxTimestamp(), c.pane()); + String path = buildOutputPath(key, window, c.pane().getTiming(), c.pane() + .getIndex()); + if (path != null) { + NexmarkUtils.error("Beginning write for '%s'", path); + int n = 0; + try (OutputStream output = Channels.newOutputStream(openGcsFile(options, + path))) { + 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, path); + } + } + } + NexmarkUtils.error("Written all %d records to '%s'", n, path); + } + savedFileCounter.addValue(1L); + c.output(KV.of(null, null)); + } + })) // 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()) + .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(Keys.create()) .apply( ParDo.named(name + ".Finalize") - .of(new DoFnWithContext() { - 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("finalized", new SumLongFn()); + .of(new DoFnWithContext() { + 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("finalized", 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( - "Finalize with record timestamp %s, window timestamp %s, pane %s", - c.timestamp(), window.maxTimestamp(), c.pane()); - String path = buildFinalizeOutputPath(window); - if (path != null) { - NexmarkUtils.error("Beginning write for '%s'", path); - try (WritableByteChannel output = openGcsFile(options, path)) { - output.write(ByteBuffer.wrap("FINISHED".getBytes())); - } - NexmarkUtils.error("Written '%s'", path); - } - c.output( - new Done("written for timestamp " + window.maxTimestamp().getMillis())); - finalizedCounter.addValue(1L); - } - } - })); + @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( + "Finalize with record timestamp %s, window timestamp %s, pane %s", + c.timestamp(), window.maxTimestamp(), c.pane()); + String path = buildFinalizeOutputPath(window); + if (path != null) { + NexmarkUtils.error("Beginning write for '%s'", path); + try (WritableByteChannel output = openGcsFile(options, path)) { + output.write(ByteBuffer.wrap("FINISHED".getBytes())); + } + NexmarkUtils.error("Written '%s'", path); + } + c.output( + new Done("written for timestamp " + window.maxTimestamp().getMillis())); + finalizedCounter.addValue(1L); + } + } + })); } @Override diff --git a/integration/java/src/main/java/com/google/cloud/dataflow/integration/nexmark/SellerPrice.java b/integration/java/src/main/java/com/google/cloud/dataflow/integration/nexmark/SellerPrice.java index bf32275b7c68..f85e9eb694be 100644 --- a/integration/java/src/main/java/com/google/cloud/dataflow/integration/nexmark/SellerPrice.java +++ b/integration/java/src/main/java/com/google/cloud/dataflow/integration/nexmark/SellerPrice.java @@ -40,16 +40,16 @@ public class SellerPrice implements KnownSize, Serializable { public void encode(SellerPrice value, OutputStream outStream, com.google.cloud.dataflow.sdk.coders.Coder.Context context) throws CoderException, IOException { - LONG_CODER.encode(value.seller, outStream, context.nested()); - LONG_CODER.encode(value.price, outStream, context.nested()); + LONG_CODER.encode(value.seller, outStream, Context.NESTED); + LONG_CODER.encode(value.price, outStream, Context.NESTED); } @Override public SellerPrice decode( InputStream inStream, com.google.cloud.dataflow.sdk.coders.Coder.Context context) throws CoderException, IOException { - long seller = LONG_CODER.decode(inStream, context.nested()); - long price = LONG_CODER.decode(inStream, context.nested()); + long seller = LONG_CODER.decode(inStream, Context.NESTED); + long price = LONG_CODER.decode(inStream, Context.NESTED); return new SellerPrice(seller, price); } }; diff --git a/integration/java/src/main/java/com/google/cloud/dataflow/integration/nexmark/WinningBids.java b/integration/java/src/main/java/com/google/cloud/dataflow/integration/nexmark/WinningBids.java index c8e327038784..10370d2578e7 100644 --- a/integration/java/src/main/java/com/google/cloud/dataflow/integration/nexmark/WinningBids.java +++ b/integration/java/src/main/java/com/google/cloud/dataflow/integration/nexmark/WinningBids.java @@ -156,17 +156,17 @@ public static AuctionOrBidWindowCoder of() { @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()); + 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; + 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); } From a7d43024c233e4d51b37948e3571f16958b5ff9e Mon Sep 17 00:00:00 2001 From: Mark Shields Date: Wed, 30 Mar 2016 13:04:10 -0700 Subject: [PATCH 07/14] Unused import --- .../cloud/dataflow/integration/nexmark/NexmarkGoogleDriver.java | 1 - 1 file changed, 1 deletion(-) diff --git a/integration/java/src/main/java/com/google/cloud/dataflow/integration/nexmark/NexmarkGoogleDriver.java b/integration/java/src/main/java/com/google/cloud/dataflow/integration/nexmark/NexmarkGoogleDriver.java index 7da5774eb27a..e5501e09228d 100644 --- a/integration/java/src/main/java/com/google/cloud/dataflow/integration/nexmark/NexmarkGoogleDriver.java +++ b/integration/java/src/main/java/com/google/cloud/dataflow/integration/nexmark/NexmarkGoogleDriver.java @@ -18,7 +18,6 @@ import com.google.cloud.dataflow.sdk.options.DataflowPipelineOptions; import com.google.cloud.dataflow.sdk.options.PipelineOptionsFactory; -import com.google.common.collect.Sets; import org.joda.time.Duration; import org.joda.time.Instant; From 8ebdde1ad3d4dd09e3fc425b85841dd1cbb64704 Mon Sep 17 00:00:00 2001 From: Mark Shields Date: Thu, 7 Apr 2016 12:50:37 -0700 Subject: [PATCH 08/14] Sync with google3 --- .../integration/nexmark/Generator.java | 31 +----- .../integration/nexmark/GeneratorConfig.java | 33 ++----- .../nexmark/NexmarkConfiguration.java | 38 +++----- .../nexmark/NexmarkGoogleDriver.java | 22 +++++ .../nexmark/NexmarkGoogleRunner.java | 95 ++++++++++++------- .../integration/nexmark/NexmarkSuite.java | 2 +- .../dataflow/integration/nexmark/Options.java | 33 +------ .../dataflow/integration/nexmark/Query11.java | 3 +- 8 files changed, 114 insertions(+), 143 deletions(-) diff --git a/integration/java/src/main/java/com/google/cloud/dataflow/integration/nexmark/Generator.java b/integration/java/src/main/java/com/google/cloud/dataflow/integration/nexmark/Generator.java index 3f63af214d6e..27152d29007b 100644 --- a/integration/java/src/main/java/com/google/cloud/dataflow/integration/nexmark/Generator.java +++ b/integration/java/src/main/java/com/google/cloud/dataflow/integration/nexmark/Generator.java @@ -243,12 +243,9 @@ public GeneratorConfig getCurrentConfig() { */ public GeneratorConfig splitAtEventId(long eventId) { long newMaxEvents = eventId - (config.firstEventId + config.firstEventNumber); - long newPreloadEvents = Math.min(config.numPreloadEvents, newMaxEvents); - GeneratorConfig remainConfig = - config.cloneWith(config.firstEventId, config.maxEvents - newMaxEvents, - config.numPreloadEvents - newPreloadEvents, config.firstEventNumber + newMaxEvents); - config = config.cloneWith( - config.firstEventId, newMaxEvents, newPreloadEvents, config.firstEventNumber); + GeneratorConfig remainConfig = config.cloneWith(config.firstEventId, + config.maxEvents - newMaxEvents, config.firstEventNumber + newMaxEvents); + config = config.cloneWith(config.firstEventId, newMaxEvents, config.firstEventNumber); return remainConfig; } @@ -537,28 +534,8 @@ public NextEvent nextEvent() { long watermark = config.timestampAndInterEventDelayUsForEvent(config.nextEventNumberForWatermark(numEvents)) .getKey(); - // In preload we'll emit each event at preload intervals instead of 'regular' intervals. - long preloadInterEventDelayUs = NexmarkUtils.QpsShape.SQUARE.interEventDelayUs( - config.configuration.preloadEventQps, config.configuration.numEventGenerators); // When, in wallclock time, we should emit the event. - long wallclockTimestamp; - if (numEvents < config.numPreloadEvents) { - // While preloading, emit as fast as the preload rate will allow us. - wallclockTimestamp = wallclockBaseTime + (numEvents * preloadInterEventDelayUs) / 1000L; - } else { - // Once preload is done, switch to true event time, but offset appropriately. - long wallclockOfFirstPostPreloadEvent = - wallclockBaseTime + (config.numPreloadEvents * preloadInterEventDelayUs) / 1000L; - // = wallclockBaseTime if numPreloadEvents = 0 - long timestampOfFirstPostPreloadEvent = - config.timestampAndInterEventDelayUsForEvent( - config.nextEventNumber(config.numPreloadEvents)).getKey(); - // = getCurrentConfig().baseTime if numPreloadEvents = 0 - wallclockTimestamp = - wallclockOfFirstPostPreloadEvent + (eventTimestamp - timestampOfFirstPostPreloadEvent); - // = wallclockBaseTime + (eventTimestamp - getCurrentConfig().baseTime) - // if numPreloadEvents = 0 - } + long wallclockTimestamp = wallclockBaseTime + (eventTimestamp - getCurrentConfig().baseTime); // Seed the random number generator with the next 'event id'. Random random = new Random(getNextEventId()); diff --git a/integration/java/src/main/java/com/google/cloud/dataflow/integration/nexmark/GeneratorConfig.java b/integration/java/src/main/java/com/google/cloud/dataflow/integration/nexmark/GeneratorConfig.java index d4045f4aa11d..4da3533a5964 100644 --- a/integration/java/src/main/java/com/google/cloud/dataflow/integration/nexmark/GeneratorConfig.java +++ b/integration/java/src/main/java/com/google/cloud/dataflow/integration/nexmark/GeneratorConfig.java @@ -71,11 +71,6 @@ class GeneratorConfig implements Serializable { /** Maximum number of events to generate. */ public final long maxEvents; - /** - * How many events to pump without rate limiting when first start. - */ - public final long numPreloadEvents; - /** * 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. @@ -95,7 +90,7 @@ class GeneratorConfig implements Serializable { public final long eventsPerEpoch; public GeneratorConfig(NexmarkConfiguration configuration, long baseTime, long firstEventId, - long maxEventsOrZero, long numPreloadEvents, long firstEventNumber) { + long maxEventsOrZero, long firstEventNumber) { this.configuration = configuration; this.interEventDelayUs = configuration.qpsShape.interEventDelayUs( configuration.firstEventQps, configuration.nextEventQps, configuration.numEventGenerators); @@ -112,7 +107,6 @@ public GeneratorConfig(NexmarkConfiguration configuration, long baseTime, long f } else { this.maxEvents = maxEventsOrZero; } - this.numPreloadEvents = numPreloadEvents; this.firstEventNumber = firstEventNumber; long eventsPerEpoch = 0; @@ -133,17 +127,14 @@ public GeneratorConfig(NexmarkConfiguration configuration, long baseTime, long f */ @Override public GeneratorConfig clone() { - return new GeneratorConfig( - configuration, baseTime, firstEventId, maxEvents, numPreloadEvents, firstEventNumber); + 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 numPreloadEvents, long firstEventNumber) { - return new GeneratorConfig( - configuration, baseTime, firstEventId, maxEvents, numPreloadEvents, firstEventNumber); + public GeneratorConfig cloneWith(long firstEventId, long maxEvents, long firstEventNumber) { + return new GeneratorConfig(configuration, baseTime, firstEventId, maxEvents, firstEventNumber); } /** @@ -159,15 +150,12 @@ public List split(int n) { } else { long subMaxEvents = maxEvents / n; long subFirstEventId = firstEventId; - long subNumPreloadEvents = numPreloadEvents / n; for (int i = 0; i < n; i++) { if (i == n - 1) { // Don't loose any events to round-down. subMaxEvents = maxEvents - subMaxEvents * (n - 1); - subNumPreloadEvents = numPreloadEvents - subNumPreloadEvents * (n - 1); } - results.add( - cloneWith(subFirstEventId, subMaxEvents, subNumPreloadEvents, firstEventNumber)); + results.add(cloneWith(subFirstEventId, subMaxEvents, firstEventNumber)); subFirstEventId += subMaxEvents; } } @@ -266,11 +254,10 @@ public KV timestampAndInterEventDelayUsForEvent(long eventNumber) { @Override public String toString() { - return String.format( - "GeneratorConfig{configuration:%s, baseTime:%d, firstEventId:%d, " - + "maxEvents:%d, numPreloadEvents:%d, firstEventNumber:%d, epochPeriodMs:%d, " - + "eventsPerEpoch:%d}", - configuration, baseTime, firstEventId, maxEvents, numPreloadEvents, firstEventNumber, - epochPeriodMs, eventsPerEpoch); + return String.format("GeneratorConfig{configuration:%s, baseTime:%d, firstEventId:%d, " + + "maxEvents:%d, firstEventNumber:%d, epochPeriodMs:%d, " + + "eventsPerEpoch:%d}", + configuration, baseTime, firstEventId, maxEvents, firstEventNumber, epochPeriodMs, + eventsPerEpoch); } } diff --git a/integration/java/src/main/java/com/google/cloud/dataflow/integration/nexmark/NexmarkConfiguration.java b/integration/java/src/main/java/com/google/cloud/dataflow/integration/nexmark/NexmarkConfiguration.java index 3dd0004dc331..9967b803881a 100644 --- a/integration/java/src/main/java/com/google/cloud/dataflow/integration/nexmark/NexmarkConfiguration.java +++ b/integration/java/src/main/java/com/google/cloud/dataflow/integration/nexmark/NexmarkConfiguration.java @@ -72,13 +72,6 @@ class NexmarkConfiguration implements Serializable { @JsonProperty public long numEvents = 100000; - /** - * Number of events to generate at the special pre-load rate. This is useful for generating - * a backlog of events on pub/sub before the main query begins. - */ - @JsonProperty - public long numPreloadEvents = 0; - /** * Number of event generators to use. Each generates events in its own timeline. */ @@ -110,10 +103,11 @@ class NexmarkConfiguration implements Serializable { public int qpsPeriodSec = 600; /** - * Overall event qps while pre-loading. Typcially as large as possible for given pub/sub quota. + * Time in seconds to preload the subscription with data, at the initial input rate of the + * pipeline. */ @JsonProperty - public int preloadEventQps = 10000; + public int preloadSeconds = 0; /** * If true, and in streaming mode, generate events only when they are due according to their @@ -275,9 +269,6 @@ public void overrideFromOptions(Options options) { if (options.getNumEvents() != null) { numEvents = options.getNumEvents(); } - if (options.getNumPreloadEvents() != null) { - numPreloadEvents = options.getNumPreloadEvents(); - } if (options.getNumEventGenerators() != null) { numEventGenerators = options.getNumEventGenerators(); } @@ -293,8 +284,8 @@ public void overrideFromOptions(Options options) { if (options.getQpsPeriodSec() != null) { qpsPeriodSec = options.getQpsPeriodSec(); } - if (options.getPreloadEventQps() != null) { - preloadEventQps = options.getPreloadEventQps(); + if (options.getPreloadSeconds() != null) { + preloadSeconds = options.getPreloadSeconds(); } if (options.getIsRateLimited() != null) { isRateLimited = options.getIsRateLimited(); @@ -386,13 +377,12 @@ public NexmarkConfiguration clone() { result.sinkType = sinkType; result.pubSubMode = pubSubMode; result.numEvents = numEvents; - result.numPreloadEvents = numPreloadEvents; result.numEventGenerators = numEventGenerators; result.qpsShape = qpsShape; result.firstEventQps = firstEventQps; result.nextEventQps = nextEventQps; result.qpsPeriodSec = qpsPeriodSec; - result.preloadEventQps = preloadEventQps; + result.preloadSeconds = preloadSeconds; result.isRateLimited = isRateLimited; result.useWallclockEventTime = useWallclockEventTime; result.avgPersonByteSize = avgPersonByteSize; @@ -439,9 +429,6 @@ public String toShortString() { if (numEvents != DEFAULT.numEvents) { sb.append(String.format("; numEvents:%d", numEvents)); } - if (numPreloadEvents != DEFAULT.numPreloadEvents) { - sb.append(String.format("; numPreloadEvents:%d", numPreloadEvents)); - } if (numEventGenerators != DEFAULT.numEventGenerators) { sb.append(String.format("; numEventGenerators:%d", numEventGenerators)); } @@ -455,8 +442,8 @@ public String toShortString() { if (qpsPeriodSec != DEFAULT.qpsPeriodSec) { sb.append(String.format("; qpsPeriodSec:%d", qpsPeriodSec)); } - if (preloadEventQps != DEFAULT.preloadEventQps) { - sb.append(String.format("; preloadEventQps:%d", preloadEventQps)); + if (preloadSeconds != DEFAULT.preloadSeconds) { + sb.append(String.format("; preloadSeconds:%d", preloadSeconds)); } if (isRateLimited != DEFAULT.isRateLimited) { sb.append(String.format("; isRateLimited:%s", isRateLimited)); @@ -557,8 +544,8 @@ public static NexmarkConfiguration fromString(String string) throws IOException @Override public int hashCode() { return Objects.hash(query, logResults, assertCorrectness, sourceType, sinkType, pubSubMode, - numEvents, numPreloadEvents, numEventGenerators, qpsShape, firstEventQps, nextEventQps, - qpsPeriodSec, preloadEventQps, isRateLimited, useWallclockEventTime, avgPersonByteSize, + numEvents, numEventGenerators, qpsShape, firstEventQps, nextEventQps, + qpsPeriodSec, preloadSeconds, isRateLimited, useWallclockEventTime, avgPersonByteSize, avgAuctionByteSize, avgBidByteSize, hotAuctionRatio, hotSellersRatio, hotBiddersRatio, windowSizeSec, windowPeriodSec, watermarkHoldbackSec, numInFlightAuctions, numActivePeople, justModelResultRate, coderStrategy, cpuDelayMs, diskBusyBytes, auctionSkip, fanout, @@ -644,13 +631,10 @@ public boolean equals(Object obj) { if (numActivePeople != other.numActivePeople) { return false; } - if (numPreloadEvents != other.numPreloadEvents) { - return false; - } if (occasionalDelaySec != other.occasionalDelaySec) { return false; } - if (preloadEventQps != other.preloadEventQps) { + if (preloadSeconds != other.preloadSeconds) { return false; } if (Double.doubleToLongBits(probDelayedEvent) diff --git a/integration/java/src/main/java/com/google/cloud/dataflow/integration/nexmark/NexmarkGoogleDriver.java b/integration/java/src/main/java/com/google/cloud/dataflow/integration/nexmark/NexmarkGoogleDriver.java index e5501e09228d..2e2971fd89e9 100644 --- a/integration/java/src/main/java/com/google/cloud/dataflow/integration/nexmark/NexmarkGoogleDriver.java +++ b/integration/java/src/main/java/com/google/cloud/dataflow/integration/nexmark/NexmarkGoogleDriver.java @@ -17,6 +17,7 @@ package com.google.cloud.dataflow.integration.nexmark; import com.google.cloud.dataflow.sdk.options.DataflowPipelineOptions; +import com.google.cloud.dataflow.sdk.options.Description; import com.google.cloud.dataflow.sdk.options.PipelineOptionsFactory; import org.joda.time.Duration; @@ -57,6 +58,27 @@ class NexmarkGoogleDriver { * 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); } /** diff --git a/integration/java/src/main/java/com/google/cloud/dataflow/integration/nexmark/NexmarkGoogleRunner.java b/integration/java/src/main/java/com/google/cloud/dataflow/integration/nexmark/NexmarkGoogleRunner.java index f8bc3968268e..c643bc77d397 100644 --- a/integration/java/src/main/java/com/google/cloud/dataflow/integration/nexmark/NexmarkGoogleRunner.java +++ b/integration/java/src/main/java/com/google/cloud/dataflow/integration/nexmark/NexmarkGoogleRunner.java @@ -599,7 +599,8 @@ public NexmarkPerf run(NexmarkConfiguration runConfiguration) new Query8(configuration), new Query9(configuration), new Query10(configuration), - new Query11(configuration)); + new Query11(configuration), + new Query12(configuration)); NexmarkQuery query = queries.get(configuration.query); queryName = query.getName(); @@ -609,7 +610,7 @@ public NexmarkPerf run(NexmarkConfiguration runConfiguration) new Query4Model(configuration), new Query5Model(configuration), new Query6Model(configuration), new Query7Model(configuration), new Query8Model(configuration), new Query9Model(configuration), - null, null); + null, null, null); NexmarkQueryModel model = models.get(configuration.query); if (configuration.justModelResultRate) { if (model == null) { @@ -714,7 +715,7 @@ private void modelResultRates(NexmarkQueryModel model) { /** * Monitor the progress of the publisher job. Return when it has produced at - * least {@code configuration.numPreloadEvents}. + * least {@code configuration.preloadSeconds} worth of events. * * @throws IOException * @throws InterruptedException @@ -729,7 +730,7 @@ private void waitForPublisherPreload() throws IOException, InterruptedException if (!(publisherResult instanceof DataflowPipelineJob)) { return; } - if (configuration.numPreloadEvents <= 0) { + if (configuration.preloadSeconds <= 0) { return; } @@ -740,11 +741,16 @@ private void waitForPublisherPreload() throws IOException, InterruptedException while (true) { PipelineResult.State state = job.getState(); long numEvents = getLong(job, publisherMonitor.getElementCounter()); - NexmarkUtils.console(null, "%s publisher (waiting for %d events, seen %d so far)", state, - configuration.numPreloadEvents, numEvents); - - if (numEvents >= 0 && numEvents >= configuration.numPreloadEvents) { - NexmarkUtils.console(null, "publisher preload done"); + if (numEvents > 0) { + int waitSeconds = configuration.preloadSeconds; + NexmarkUtils.console(null, "%s publisher (Saw first event, waiting %ds for preload)", state, + configuration.preloadSeconds); + while (waitSeconds > 0) { + NexmarkUtils.console( + null, "%s publisher (%ds until preload is done)", state, waitSeconds); + Thread.sleep(TimeUnit.SECONDS.toMillis(Math.min(60, waitSeconds))); + waitSeconds -= 60; + } return; } @@ -788,11 +794,15 @@ private NexmarkPerf monitor(NexmarkQuery query) throws IOException, InterruptedE if (!(mainResult instanceof DataflowPipelineJob)) { return null; } - if (!configuration.debug) { - return null; + // If we are not in debug mode, we have no event count or result count monitors. + boolean monitorsActive = configuration.debug; + + if (monitorsActive) { + NexmarkUtils.console(null, "waiting for main pipeline to 'finish'"); + } else { + NexmarkUtils.console(null, "debug=false, so job will not self-cancel"); } - NexmarkUtils.console(null, "waiting for main pipeline to 'finish'"); DataflowPipelineJob job = (DataflowPipelineJob) mainResult; DataflowPipelineJob publisherJob = (DataflowPipelineJob) publisherResult; @@ -800,8 +810,11 @@ private NexmarkPerf monitor(NexmarkQuery query) throws IOException, InterruptedE Instant start = Instant.now(); Instant end = options.getRunningTimeMinutes() != null - ? start.plus(Duration.standardMinutes(options.getRunningTimeMinutes())) - : new Instant(Long.MAX_VALUE); + ? start.plus(Duration.standardMinutes(options.getRunningTimeMinutes())) + : new Instant(Long.MAX_VALUE); + if (options.getPreloadSeconds() != null) { + end = end.minus(Duration.standardSeconds(options.getPreloadSeconds())); + } Instant lastActivity = null; NexmarkPerf perf = null; boolean waitingForShutdown = false; @@ -824,8 +837,13 @@ private NexmarkPerf monitor(NexmarkQuery query) throws IOException, InterruptedE NexmarkUtils.console( now, "%s %s%s", state, queryName, waitingForShutdown ? " (waiting for shutdown)" : ""); - NexmarkPerf currPerf = - currentPerf(start, now, job, snapshots, query.eventMonitor, query.resultMonitor); + NexmarkPerf currPerf; + if (monitorsActive) { + currPerf = currentPerf(start, now, job, snapshots, query.eventMonitor, + query.resultMonitor); + } else { + currPerf = null; + } if (perf == null || perf.anyActivity(currPerf)) { lastActivity = now; @@ -837,7 +855,8 @@ private NexmarkPerf monitor(NexmarkQuery query) throws IOException, InterruptedE NexmarkUtils.console(now, "job has fatal errors, cancelling."); errors.add(String.format("Pipeline reported %s fatal errors", query.getFatalCount())); waitingForShutdown = true; - } else if (configuration.numEvents > 0 && currPerf.numEvents == configuration.numEvents + } else if (monitorsActive && configuration.numEvents > 0 + && currPerf.numEvents == configuration.numEvents && currPerf.numResults >= 0 && quietFor.isLongerThan(DONE_DELAY)) { NexmarkUtils.console(now, "streaming query appears to have finished, cancelling job."); waitingForShutdown = true; @@ -854,7 +873,7 @@ private NexmarkPerf monitor(NexmarkQuery query) throws IOException, InterruptedE String.format("Streaming query was stuck for %d min", quietFor.getStandardMinutes())); } - errors.addAll(checkWatermarks(job, now.isAfter(start.plus(Duration.standardMinutes(5))))); + errors.addAll(checkWatermarks(job, start)); if (waitingForShutdown) { job.cancel(); @@ -938,7 +957,8 @@ private MetricType getMetricType(MetricUpdate metric) { *

*

Returns a list of errors detected. */ - private List checkWatermarks(DataflowPipelineJob job, boolean watermarksExpected) { + private List checkWatermarks(DataflowPipelineJob job, Instant start) { + Instant now = Instant.now(); List errors = new ArrayList<>(); try { JobMetrics metricResponse = job.getDataflowClient() @@ -964,24 +984,29 @@ private List checkWatermarks(DataflowPipelineJob job, boolean watermarks new Instant(scalar.divideToIntegralValue(new BigDecimal(1000)).longValueExact()); Instant updateTime = Instant.parse(metric.getUpdateTime()); - 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 (options.getWatermarkValidationDelaySeconds() == null + || now.isAfter(start.plus( + Duration.standardSeconds(options.getWatermarkValidationDelaySeconds())))) { + 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(null, msg); + 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(null, msg); + } } - } - if (!foundWatermarks) { - NexmarkUtils.console(null, "No known watermarks in update: " + metrics); - if (watermarksExpected) { - errors.add("No known watermarks found. Metrics were " + metrics); + if (!foundWatermarks) { + NexmarkUtils.console(null, "No known watermarks in update: " + metrics); + if (now.isAfter(start.plus(Duration.standardMinutes(5)))) { + errors.add("No known watermarks found. Metrics were " + metrics); + } } } } diff --git a/integration/java/src/main/java/com/google/cloud/dataflow/integration/nexmark/NexmarkSuite.java b/integration/java/src/main/java/com/google/cloud/dataflow/integration/nexmark/NexmarkSuite.java index 1f2c2a046c85..2c2bc8b8bc4f 100644 --- a/integration/java/src/main/java/com/google/cloud/dataflow/integration/nexmark/NexmarkSuite.java +++ b/integration/java/src/main/java/com/google/cloud/dataflow/integration/nexmark/NexmarkSuite.java @@ -59,7 +59,7 @@ private static List defaultConf() { private static List smoke() { List configurations = new ArrayList<>(); - for (int query = 0; query <= 10; query++) { + for (int query = 0; query <= 12; query++) { NexmarkConfiguration configuration = NexmarkConfiguration.DEFAULT.clone(); configuration.query = query; configuration.numEvents = 100_000; diff --git a/integration/java/src/main/java/com/google/cloud/dataflow/integration/nexmark/Options.java b/integration/java/src/main/java/com/google/cloud/dataflow/integration/nexmark/Options.java index 0f1c4b023ef5..82081d004f1d 100644 --- a/integration/java/src/main/java/com/google/cloud/dataflow/integration/nexmark/Options.java +++ b/integration/java/src/main/java/com/google/cloud/dataflow/integration/nexmark/Options.java @@ -86,12 +86,12 @@ public interface Options { void setNumEvents(Long numEvents); - @Description("Number of events to generate at the special pre-load rate. This is useful " - + "for generating a backlog of events on pub/sub before the main query begins.") + @Description("Time in seconds to preload the subscription with data, at the initial input rate " + + "of the pipeline.") @Nullable - Long getNumPreloadEvents(); + Integer getPreloadSeconds(); - void setNumPreloadEvents(Long numPreloadEvents); + void setPreloadSeconds(Integer preloadSeconds); @Description("Number of unbounded sources to create events.") @Nullable @@ -123,13 +123,6 @@ public interface Options { void setQpsPeriodSec(Integer qpsPeriodSec); - @Description("Overall event qps while pre-loading. " - + "Typcially as large as possible for given pub/sub quota.") - @Nullable - Integer getPreloadEventQps(); - - void setPreloadEventQps(Integer preloadEventQps); - @Description("If true, relay events in real time in streaming mode.") @Nullable Boolean getIsRateLimited(); @@ -330,24 +323,6 @@ public interface Options { void setOutOfOrderGroupSize(Long outOfOrderGroupSize); - - @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("If false, do not add the Monitor and Snoop transforms.") @Nullable Boolean getDebug(); diff --git a/integration/java/src/main/java/com/google/cloud/dataflow/integration/nexmark/Query11.java b/integration/java/src/main/java/com/google/cloud/dataflow/integration/nexmark/Query11.java index 90f61ffb3dae..06d4f6991bdf 100644 --- a/integration/java/src/main/java/com/google/cloud/dataflow/integration/nexmark/Query11.java +++ b/integration/java/src/main/java/com/google/cloud/dataflow/integration/nexmark/Query11.java @@ -20,6 +20,7 @@ import com.google.cloud.dataflow.sdk.transforms.DoFn; import com.google.cloud.dataflow.sdk.transforms.ParDo; import com.google.cloud.dataflow.sdk.transforms.windowing.AfterPane; +import com.google.cloud.dataflow.sdk.transforms.windowing.Repeatedly; import com.google.cloud.dataflow.sdk.transforms.windowing.Sessions; import com.google.cloud.dataflow.sdk.transforms.windowing.Window; import com.google.cloud.dataflow.sdk.values.KV; @@ -52,7 +53,7 @@ public void processElement(ProcessContext c) { })) .apply(Window.>into( Sessions.withGapDuration(Duration.standardSeconds(configuration.windowSizeSec))) - .triggering(AfterPane.elementCountAtLeast(configuration.maxLogEvents)) + .triggering(Repeatedly.forever(AfterPane.elementCountAtLeast(configuration.maxLogEvents))) .discardingFiredPanes() .withAllowedLateness(Duration.standardSeconds(configuration.occasionalDelaySec / 2))) .apply(Count.perKey()) From 27780a94d485137ebdd0f01bf96cf5807f5f60be Mon Sep 17 00:00:00 2001 From: Mark Shields Date: Thu, 7 Apr 2016 12:51:06 -0700 Subject: [PATCH 09/14] add query12 --- .../dataflow/integration/nexmark/Query12.java | 80 +++++++++++++++++++ 1 file changed, 80 insertions(+) create mode 100644 integration/java/src/main/java/com/google/cloud/dataflow/integration/nexmark/Query12.java diff --git a/integration/java/src/main/java/com/google/cloud/dataflow/integration/nexmark/Query12.java b/integration/java/src/main/java/com/google/cloud/dataflow/integration/nexmark/Query12.java new file mode 100644 index 000000000000..6f1a6088ef55 --- /dev/null +++ b/integration/java/src/main/java/com/google/cloud/dataflow/integration/nexmark/Query12.java @@ -0,0 +1,80 @@ +/* + * Copyright (C) 2015 Google Inc. + * + * Licensed under the Apache License, Version 2.0 (the "License"); you may not + * use this file except in compliance with the License. You may obtain a copy of + * the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT + * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the + * License for the specific language governing permissions and limitations under + * the License. + */ + +package com.google.cloud.dataflow.integration.nexmark; + +import com.google.cloud.dataflow.sdk.transforms.Count; +import com.google.cloud.dataflow.sdk.transforms.DoFn; +import com.google.cloud.dataflow.sdk.transforms.ParDo; +import com.google.cloud.dataflow.sdk.transforms.windowing.AfterPane; +import com.google.cloud.dataflow.sdk.transforms.windowing.AfterProcessingTime; +import com.google.cloud.dataflow.sdk.transforms.windowing.GlobalWindows; +import com.google.cloud.dataflow.sdk.transforms.windowing.Repeatedly; +import com.google.cloud.dataflow.sdk.transforms.windowing.Sessions; +import com.google.cloud.dataflow.sdk.transforms.windowing.Window; +import com.google.cloud.dataflow.sdk.values.KV; +import com.google.cloud.dataflow.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)); + } +} From 35b0d6eb951851af44b78c5eaf5121749ef0755b Mon Sep 17 00:00:00 2001 From: Mark Shields Date: Thu, 7 Apr 2016 18:00:43 -0700 Subject: [PATCH 10/14] Started reworking runner --- .../nexmark/NexmarkGoogleRunner.java | 2 +- .../integration/nexmark/NexmarkRunner.java | 1092 +++++++++++++++++ .../integration/nexmark/NexmarkUtils.java | 25 +- 3 files changed, 1096 insertions(+), 23 deletions(-) create mode 100644 integration/java/src/main/java/com/google/cloud/dataflow/integration/nexmark/NexmarkRunner.java diff --git a/integration/java/src/main/java/com/google/cloud/dataflow/integration/nexmark/NexmarkGoogleRunner.java b/integration/java/src/main/java/com/google/cloud/dataflow/integration/nexmark/NexmarkGoogleRunner.java index c643bc77d397..660de60446d8 100644 --- a/integration/java/src/main/java/com/google/cloud/dataflow/integration/nexmark/NexmarkGoogleRunner.java +++ b/integration/java/src/main/java/com/google/cloud/dataflow/integration/nexmark/NexmarkGoogleRunner.java @@ -63,7 +63,7 @@ import javax.annotation.Nullable; /** - * Run a query according to specific set of options on Google Dataflow. + * Run a singe Nexmark query using a given configuration on Google Dataflow. */ class NexmarkGoogleRunner { /** diff --git a/integration/java/src/main/java/com/google/cloud/dataflow/integration/nexmark/NexmarkRunner.java b/integration/java/src/main/java/com/google/cloud/dataflow/integration/nexmark/NexmarkRunner.java new file mode 100644 index 000000000000..d28b0979ce91 --- /dev/null +++ b/integration/java/src/main/java/com/google/cloud/dataflow/integration/nexmark/NexmarkRunner.java @@ -0,0 +1,1092 @@ +package com.google.cloud.dataflow.integration.nexmark; + +import java.io.IOException; +import javax.annotation.Nullable; + +/** + * Run a single Nexmark query using a given configuration. + */ +public class NexmarkRunner { + private final Options options; + + /** + * Which configuration should we run. + */ + @Nullable + private NexmarkConfiguration configuration; + + /** + * Accumulate the pub/sub subscriptions etc which should be cleaned up on end of run. + */ + @Nullable + private PubsubHelper pubsub; + + /** + * Monitor for published events if in pub/sub COMBINED mode. + */ + @Nullable + private Monitor publisherMonitor; + + /** + * Query name. + */ + @Nullable + private String queryName; + + /** + * If sending events via pub/sub, the full topic name to use. + */ + @Nullable + private String inputTopic; + + public NexmarkRunner(Options options) { + this.options = options; + } + + /** + * Return a pub/sub helper. + * + * @throws IOException + */ + private PubsubHelper getPubsub() throws IOException { + if (pubsub == null) { + pubsub = PubsubHelper.create(options, options.getProject()); + } + return pubsub; + } + + /** + * Return a source of synthetic events. + */ + private PCollection createSyntheticSource(Pipeline p) { + if (p.getRunner() instanceof DirectPipelineRunner || !options.isStreaming()) { + return p.apply(NexmarkUtils.batchEventsSource(queryName, configuration)); + } else { + return p.apply(NexmarkUtils.streamEventsSource(queryName, configuration)); + } + } + + /** + * Return pub/sub sink. + */ + private PubsubIO.Write.Bound pubsubEventSink() { + PubsubIO.Write.Bound io = + PubsubIO.Write.named(queryName + ".Write(" + inputTopic + ")") + .topic(inputTopic) + .idLabel(NexmarkUtils.PUBSUB_ID) + .withCoder(Event.CODER) + .named(queryName + ".PubsubSourceWrite"); + if (!configuration.usePubsubPublishTime) { + io = io.timestampLabel(NexmarkUtils.PUBSUB_TIMESTAMP); + } + return io; + } + + /** + * Return pub/sub source. + */ + private PubsubIO.Read.Bound pubsubEventSource(String subscription) { + PubsubIO.Read.Bound io = + PubsubIO.Read.named(queryName + ".Read(" + subscription + ")") + .subscription(subscription) + .idLabel(NexmarkUtils.PUBSUB_ID) + .withCoder(Event.CODER) + .named(queryName + ".PubsubSourceRead"); + if (!configuration.usePubsubPublishTime) { + io = io.timestampLabel(NexmarkUtils.PUBSUB_TIMESTAMP); + } + return io; + } + + /** + * Return pub/sub sink for results. + */ + private PubsubIO.Write.Bound pubsubResultSink(String topic) { + PubsubIO.Write.Bound io = + PubsubIO.Write.named(queryName + ".Write(" + topic + ")") + .topic(topic) + .idLabel(NexmarkUtils.PUBSUB_ID) + .named(queryName + ".PubsubSinkWrite"); + if (!configuration.usePubsubPublishTime) { + io = io.timestampLabel(NexmarkUtils.PUBSUB_TIMESTAMP); + } + return io; + } + + /** + * Return number of cores for configuration's machine type. + */ + private int coresPerWorker(DataflowPipelineOptions options) { + 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; + } + } + + /** + * Return source of events for this run, or null if we are simply publishing events + * to pub/sub. + * + * @throws IOException + * @throws InterruptedException + */ + private PCollection createSource(Pipeline p, long now) + throws IOException, InterruptedException { + PCollection source = null; + switch (configuration.sourceType) { + case DIRECT: + source = createSyntheticSource(p); + break; + case AVRO: + if (options.getInputFilePrefix() == null) { + throw new RuntimeException( + "If sourceType is AVRO, --inputFilePrefix must be specified."); + } + PCollection preTimestamp = p.apply(AvroIO.Read.named("ReadFromAvro") + .from(options.getInputFilePrefix() + + "*.avro") + .withSchema(Event.class)); + source = preTimestamp.apply("adjust timestamp", + ParDo.of(NexmarkQuery.EVENT_TIMESTAMP_FROM_DATA)); + break; + case PUBSUB: + if (pubsubTopic == null) { + throw new RuntimeException("must supply a --pubsubTopic if using --sourceType=PUBSUB"); + } + + // Check some flags. + switch (configuration.pubSubMode) { + case SUBSCRIBE_ONLY: + break; + case PUBLISH_ONLY: + if (manageResources && options.getRunner() != BlockingDataflowPipelineRunner.class) { + throw new RuntimeException( + "If --manageResources=true and --pubSubMode=PUBLISH_ONLY then " + + "--runner=BlockingDataflowPipelineRunner so that this program " + + "can cleanup the pub/sub topic on exit."); + } + break; + case COMBINED: + if (options.getRunner() != DataflowPipelineRunner.class || !monitorJobs) { + throw new RuntimeException( + "if --pubSubMode=COMBINED then --runner=DataflowPipelineRunner and " + + "--monitorJobs=true so that the two pipelines can be managed."); + } + break; + } + + // Choose a topic name for events, and optionally topics to use to synchronize + // the publisher and subscriber jobs. + String shortTopic; + if (uniqify) { + // Salt the topic name so we can run multiple jobs in parallel. + shortTopic = String.format("%s_%d_%s_source", pubsubTopic, now, queryName); + } else { + shortTopic = String.format("%s_%s_source", pubsubTopic, queryName); + } + String shortSubscription = shortTopic; + + // Create/confirm the topic. + if (!manageResources || configuration.pubSubMode == PubSubMode.SUBSCRIBE_ONLY) { + // The topic should already have been created by the user or + // a companion 'PUBLISH_ONLY' process. + inputTopic = getPubsub().reuseTopic(shortTopic); + } else { + // Create a fresh topic to loopback via. It will be destroyed when the + // (necessarily blocking) job is done. + inputTopic = getPubsub().createTopic(shortTopic); + } + + // Create/confirm the subscription. + String subscription = null; + if (configuration.pubSubMode == PubSubMode.PUBLISH_ONLY) { + // Nothing to consume. + } else if (!manageResources) { + // The subscription should already have been created by the user. + subscription = getPubsub().reuseSubscription(shortTopic, shortSubscription); + } else { + subscription = getPubsub().createSubscription(shortTopic, shortSubscription); + } + + // Setup the sink for the publisher. + switch (configuration.pubSubMode) { + case SUBSCRIBE_ONLY: + // Nothing to publish. + break; + case PUBLISH_ONLY: + // Send synthesized events to pub/sub in this job. + createSyntheticSource(p).apply(NexmarkUtils.snoop(queryName)).apply(pubsubEventSink()); + break; + case COMBINED: + if (options.getRunner() != DataflowPipelineRunner.class || !monitorJobs) { + throw new RuntimeException( + "if --pubSubMode=COMBINED then you must use --runner=DataflowPipelineRunner " + + "and --monitorJobs=true so that the publisher job can be shutdown cleanly"); + } + // Send synthesized events to pub/sub 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. + String jobName = options.getJobName(); + String appName = options.getAppName(); + options.setJobName("p-" + jobName); + options.setAppName("p-" + appName); + int coresPerWorker = coresPerWorker(options); + 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"); + Pipeline q = Pipeline.create(options); + createSyntheticSource(q) + .apply(publisherMonitor.getTransform()) + .apply(pubsubEventSink()); + PrintStream stdout = System.out; + try { + // Suppress output of publisher job; it makes the output of the command harder to + // interpret. + System.setOut(new PrintStream(new OutputStream() { + @Override + public void write(int b) { + } + })); + publisherResult = q.run(); + } finally { + System.setOut(stdout); + } + System.out.println( + "Publisher job running as " + ((DataflowPipelineJob) publisherResult).getJobId()); + options.setJobName(jobName); + options.setAppName(appName); + options.setMaxNumWorkers(options.getMaxNumWorkers()); + options.setNumWorkers(options.getNumWorkers()); + waitForPublisherPreload(); + 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 = p.apply(pubsubEventSource(subscription)); + break; + } + break; + } + return source; + } + + /** + * Consume the given results. + * + * @throws IOException + * @throws InterruptedException + */ + private void createSink(PCollection> results, long now) + throws IOException, InterruptedException { + + if (NexmarkUtils.SinkType.COUNT_ONLY == configuration.sinkType) { + results.apply(NexmarkUtils.devNull(queryName)); + return; + } + PCollection formattedResults = results.apply(NexmarkUtils.format(queryName)); + + if (configuration.logResults) { + formattedResults = formattedResults.apply(NexmarkUtils.log(queryName)); + } + + switch (configuration.sinkType) { + case DEVNULL: + // Discard all results + formattedResults.apply(NexmarkUtils.devNull(queryName)); + break; + case PUBSUB: + // Publish results to pubsub. Don't bother consuming them, the published + // results will be discarded when we delete the 'sink' topic. + if (pubsubTopic == null) { + throw new RuntimeException("must supply a --pubsubTopic if using --sinkType=PUBSUB"); + } + String shortTopic; + if (uniqify) { + shortTopic = String.format("%s_%d_%s_sink", pubsubTopic, now, queryName); + } else { + shortTopic = String.format("%s_%s_sink", pubsubTopic, queryName); + } + String topic; + if (!manageResources) { + topic = getPubsub().reuseTopic(shortTopic); + } else { + topic = getPubsub().createTopic(shortTopic); + } + formattedResults.apply(pubsubResultSink(topic)); + break; + case TEXT: + // Write results to text. Only works in batch mode. + if (options.isStreaming()) { + throw new RuntimeException("can only use --sinkType=TEXT with --streaming=false"); + } + if (outputPath == null) { + throw new RuntimeException("must supply an --outputPath if using --sinkType=TEXT"); + } + String fullFilename; + if (uniqify) { + fullFilename = String.format("%s/nexmark_%d_%s.txt", outputPath, now, queryName); + } else { + fullFilename = String.format("%s/nexmark_%s.txt", outputPath, queryName); + } + formattedResults.apply(TextIO.Write.to(fullFilename).named(queryName + ".Text")); + break; + case AVRO: + NexmarkUtils.console(null, "WARNING: with --sinkType=AVRO, actual query results will be " + + "discarded.", outputPath); + break; + case BIGQUERY: + NexmarkUtils.console(null, "Writing events to BigQuery table %s", outputPath); + formattedResults + .apply(ParDo.of(new StringToTableRow())) + .apply(BigQueryIO.Write.named("WriteBigQuery(Events)") + .to(options.getProject() + ":nexmark.table_" + + new Random().nextInt(Integer.MAX_VALUE)) + .withSchema(new TableSchema().setFields( + new ArrayList() { + { + add(new TableFieldSchema().setName("event") + .setType("STRING")); + } + }))); + break; + case COUNT_ONLY: + // Short-circuited above. + throw new RuntimeException(); + } + } + + private static class StringToTableRow extends DoFn { + @Override + public void processElement(ProcessContext c) { + TableRow row = new TableRow(); + row.set("event", c.element()); + c.output(row); + } + } + + /** + * Sink all raw Events in {@code source} to {@code this.outputPath}. + *

+ * 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 sinkToAvro(final PCollection source) { + if (options.isStreaming()) { + throw new RuntimeException("can only use Avro SinkType with --streaming=false"); + } + if (outputPath == null) { + throw new RuntimeException("Must supply an --outputPath if using --sinkType=AVRO"); + } + NexmarkUtils.console(null, "Writing events in Avro to %s", outputPath); + source.apply(AvroIO.Write.named("WriteAvro(Events)") + .to(outputPath + "/event") + .withSuffix(".avro") + .withSchema(Event.class)); + source.apply(NexmarkQuery.JUST_BIDS) + .apply(AvroIO.Write.named("WriteAvro(Bids)") + .to(outputPath + "/bid") + .withSuffix(".avro") + .withSchema(Bid.class)); + source.apply(NexmarkQuery.JUST_NEW_AUCTIONS) + .apply(AvroIO.Write.named("WriteAvro(Auctions)") + .to(outputPath + "/auction") + .withSuffix(".avro") + .withSchema(Auction.class)); + source.apply(NexmarkQuery.JUST_NEW_PERSONS) + .apply(AvroIO.Write.named("WriteAvro(People)") + .to(outputPath + "/person") + .withSuffix(".avro") + .withSchema(Person.class)); + } + + /** + * Run {@code configuration} and return its performance (if using DataflowPipelineRunner + * and {@link #monitorJobs} is true). + * + * @throws IOException + * @throws InterruptedException + */ + @Nullable + public NexmarkPerf run(NexmarkConfiguration runConfiguration) + throws IOException, InterruptedException { + // + // Setup per-run state. + // + Preconditions.checkState(configuration == null); + Preconditions.checkState(pubsub == null); + Preconditions.checkState(publisherResult == null); + Preconditions.checkState(mainResult == null); + Preconditions.checkState(queryName == null); + configuration = runConfiguration; + + // GCS URI patterns to delete on exit. + List pathsToDelete = new ArrayList<>(); + + try { + NexmarkUtils.console(null, "running %s", configuration.toShortString()); + + if (configuration.numEvents < 0) { + NexmarkUtils.console(null, "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 (configuration.justModelResultRate) { + 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); + + // 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) { + if (monitorJobs && options.getRunner() != DataflowPipelineRunner.class) { + throw new RuntimeException("can only use --monitorJobs=true if also " + + "using --runner=DataflowPipelineRunner"); + } + + // Optionally sink events in Avro format + if (configuration.sinkType == NexmarkUtils.SinkType.AVRO) { + this.sinkToAvro(source); + } + + // Special hacks for Query 10 (big logger). + if (configuration.query == 10) { + String path = null; + if (outputPath != null) { + if (uniqify) { + path = String.format("%s/%d_logs", outputPath, now); + } else { + path = String.format("%s/logs", outputPath); + } + } + ((Query10) query).setOutputPath(path); + ((Query10) query).setMaxNumWorkers( + Math.max(options.getMaxNumWorkers(), options.getNumWorkers())); + if (path != null && manageResources) { + pathsToDelete.add(path + "/**"); + } + } + + // Apply query. + PCollection> results = source.apply(query); + + if (configuration.assertCorrectness) { + 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(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. + DataflowAssert.that(results).satisfies(model.assertionFor()); + } + + // Output results. + createSink(results, now); + } + + mainResult = p.run(); + return monitor(query); + } finally { + // + // Cleanup per-run state. + // + if (pubsub != null) { + // Delete any subscriptions and topics we created. + pubsub.cleanup(); + pubsub = null; + } + // TODO: Find a way to cleanup pathsToDelete robustly without depending on gsutil. + configuration = null; + publisherResult = null; + mainResult = null; + queryName = null; + } + } + + /** + * 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(null, "Query%d: only %d samples", model.configuration.query, n); + } else { + NexmarkUtils.console(null, "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)); + } + } + + /** + * Monitor the progress of the publisher job. Return when it has produced at + * least {@code configuration.preloadSeconds} worth of events. + * + * @throws IOException + * @throws InterruptedException + */ + private void waitForPublisherPreload() throws IOException, InterruptedException { + if (!monitorJobs) { + return; + } + if (options.getRunner() != DataflowPipelineRunner.class) { + return; + } + if (!(publisherResult instanceof DataflowPipelineJob)) { + return; + } + if (configuration.preloadSeconds <= 0) { + return; + } + + NexmarkUtils.console(null, "waiting for publisher to pre-load"); + + DataflowPipelineJob job = (DataflowPipelineJob) publisherResult; + + while (true) { + PipelineResult.State state = job.getState(); + long numEvents = getLong(job, publisherMonitor.getElementCounter()); + if (numEvents > 0) { + int waitSeconds = configuration.preloadSeconds; + NexmarkUtils.console(null, "%s publisher (Saw first event, waiting %ds for preload)", state, + configuration.preloadSeconds); + while (waitSeconds > 0) { + NexmarkUtils.console( + null, "%s publisher (%ds until preload is done)", state, waitSeconds); + Thread.sleep(TimeUnit.SECONDS.toMillis(Math.min(60, waitSeconds))); + waitSeconds -= 60; + } + return; + } + + switch (state) { + case UNKNOWN: + case STOPPED: + case RUNNING: + // Keep waiting. + break; + case DONE: + // All done. + NexmarkUtils.console(null, "publisher pipeline done"); + return; + case CANCELLED: + case FAILED: + case UPDATED: + // Something went wrong. + NexmarkUtils.console(null, "publisher pipeline failed", state, numEvents); + return; + } + + Thread.sleep(PERF_DELAY.getMillis()); + } + } + + /** + * Monitor the performance and progress of a running job. Return final performance if + * it was measured. + * + * @throws IOException + * @throws InterruptedException + */ + @Nullable + private NexmarkPerf monitor(NexmarkQuery query) throws IOException, InterruptedException { + if (!monitorJobs) { + return null; + } + if (options.getRunner() != DataflowPipelineRunner.class) { + return null; + } + if (!(mainResult instanceof DataflowPipelineJob)) { + return null; + } + // If we are not in debug mode, we have no event count or result count monitors. + boolean monitorsActive = configuration.debug; + + if (monitorsActive) { + NexmarkUtils.console(null, "waiting for main pipeline to 'finish'"); + } else { + NexmarkUtils.console(null, "debug=false, so job will not self-cancel"); + } + + + DataflowPipelineJob job = (DataflowPipelineJob) mainResult; + DataflowPipelineJob publisherJob = (DataflowPipelineJob) publisherResult; + List snapshots = new ArrayList<>(); + Instant start = Instant.now(); + Instant end = + options.getRunningTimeMinutes() != null + ? start.plus(Duration.standardMinutes(options.getRunningTimeMinutes())) + : new Instant(Long.MAX_VALUE); + if (options.getPreloadSeconds() != null) { + end = end.minus(Duration.standardSeconds(options.getPreloadSeconds())); + } + Instant lastActivity = null; + NexmarkPerf perf = null; + boolean waitingForShutdown = false; + boolean publisherCancelled = false; + List errors = new ArrayList<>(); + + while (true) { + Instant now = Instant.now(); + if (now.isAfter(end) && !waitingForShutdown) { + NexmarkUtils.console(null, "Reached end of test, cancelling job"); + job.cancel(); + if (publisherResult != null) { + publisherJob.cancel(); + publisherCancelled = true; + } + waitingForShutdown = true; + } + + PipelineResult.State state = job.getState(); + NexmarkUtils.console( + now, "%s %s%s", state, queryName, waitingForShutdown ? " (waiting for shutdown)" : ""); + + NexmarkPerf currPerf; + if (monitorsActive) { + currPerf = currentPerf(start, now, job, snapshots, query.eventMonitor, + query.resultMonitor); + } else { + currPerf = null; + } + + if (perf == null || perf.anyActivity(currPerf)) { + lastActivity = now; + } + + if (options.isStreaming() && !waitingForShutdown) { + Duration quietFor = new Duration(lastActivity, now); + if (query.getFatalCount() != null && getLong(job, query.getFatalCount()) > 0) { + NexmarkUtils.console(now, "job has fatal errors, cancelling."); + errors.add(String.format("Pipeline reported %s fatal errors", query.getFatalCount())); + waitingForShutdown = true; + } else if (monitorsActive && configuration.numEvents > 0 + && currPerf.numEvents == configuration.numEvents + && currPerf.numResults >= 0 && quietFor.isLongerThan(DONE_DELAY)) { + NexmarkUtils.console(now, "streaming query appears to have finished, cancelling job."); + waitingForShutdown = true; + } else if (quietFor.isLongerThan(STUCK_TERMINATE_DELAY)) { + NexmarkUtils.console( + now, "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( + now, "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, start)); + + if (waitingForShutdown) { + job.cancel(); + } + } + + 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 (lastActivity.equals(now)) { + NexmarkUtils.console(now, "new perf %s", perf); + } else { + NexmarkUtils.console(now, "no activity"); + } + + Thread.sleep(PERF_DELAY.getMillis()); + } + + perf.errors = errors; + perf.snapshots = snapshots; + NexmarkUtils.console(null, "final perf %s", perf); + + if (publisherResult != null) { + if (publisherCancelled) { + publisherJob.waitToFinish(5, TimeUnit.MINUTES, null); + } else { + publisherJob.cancel(); + } + } + + 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, Instant start) { + Instant now = Instant.now(); + 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.isAfter(start.plus( + Duration.standardSeconds(options.getWatermarkValidationDelaySeconds())))) { + 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(null, msg); + } + } + if (!foundWatermarks) { + NexmarkUtils.console(null, "No known watermarks in update: " + metrics); + if (now.isAfter(start.plus(Duration.standardMinutes(5)))) { + errors.add("No known watermarks found. Metrics were " + metrics); + } + } + } + } + } catch (IOException e) { + NexmarkUtils.console(null, "Warning: failed to get JobMetrics: " + e); + } + + return errors; + } + + /** + * Return the current performance given {@code eventMonitor} and {@code resultMonitor}. + */ + private NexmarkPerf currentPerf( + Instant start, Instant 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 - start.getMillis()) / 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.getMillis() - 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.getMillis() - start.getMillis()) / 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( + null, "%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(null, + "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( + null, "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( + Instant 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.getMillis()) > Duration.standardDays(10000).getMillis()) { + return -1; + } + return value; + } catch (AggregatorRetrievalException e) { + return -1; + } + } + +} diff --git a/integration/java/src/main/java/com/google/cloud/dataflow/integration/nexmark/NexmarkUtils.java b/integration/java/src/main/java/com/google/cloud/dataflow/integration/nexmark/NexmarkUtils.java index 77dd3c6af44c..003fbabd5589 100644 --- a/integration/java/src/main/java/com/google/cloud/dataflow/integration/nexmark/NexmarkUtils.java +++ b/integration/java/src/main/java/com/google/cloud/dataflow/integration/nexmark/NexmarkUtils.java @@ -290,28 +290,9 @@ public static void setupPipeline(CoderStrategy coderStrategy, Pipeline p) { * Return a generator config to match the given {@code options}. */ public static GeneratorConfig standardGeneratorConfig(NexmarkConfiguration configuration) { - GeneratorConfig config = new GeneratorConfig( - configuration, BASE_TIME, 0, configuration.numEvents, configuration.numPreloadEvents, 0); - if (configuration.useWallclockEventTime) { - // We want events to have a timestamp matching wallclock time. - long now = System.currentTimeMillis(); - // How long, in wallclock time, do we expect it to take to pre-load events? - long preloadInterEventDelayUs = QpsShape.SQUARE.interEventDelayUs( - configuration.preloadEventQps, configuration.numEventGenerators); - long wallclockPreloadMs = - (configuration.numPreloadEvents * preloadInterEventDelayUs) - / (1000L * configuration.numEventGenerators); - // How long, in event time, does the event stream take to generate the pre-load events? - long firstPreloadEventTime = config.timestampAndInterEventDelayUsForEvent(0).getKey(); - long firstPostloadEventTime = - config.timestampAndInterEventDelayUsForEvent( - configuration.numPreloadEvents / configuration.numEventGenerators).getKey(); - long eventPreloadMs = firstPostloadEventTime - firstPreloadEventTime; - // Shift base time s.t. firstPostloadEventTime = now + wallclockPreLoadMs - config = new GeneratorConfig(configuration, now + wallclockPreloadMs - eventPreloadMs, 0, - configuration.numEvents, configuration.numPreloadEvents, 0); - } - return config; + return new GeneratorConfig(configuration, + configuration.useWallclockEventTime ? System.currentTimeMillis() : BASE_TIME, 0, + configuration.numEvents, 0); } /** From 18979fdce428be5acf9f3c007d8b1f9ac3bf5a9e Mon Sep 17 00:00:00 2001 From: Mark Shields Date: Fri, 8 Apr 2016 16:14:40 -0700 Subject: [PATCH 11/14] More de-googleization --- .../nexmark/NexmarkConfiguration.java | 8 +- .../integration/nexmark/NexmarkDriver.java | 279 ++++ .../nexmark/NexmarkGoogleDriver.java | 59 +- .../nexmark/NexmarkGoogleRunner.java | 848 +++--------- .../integration/nexmark/NexmarkPerf.java | 8 +- .../integration/nexmark/NexmarkQuery.java | 30 +- .../integration/nexmark/NexmarkRunner.java | 1204 ++++++----------- .../integration/nexmark/NexmarkUtils.java | 7 +- .../dataflow/integration/nexmark/Options.java | 3 +- .../integration/nexmark/PubsubHelper.java | 119 +- .../integration/nexmark/WinningBids.java | 2 +- 11 files changed, 940 insertions(+), 1627 deletions(-) create mode 100644 integration/java/src/main/java/com/google/cloud/dataflow/integration/nexmark/NexmarkDriver.java diff --git a/integration/java/src/main/java/com/google/cloud/dataflow/integration/nexmark/NexmarkConfiguration.java b/integration/java/src/main/java/com/google/cloud/dataflow/integration/nexmark/NexmarkConfiguration.java index 9967b803881a..6152b43c38a6 100644 --- a/integration/java/src/main/java/com/google/cloud/dataflow/integration/nexmark/NexmarkConfiguration.java +++ b/integration/java/src/main/java/com/google/cloud/dataflow/integration/nexmark/NexmarkConfiguration.java @@ -537,8 +537,12 @@ public String toString() { * * @throws IOException */ - public static NexmarkConfiguration fromString(String string) throws IOException { - return NexmarkUtils.MAPPER.readValue(string, NexmarkConfiguration.class); + 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 diff --git a/integration/java/src/main/java/com/google/cloud/dataflow/integration/nexmark/NexmarkDriver.java b/integration/java/src/main/java/com/google/cloud/dataflow/integration/nexmark/NexmarkDriver.java new file mode 100644 index 000000000000..39ae84f3a329 --- /dev/null +++ b/integration/java/src/main/java/com/google/cloud/dataflow/integration/nexmark/NexmarkDriver.java @@ -0,0 +1,279 @@ +package com.google.cloud.dataflow.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(O 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/com/google/cloud/dataflow/integration/nexmark/NexmarkGoogleDriver.java b/integration/java/src/main/java/com/google/cloud/dataflow/integration/nexmark/NexmarkGoogleDriver.java index 2e2971fd89e9..47baa9a231a3 100644 --- a/integration/java/src/main/java/com/google/cloud/dataflow/integration/nexmark/NexmarkGoogleDriver.java +++ b/integration/java/src/main/java/com/google/cloud/dataflow/integration/nexmark/NexmarkGoogleDriver.java @@ -53,7 +53,7 @@ * * http://datalab.cs.pdx.edu/niagaraST/NEXMark/ */ -class NexmarkGoogleDriver { +class NexmarkGoogleDriver extends NexmarkDriver { /** * Command line flags. */ @@ -61,23 +61,27 @@ 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); } @@ -92,39 +96,8 @@ public static void main(String[] args) throws IOException, InterruptedException NexmarkGoogleOptions options = PipelineOptionsFactory.fromArgs(args) .withValidation() .as(NexmarkGoogleOptions.class); - NexmarkGoogleRunner runner = - new NexmarkGoogleRunner(options, options.getOutputPath(), options.getPubsubTopic(), - options.getMonitorJobs(), options.getUniqify(), - options.getManageResources()); - 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); - } - } - - System.exit(successful ? 0 : 1); + NexmarkGoogleRunner runner = new NexmarkGoogleRunner(options); + new NexmarkGoogleDriver().runAll(options, runner); } /** @@ -145,8 +118,8 @@ private static void appendPerf( lines.add(configuration.toString()); lines.add(perf.toString()); Files.write(Paths.get(perfFilename), lines, StandardCharsets.UTF_8, StandardOpenOption.CREATE, - StandardOpenOption.APPEND); - NexmarkUtils.console(null, "appended results to perf file %s.", perfFilename); + StandardOpenOption.APPEND); + NexmarkUtils.console("appended results to perf file %s.", perfFilename); } /** @@ -170,8 +143,8 @@ private static Map loadBaseline( NexmarkPerf perf = NexmarkPerf.fromString(lines.get(i)); baseline.put(configuration, perf); } - NexmarkUtils.console( - null, "loaded %d entries from baseline file %s.", baseline.size(), baselineFilename); + NexmarkUtils.console("loaded %d entries from baseline file %s.", baseline.size(), + baselineFilename); return baseline; } @@ -214,7 +187,7 @@ private static void saveSummary( 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)")); + "(Baseline)", "Events(/sec)", "(Baseline)", "Results", "(Baseline)")); conf = 0; for (NexmarkConfiguration configuration : configurations) { String line = String.format(" %04d ", conf++); @@ -276,8 +249,8 @@ private static void saveSummary( if (summaryFilename != null) { Files.write(Paths.get(summaryFilename), lines, StandardCharsets.UTF_8, - StandardOpenOption.CREATE, StandardOpenOption.APPEND); - NexmarkUtils.console(null, "appended summary to summary file %s.", summaryFilename); + StandardOpenOption.CREATE, StandardOpenOption.APPEND); + NexmarkUtils.console("appended summary to summary file %s.", summaryFilename); } } @@ -315,8 +288,8 @@ private static void saveJavascript( lines.add("];"); Files.write(Paths.get(javascriptFilename), lines, StandardCharsets.UTF_8, - StandardOpenOption.CREATE, StandardOpenOption.TRUNCATE_EXISTING); - NexmarkUtils.console(null, "saved javascript to file %s.", javascriptFilename); + StandardOpenOption.CREATE, StandardOpenOption.TRUNCATE_EXISTING); + NexmarkUtils.console("saved javascript to file %s.", javascriptFilename); } } diff --git a/integration/java/src/main/java/com/google/cloud/dataflow/integration/nexmark/NexmarkGoogleRunner.java b/integration/java/src/main/java/com/google/cloud/dataflow/integration/nexmark/NexmarkGoogleRunner.java index 660de60446d8..e54ff0d4f0dd 100644 --- a/integration/java/src/main/java/com/google/cloud/dataflow/integration/nexmark/NexmarkGoogleRunner.java +++ b/integration/java/src/main/java/com/google/cloud/dataflow/integration/nexmark/NexmarkGoogleRunner.java @@ -65,7 +65,7 @@ /** * Run a singe Nexmark query using a given configuration on Google Dataflow. */ -class NexmarkGoogleRunner { +class NexmarkGoogleRunner extends NexmarkRunner { /** * How long to let streaming pipeline run after all events have been generated and we've * seen no activity. @@ -98,156 +98,17 @@ class NexmarkGoogleRunner { */ private static final Duration MIN_WINDOW = Duration.standardMinutes(2); - // Following is valid over all runs. - - private final NexmarkGoogleDriver.NexmarkGoogleOptions options; - - @Nullable - private final String outputPath; - - private final String pubsubTopic; - - private final boolean monitorJobs; - - // Following is valid per-run only. - - /** - * Which configuration should we run. - */ - @Nullable - private NexmarkConfiguration configuration; - - /** - * Accumulate the pub/sub subscriptions etc which should be cleaned up on end of run. - */ - @Nullable - private PubsubHelper pubsub; - - /** - * Pipeline 'result' for the publishing pipeline if in pub/sub COMBINED mode. - */ - @Nullable - private PipelineResult publisherResult; - - /** - * Result for the main query pipeline. - */ - @Nullable - private PipelineResult mainResult; - - /** - * Monitor for published events if in pub/sub COMBINED mode. - */ - @Nullable - private Monitor publisherMonitor; - - /** - * Query name. - */ - @Nullable - private String queryName; - - /** - * If sending events via pub/sub, the full topic name to use. - */ - @Nullable - private String inputTopic; - - /** - * If true, make sure all topic, subscription and gcs file names are unique. - */ - private final boolean uniqify; - - /** - * If true, manage the creation and cleanup of topics, subscriptions and gcs files. - */ - private final boolean manageResources; - - public NexmarkGoogleRunner( - NexmarkGoogleDriver.NexmarkGoogleOptions options, @Nullable String outputPath, - String pubsubTopic, boolean monitorJobs, boolean uniqify, boolean manageResources) { - this.options = options; - this.outputPath = outputPath != null && outputPath.isEmpty() ? null : outputPath; - this.pubsubTopic = pubsubTopic != null && pubsubTopic.isEmpty() ? null : pubsubTopic; - this.monitorJobs = monitorJobs; - this.uniqify = uniqify; - this.manageResources = manageResources; - } - - /** - * Return a pub/sub helper. - * - * @throws IOException - */ - private PubsubHelper getPubsub() throws IOException { - if (pubsub == null) { - pubsub = PubsubHelper.create(options, options.getProject()); - } - return pubsub; + public NexmarkGoogleRunner(NexmarkGoogleDriver.NexmarkGoogleOptions options) { + super(options); } - /** - * Return a source of synthetic events. - */ - private PCollection createSyntheticSource(Pipeline p) { - if (p.getRunner() instanceof DirectPipelineRunner || !options.isStreaming()) { - return p.apply(NexmarkUtils.batchEventsSource(queryName, configuration)); - } else { - return p.apply(NexmarkUtils.streamEventsSource(queryName, configuration)); - } + @Override + protected boolean isStreaming() { + return options.isStreaming(); } - /** - * Return pub/sub sink. - */ - private PubsubIO.Write.Bound pubsubEventSink() { - PubsubIO.Write.Bound io = - PubsubIO.Write.named(queryName + ".Write(" + inputTopic + ")") - .topic(inputTopic) - .idLabel(NexmarkUtils.PUBSUB_ID) - .withCoder(Event.CODER) - .named(queryName + ".PubsubSourceWrite"); - if (!configuration.usePubsubPublishTime) { - io = io.timestampLabel(NexmarkUtils.PUBSUB_TIMESTAMP); - } - return io; - } - - /** - * Return pub/sub source. - */ - private PubsubIO.Read.Bound pubsubEventSource(String subscription) { - PubsubIO.Read.Bound io = - PubsubIO.Read.named(queryName + ".Read(" + subscription + ")") - .subscription(subscription) - .idLabel(NexmarkUtils.PUBSUB_ID) - .withCoder(Event.CODER) - .named(queryName + ".PubsubSourceRead"); - if (!configuration.usePubsubPublishTime) { - io = io.timestampLabel(NexmarkUtils.PUBSUB_TIMESTAMP); - } - return io; - } - - /** - * Return pub/sub sink for results. - */ - private PubsubIO.Write.Bound pubsubResultSink(String topic) { - PubsubIO.Write.Bound io = - PubsubIO.Write.named(queryName + ".Write(" + topic + ")") - .topic(topic) - .idLabel(NexmarkUtils.PUBSUB_ID) - .named(queryName + ".PubsubSinkWrite"); - if (!configuration.usePubsubPublishTime) { - io = io.timestampLabel(NexmarkUtils.PUBSUB_TIMESTAMP); - } - return io; - } - - /** - * Return number of cores for configuration's machine type. - */ - private int coresPerWorker(DataflowPipelineOptions options) { + @Override + protected int coresPerWorker() { String machineType = options.getWorkerMachineType(); if (machineType == null || machineType.isEmpty()) { return 1; @@ -263,465 +124,52 @@ private int coresPerWorker(DataflowPipelineOptions options) { } } - /** - * Return source of events for this run, or null if we are simply publishing events - * to pub/sub. - * - * @throws IOException - * @throws InterruptedException - */ - private PCollection createSource(Pipeline p, long now) - throws IOException, InterruptedException { - PCollection source = null; - switch (configuration.sourceType) { - case DIRECT: - source = createSyntheticSource(p); - break; - case AVRO: - if (options.getInputFilePrefix() == null) { - throw new RuntimeException( - "If sourceType is AVRO, --inputFilePrefix must be specified."); - } - PCollection preTimestamp = p.apply(AvroIO.Read.named("ReadFromAvro") - .from(options.getInputFilePrefix() - + "*.avro") - .withSchema(Event.class)); - source = preTimestamp.apply("adjust timestamp", - ParDo.of(NexmarkQuery.EVENT_TIMESTAMP_FROM_DATA)); - break; - case PUBSUB: - if (pubsubTopic == null) { - throw new RuntimeException("must supply a --pubsubTopic if using --sourceType=PUBSUB"); - } - - // Check some flags. - switch (configuration.pubSubMode) { - case SUBSCRIBE_ONLY: - break; - case PUBLISH_ONLY: - if (manageResources && options.getRunner() != BlockingDataflowPipelineRunner.class) { - throw new RuntimeException( - "If --manageResources=true and --pubSubMode=PUBLISH_ONLY then " - + "--runner=BlockingDataflowPipelineRunner so that this program " - + "can cleanup the pub/sub topic on exit."); - } - break; - case COMBINED: - if (options.getRunner() != DataflowPipelineRunner.class || !monitorJobs) { - throw new RuntimeException( - "if --pubSubMode=COMBINED then --runner=DataflowPipelineRunner and " - + "--monitorJobs=true so that the two pipelines can be managed."); - } - break; - } - - // Choose a topic name for events, and optionally topics to use to synchronize - // the publisher and subscriber jobs. - String shortTopic; - if (uniqify) { - // Salt the topic name so we can run multiple jobs in parallel. - shortTopic = String.format("%s_%d_%s_source", pubsubTopic, now, queryName); - } else { - shortTopic = String.format("%s_%s_source", pubsubTopic, queryName); - } - String shortSubscription = shortTopic; - - // Create/confirm the topic. - if (!manageResources || configuration.pubSubMode == PubSubMode.SUBSCRIBE_ONLY) { - // The topic should already have been created by the user or - // a companion 'PUBLISH_ONLY' process. - inputTopic = getPubsub().reuseTopic(shortTopic); - } else { - // Create a fresh topic to loopback via. It will be destroyed when the - // (necessarily blocking) job is done. - inputTopic = getPubsub().createTopic(shortTopic); - } - - // Create/confirm the subscription. - String subscription = null; - if (configuration.pubSubMode == PubSubMode.PUBLISH_ONLY) { - // Nothing to consume. - } else if (!manageResources) { - // The subscription should already have been created by the user. - subscription = getPubsub().reuseSubscription(shortTopic, shortSubscription); - } else { - subscription = getPubsub().createSubscription(shortTopic, shortSubscription); - } - - // Setup the sink for the publisher. - switch (configuration.pubSubMode) { - case SUBSCRIBE_ONLY: - // Nothing to publish. - break; - case PUBLISH_ONLY: - // Send synthesized events to pub/sub in this job. - createSyntheticSource(p).apply(NexmarkUtils.snoop(queryName)).apply(pubsubEventSink()); - break; - case COMBINED: - if (options.getRunner() != DataflowPipelineRunner.class || !monitorJobs) { - throw new RuntimeException( - "if --pubSubMode=COMBINED then you must use --runner=DataflowPipelineRunner " - + "and --monitorJobs=true so that the publisher job can be shutdown cleanly"); - } - // Send synthesized events to pub/sub 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. - String jobName = options.getJobName(); - String appName = options.getAppName(); - options.setJobName("p-" + jobName); - options.setAppName("p-" + appName); - int coresPerWorker = coresPerWorker(options); - 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"); - Pipeline q = Pipeline.create(options); - createSyntheticSource(q) - .apply(publisherMonitor.getTransform()) - .apply(pubsubEventSink()); - PrintStream stdout = System.out; - try { - // Suppress output of publisher job; it makes the output of the command harder to - // interpret. - System.setOut(new PrintStream(new OutputStream() { - @Override - public void write(int b) { - } - })); - publisherResult = q.run(); - } finally { - System.setOut(stdout); - } - System.out.println( - "Publisher job running as " + ((DataflowPipelineJob) publisherResult).getJobId()); - options.setJobName(jobName); - options.setAppName(appName); - options.setMaxNumWorkers(options.getMaxNumWorkers()); - options.setNumWorkers(options.getNumWorkers()); - waitForPublisherPreload(); - 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 = p.apply(pubsubEventSource(subscription)); - break; - } - break; - } - return source; - } - - /** - * Consume the given results. - * - * @throws IOException - * @throws InterruptedException - */ - private void createSink(PCollection> results, long now) - throws IOException, InterruptedException { - - if (NexmarkUtils.SinkType.COUNT_ONLY == configuration.sinkType) { - results.apply(NexmarkUtils.devNull(queryName)); - return; - } - PCollection formattedResults = results.apply(NexmarkUtils.format(queryName)); - - if (configuration.logResults) { - formattedResults = formattedResults.apply(NexmarkUtils.log(queryName)); - } - - switch (configuration.sinkType) { - case DEVNULL: - // Discard all results - formattedResults.apply(NexmarkUtils.devNull(queryName)); - break; - case PUBSUB: - // Publish results to pubsub. Don't bother consuming them, the published - // results will be discarded when we delete the 'sink' topic. - if (pubsubTopic == null) { - throw new RuntimeException("must supply a --pubsubTopic if using --sinkType=PUBSUB"); - } - String shortTopic; - if (uniqify) { - shortTopic = String.format("%s_%d_%s_sink", pubsubTopic, now, queryName); - } else { - shortTopic = String.format("%s_%s_sink", pubsubTopic, queryName); - } - String topic; - if (!manageResources) { - topic = getPubsub().reuseTopic(shortTopic); - } else { - topic = getPubsub().createTopic(shortTopic); - } - formattedResults.apply(pubsubResultSink(topic)); - break; - case TEXT: - // Write results to text. Only works in batch mode. - if (options.isStreaming()) { - throw new RuntimeException("can only use --sinkType=TEXT with --streaming=false"); - } - if (outputPath == null) { - throw new RuntimeException("must supply an --outputPath if using --sinkType=TEXT"); - } - String fullFilename; - if (uniqify) { - fullFilename = String.format("%s/nexmark_%d_%s.txt", outputPath, now, queryName); - } else { - fullFilename = String.format("%s/nexmark_%s.txt", outputPath, queryName); - } - formattedResults.apply(TextIO.Write.to(fullFilename).named(queryName + ".Text")); - break; - case AVRO: - NexmarkUtils.console(null, "WARNING: with --sinkType=AVRO, actual query results will be " - + "discarded.", outputPath); - break; - case BIGQUERY: - NexmarkUtils.console(null, "Writing events to BigQuery table %s", outputPath); - formattedResults - .apply(ParDo.of(new StringToTableRow())) - .apply(BigQueryIO.Write.named("WriteBigQuery(Events)") - .to(options.getProject() + ":nexmark.table_" - + new Random().nextInt(Integer.MAX_VALUE)) - .withSchema(new TableSchema().setFields( - new ArrayList() { - { - add(new TableFieldSchema().setName("event") - .setType("STRING")); - } - }))); - break; - case COUNT_ONLY: - // Short-circuited above. - throw new RuntimeException(); - } + @Override + protected int maxNumWorkers() { + return Math.max(options.getNumWorkers(), options.getMaxNumWorkers()); } - private static class StringToTableRow extends DoFn { - @Override - public void processElement(ProcessContext c) { - TableRow row = new TableRow(); - row.set("event", c.element()); - c.output(row); - } + @Override + protected boolean isBlocking() { + return options.getRunner() == BlockingDataflowPipelineRunner.class; } - /** - * Sink all raw Events in {@code source} to {@code this.outputPath}. - *

- * 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 sinkToAvro(final PCollection source) { - if (options.isStreaming()) { - throw new RuntimeException("can only use Avro SinkType with --streaming=false"); - } - if (outputPath == null) { - throw new RuntimeException("Must supply an --outputPath if using --sinkType=AVRO"); - } - NexmarkUtils.console(null, "Writing events in Avro to %s", outputPath); - source.apply(AvroIO.Write.named("WriteAvro(Events)") - .to(outputPath + "/event") - .withSuffix(".avro") - .withSchema(Event.class)); - source.apply(NexmarkQuery.JUST_BIDS) - .apply(AvroIO.Write.named("WriteAvro(Bids)") - .to(outputPath + "/bid") - .withSuffix(".avro") - .withSchema(Bid.class)); - source.apply(NexmarkQuery.JUST_NEW_AUCTIONS) - .apply(AvroIO.Write.named("WriteAvro(Auctions)") - .to(outputPath + "/auction") - .withSuffix(".avro") - .withSchema(Auction.class)); - source.apply(NexmarkQuery.JUST_NEW_PERSONS) - .apply(AvroIO.Write.named("WriteAvro(People)") - .to(outputPath + "/person") - .withSuffix(".avro") - .withSchema(Person.class)); + @Override + protected boolean canMonitor() { + return options.getRunner() == DataflowPipelineRunner.class; } - /** - * Run {@code configuration} and return its performance (if using DataflowPipelineRunner - * and {@link #monitorJobs} is true). - * - * @throws IOException - * @throws InterruptedException - */ - @Nullable - public NexmarkPerf run(NexmarkConfiguration runConfiguration) - throws IOException, InterruptedException { - // - // Setup per-run state. - // - Preconditions.checkState(configuration == null); - Preconditions.checkState(pubsub == null); - Preconditions.checkState(publisherResult == null); - Preconditions.checkState(mainResult == null); - Preconditions.checkState(queryName == null); - configuration = runConfiguration; - - // GCS URI patterns to delete on exit. - List pathsToDelete = new ArrayList<>(); - + @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 { - NexmarkUtils.console(null, "running %s", configuration.toShortString()); - - if (configuration.numEvents < 0) { - NexmarkUtils.console(null, "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 (configuration.justModelResultRate) { - 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); - - // 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) { - if (monitorJobs && options.getRunner() != DataflowPipelineRunner.class) { - throw new RuntimeException("can only use --monitorJobs=true if also " - + "using --runner=DataflowPipelineRunner"); - } - - // Optionally sink events in Avro format - if (configuration.sinkType == NexmarkUtils.SinkType.AVRO) { - this.sinkToAvro(source); - } - - // Special hacks for Query 10 (big logger). - if (configuration.query == 10) { - String path = null; - if (outputPath != null) { - if (uniqify) { - path = String.format("%s/%d_logs", outputPath, now); - } else { - path = String.format("%s/logs", outputPath); - } - } - ((Query10) query).setOutputPath(path); - ((Query10) query).setMaxNumWorkers( - Math.max(options.getMaxNumWorkers(), options.getNumWorkers())); - if (path != null && manageResources) { - pathsToDelete.add(path + "/**"); - } - } - - // Apply query. - PCollection> results = source.apply(query); - - if (configuration.assertCorrectness) { - 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(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. - DataflowAssert.that(results).satisfies(model.assertionFor()); - } - - // Output results. - createSink(results, now); - } - - mainResult = p.run(); - return monitor(query); + builder.build(options); } finally { - // - // Cleanup per-run state. - // - if (pubsub != null) { - // Delete any subscriptions and topics we created. - pubsub.cleanup(); - pubsub = null; - } - // TODO: Find a way to cleanup pathsToDelete robustly without depending on gsutil. - configuration = null; - publisherResult = null; - mainResult = null; - queryName = null; - } - } - - /** - * 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(null, "Query%d: only %d samples", model.configuration.query, n); - } else { - NexmarkUtils.console(null, "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)); + 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 produced at - * least {@code configuration.preloadSeconds} worth of events. - * - * @throws IOException - * @throws InterruptedException + * Monitor the progress of the publisher job. Return when it has been generating events for + * at least {@code configuration.preloadSeconds}. */ - private void waitForPublisherPreload() throws IOException, InterruptedException { - if (!monitorJobs) { + @Override + protected void waitForPublisherPreload() { + Preconditions.checkNotNull(publisherMonitor); + Preconditions.checkNotNull(publisherResult); + if (!options.getMonitorJobs()) { return; } if (options.getRunner() != DataflowPipelineRunner.class) { @@ -734,58 +182,66 @@ private void waitForPublisherPreload() throws IOException, InterruptedException return; } - NexmarkUtils.console(null, "waiting for publisher to pre-load"); + 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(); - long numEvents = getLong(job, publisherMonitor.getElementCounter()); - if (numEvents > 0) { - int waitSeconds = configuration.preloadSeconds; - NexmarkUtils.console(null, "%s publisher (Saw first event, waiting %ds for preload)", state, - configuration.preloadSeconds); - while (waitSeconds > 0) { - NexmarkUtils.console( - null, "%s publisher (%ds until preload is done)", state, waitSeconds); - Thread.sleep(TimeUnit.SECONDS.toMillis(Math.min(60, waitSeconds))); - waitSeconds -= 60; - } - return; - } - switch (state) { case UNKNOWN: - case STOPPED: - case RUNNING: // Keep waiting. + NexmarkUtils.console("%s publisher (%d events)", state, numEvents); break; + case STOPPED: case DONE: - // All done. - NexmarkUtils.console(null, "publisher pipeline done"); - return; case CANCELLED: case FAILED: case UPDATED: - // Something went wrong. - NexmarkUtils.console(null, "publisher pipeline failed", state, numEvents); + 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; } - Thread.sleep(PERF_DELAY.getMillis()); + 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. - * - * @throws IOException - * @throws InterruptedException */ + @Override @Nullable - private NexmarkPerf monitor(NexmarkQuery query) throws IOException, InterruptedException { - if (!monitorJobs) { + protected NexmarkPerf monitor(NexmarkQuery query) { + if (!options.getMonitorJobs()) { return null; } if (options.getRunner() != DataflowPipelineRunner.class) { @@ -794,89 +250,100 @@ private NexmarkPerf monitor(NexmarkQuery query) throws IOException, InterruptedE if (!(mainResult instanceof DataflowPipelineJob)) { return null; } + // If we are not in debug mode, we have no event count or result count monitors. - boolean monitorsActive = configuration.debug; + boolean monitorsActive = configuration.debug; if (monitorsActive) { - NexmarkUtils.console(null, "waiting for main pipeline to 'finish'"); + NexmarkUtils.console("waiting for main pipeline to 'finish'"); } else { - NexmarkUtils.console(null, "debug=false, so job will not self-cancel"); + NexmarkUtils.console("--debug=false, so job will not self-cancel"); } - DataflowPipelineJob job = (DataflowPipelineJob) mainResult; DataflowPipelineJob publisherJob = (DataflowPipelineJob) publisherResult; List snapshots = new ArrayList<>(); - Instant start = Instant.now(); - Instant end = - options.getRunningTimeMinutes() != null - ? start.plus(Duration.standardMinutes(options.getRunningTimeMinutes())) - : new Instant(Long.MAX_VALUE); + long startMsSinceEpoch = System.currentTimeMillis(); + long endMsSinceEpoch = -1; + if (options.getRunningTimeMinutes() != null) { + endMsSinceEpoch = startMsSinceEpoch + + Duration.standardMinutes(options.getRunningTimeMinutes()).getMillis(); + } if (options.getPreloadSeconds() != null) { - end = end.minus(Duration.standardSeconds(options.getPreloadSeconds())); + endMsSinceEpoch -= Duration.standardSeconds(options.getPreloadSeconds()).getMillis(); } - Instant lastActivity = null; + long lastActivityMsSinceEpoch = -1; NexmarkPerf perf = null; boolean waitingForShutdown = false; boolean publisherCancelled = false; List errors = new ArrayList<>(); while (true) { - Instant now = Instant.now(); - if (now.isAfter(end) && !waitingForShutdown) { - NexmarkUtils.console(null, "Reached end of test, cancelling job"); - job.cancel(); + long now = System.currentTimeMillis(); + if (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) { - publisherJob.cancel(); + 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( - now, "%s %s%s", state, queryName, waitingForShutdown ? " (waiting for shutdown)" : ""); + NexmarkUtils.console("%s %s%s", state, queryName, + waitingForShutdown ? " (waiting for shutdown)" : ""); NexmarkPerf currPerf; if (monitorsActive) { - currPerf = currentPerf(start, now, job, snapshots, query.eventMonitor, - query.resultMonitor); + currPerf = currentPerf(startMsSinceEpoch, now, job, snapshots, query.eventMonitor, + query.resultMonitor); } else { currPerf = null; } if (perf == null || perf.anyActivity(currPerf)) { - lastActivity = now; + lastActivityMsSinceEpoch = now; } if (options.isStreaming() && !waitingForShutdown) { - Duration quietFor = new Duration(lastActivity, now); + Duration quietFor = new Duration(lastActivityMsSinceEpoch, now); if (query.getFatalCount() != null && getLong(job, query.getFatalCount()) > 0) { - NexmarkUtils.console(now, "job has fatal errors, cancelling."); + NexmarkUtils.console("job has fatal errors, cancelling."); errors.add(String.format("Pipeline reported %s fatal errors", query.getFatalCount())); waitingForShutdown = true; } else if (monitorsActive && configuration.numEvents > 0 && currPerf.numEvents == configuration.numEvents && currPerf.numResults >= 0 && quietFor.isLongerThan(DONE_DELAY)) { - NexmarkUtils.console(now, "streaming query appears to have finished, cancelling job."); + NexmarkUtils.console("streaming query appears to have finished, cancelling job."); waitingForShutdown = true; } else if (quietFor.isLongerThan(STUCK_TERMINATE_DELAY)) { - NexmarkUtils.console( - now, "streaming query appears to have gotten stuck, cancelling job."); + 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( - now, "WARNING: streaming query appears to have been stuck for %d min.", + 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, start)); + errors.addAll(checkWatermarks(job, startMsSinceEpoch)); if (waitingForShutdown) { - job.cancel(); + try { + job.cancel(); + } catch (IOException e) { + throw new RuntimeException("Unable to cancel main job: ", e); + } } } @@ -911,24 +378,36 @@ private NexmarkPerf monitor(NexmarkQuery query) throws IOException, InterruptedE break; } - if (lastActivity.equals(now)) { - NexmarkUtils.console(now, "new perf %s", perf); + if (lastActivityMsSinceEpoch == now) { + NexmarkUtils.console("new perf %s", perf); } else { - NexmarkUtils.console(now, "no activity"); + NexmarkUtils.console("no activity"); } - Thread.sleep(PERF_DELAY.getMillis()); + try { + Thread.sleep(PERF_DELAY.getMillis()); + } catch (InterruptedException e) { + Thread.interrupted(); + NexmarkUtils.console("Interrupted: pipeline is still running"); + } } perf.errors = errors; perf.snapshots = snapshots; - NexmarkUtils.console(null, "final perf %s", perf); + NexmarkUtils.console("final perf %s", perf); if (publisherResult != null) { - if (publisherCancelled) { - publisherJob.waitToFinish(5, TimeUnit.MINUTES, null); - } else { - publisherJob.cancel(); + try { + if (publisherCancelled) { + publisherJob.waitToFinish(5, TimeUnit.MINUTES, null); + } else { + publisherJob.cancel(); + } + } 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); } } @@ -957,8 +436,8 @@ private MetricType getMetricType(MetricUpdate metric) { *

*

Returns a list of errors detected. */ - private List checkWatermarks(DataflowPipelineJob job, Instant start) { - Instant now = Instant.now(); + private List checkWatermarks(DataflowPipelineJob job, long startMsSinceEpoch) { + long now = System.currentTimeMillis(); List errors = new ArrayList<>(); try { JobMetrics metricResponse = job.getDataflowClient() @@ -985,8 +464,9 @@ private List checkWatermarks(DataflowPipelineJob job, Instant start) { Instant updateTime = Instant.parse(metric.getUpdateTime()); if (options.getWatermarkValidationDelaySeconds() == null - || now.isAfter(start.plus( - Duration.standardSeconds(options.getWatermarkValidationDelaySeconds())))) { + || now > startMsSinceEpoch + + Duration.standardSeconds(options.getWatermarkValidationDelaySeconds()) + .getMillis()) { Duration threshold = null; if (type == MetricType.SYSTEM_WATERMARK && options.getMaxSystemLagSeconds() != null) { threshold = Duration.standardSeconds(options.getMaxSystemLagSeconds()); @@ -999,19 +479,19 @@ private List checkWatermarks(DataflowPipelineJob job, Instant start) { 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(null, msg); + NexmarkUtils.console(msg); } } - if (!foundWatermarks) { - NexmarkUtils.console(null, "No known watermarks in update: " + metrics); - if (now.isAfter(start.plus(Duration.standardMinutes(5)))) { - errors.add("No known watermarks found. Metrics were " + metrics); - } + } + 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(null, "Warning: failed to get JobMetrics: " + e); + NexmarkUtils.console("Warning: failed to get JobMetrics: " + e); } return errors; @@ -1021,7 +501,7 @@ private List checkWatermarks(DataflowPipelineJob job, Instant start) { * Return the current performance given {@code eventMonitor} and {@code resultMonitor}. */ private NexmarkPerf currentPerf( - Instant start, Instant now, DataflowPipelineJob job, + long startMsSinceEpoch, long now, DataflowPipelineJob job, List snapshots, Monitor eventMonitor, Monitor resultMonitor) { NexmarkPerf perf = new NexmarkPerf(); @@ -1082,7 +562,7 @@ private NexmarkPerf currentPerf( } if (eventStart >= 0) { - perf.startupDelaySec = (eventStart - start.getMillis()) / 1000.0; + perf.startupDelaySec = (eventStart - startMsSinceEpoch) / 1000.0; } if (resultStart >= 0 && eventStart >= 0 && resultStart >= eventStart) { @@ -1096,14 +576,14 @@ private NexmarkPerf currentPerf( if (resultEnd >= 0) { // Fill in the shutdown delay assuming the job has now finished. - perf.shutdownDelaySec = (now.getMillis() - resultEnd) / 1000.0; + 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.getMillis() - start.getMillis()) / 1000.0; + snapshot.secSinceStart = (now - startMsSinceEpoch) / 1000.0; snapshot.runtimeSec = perf.runtimeSec; snapshot.numEvents = numEvents; snapshot.numResults = numResults; @@ -1142,8 +622,8 @@ private void captureSteadyState(NexmarkPerf perf, List aggregator) * Return the current value for a time counter, or -1 if can't be retrieved. */ private long getTimestamp( - Instant now, DataflowPipelineJob job, Aggregator aggregator) { + 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.getMillis()) > Duration.standardDays(10000).getMillis()) { + if (Math.abs(value - now) > Duration.standardDays(10000).getMillis()) { return -1; } return value; diff --git a/integration/java/src/main/java/com/google/cloud/dataflow/integration/nexmark/NexmarkPerf.java b/integration/java/src/main/java/com/google/cloud/dataflow/integration/nexmark/NexmarkPerf.java index fe74f65f7118..22cc18ebdd0c 100644 --- a/integration/java/src/main/java/com/google/cloud/dataflow/integration/nexmark/NexmarkPerf.java +++ b/integration/java/src/main/java/com/google/cloud/dataflow/integration/nexmark/NexmarkPerf.java @@ -180,8 +180,12 @@ public String toString() { * * @throws IOException */ - public static NexmarkPerf fromString(String string) throws IOException { - return NexmarkUtils.MAPPER.readValue(string, NexmarkPerf.class); + 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); + } } /** diff --git a/integration/java/src/main/java/com/google/cloud/dataflow/integration/nexmark/NexmarkQuery.java b/integration/java/src/main/java/com/google/cloud/dataflow/integration/nexmark/NexmarkQuery.java index dc0f69c73c7c..8f77ab102165 100644 --- a/integration/java/src/main/java/com/google/cloud/dataflow/integration/nexmark/NexmarkQuery.java +++ b/integration/java/src/main/java/com/google/cloud/dataflow/integration/nexmark/NexmarkQuery.java @@ -152,20 +152,22 @@ public void processElement(ProcessContext c) { } }); - public static final DoFn EVENT_TIMESTAMP_FROM_DATA = 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 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. diff --git a/integration/java/src/main/java/com/google/cloud/dataflow/integration/nexmark/NexmarkRunner.java b/integration/java/src/main/java/com/google/cloud/dataflow/integration/nexmark/NexmarkRunner.java index d28b0979ce91..b616ac2e8b5f 100644 --- a/integration/java/src/main/java/com/google/cloud/dataflow/integration/nexmark/NexmarkRunner.java +++ b/integration/java/src/main/java/com/google/cloud/dataflow/integration/nexmark/NexmarkRunner.java @@ -1,275 +1,432 @@ package com.google.cloud.dataflow.integration.nexmark; -import java.io.IOException; +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.cloud.dataflow.sdk.Pipeline; +import com.google.cloud.dataflow.sdk.PipelineResult; +import com.google.cloud.dataflow.sdk.io.AvroIO; +import com.google.cloud.dataflow.sdk.io.BigQueryIO; +import com.google.cloud.dataflow.sdk.io.PubsubIO; +import com.google.cloud.dataflow.sdk.io.TextIO; +import com.google.cloud.dataflow.sdk.testing.DataflowAssert; +import com.google.cloud.dataflow.sdk.transforms.DoFn; +import com.google.cloud.dataflow.sdk.transforms.ParDo; +import com.google.cloud.dataflow.sdk.values.PCollection; +import com.google.cloud.dataflow.sdk.values.TimestampedValue; +import com.google.common.base.Preconditions; +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 javax.annotation.Nullable; /** * Run a single Nexmark query using a given configuration. */ -public class NexmarkRunner { - private final Options options; +public abstract class NexmarkRunner { + /** + * Options shared by all runs. + */ + protected final O options; /** - * Which configuration should we run. + * Which configuration we are running. */ @Nullable - private NexmarkConfiguration configuration; + protected NexmarkConfiguration configuration; /** * Accumulate the pub/sub subscriptions etc which should be cleaned up on end of run. */ @Nullable - private PubsubHelper pubsub; + protected PubsubHelper pubsub; /** - * Monitor for published events if in pub/sub COMBINED mode. + * If in --pubsubMode=COMBINED, the event monitor for the publisher pipeline. Otherwise null. */ @Nullable - private Monitor publisherMonitor; + protected Monitor publisherMonitor; /** - * Query name. + * If in --pubsubMode=COMBINED, the pipeline result for the publisher pipeline. Otherwise null. */ @Nullable - private String queryName; + protected PipelineResult publisherResult; /** - * If sending events via pub/sub, the full topic name to use. + * Result for the main pipeline. */ @Nullable - private String inputTopic; + protected PipelineResult mainResult; - public NexmarkRunner(Options options) { + /** + * Query name we are running. + */ + @Nullable + protected String queryName; + + public NexmarkRunner(O options) { this.options = options; } /** - * Return a pub/sub helper. - * - * @throws IOException + * Return a Pubsub helper. */ - private PubsubHelper getPubsub() throws IOException { + private PubsubHelper getPubsub() { if (pubsub == null) { - pubsub = PubsubHelper.create(options, options.getProject()); + 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 is blocking. + */ + protected abstract boolean isBlocking(); + + /** + * Return true if runner can monitor running jobs. + */ + protected abstract boolean canMonitor(); + + protected interface PipelineBuilder { + void build(O 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 null immediately. + */ + protected abstract NexmarkPerf monitor(NexmarkQuery query); + + // ================================================================================ + // Basic sources and sinks + // ================================================================================ + /** * Return a source of synthetic events. */ - private PCollection createSyntheticSource(Pipeline p) { - if (p.getRunner() instanceof DirectPipelineRunner || !options.isStreaming()) { + private PCollection sourceFromSynthetic(Pipeline p) { + if (isStreaming()) { + NexmarkUtils.console("Generating %d events in batch mode", configuration.numEvents); return p.apply(NexmarkUtils.batchEventsSource(queryName, configuration)); } else { + NexmarkUtils.console("Generating %d events in streaming mode", configuration.numEvents); return p.apply(NexmarkUtils.streamEventsSource(queryName, configuration)); } } /** - * Return pub/sub sink. + * Return source of events from Pubsub. */ - private PubsubIO.Write.Bound pubsubEventSink() { - PubsubIO.Write.Bound io = - PubsubIO.Write.named(queryName + ".Write(" + inputTopic + ")") - .topic(inputTopic) - .idLabel(NexmarkUtils.PUBSUB_ID) - .withCoder(Event.CODER) - .named(queryName + ".PubsubSourceWrite"); + private PCollection sourceEventsFromPubsub(Pipeline p, long now) { + String baseTopic = options.getPubsubTopic(); + if (baseTopic == null || baseTopic.isEmpty()) { + throw new RuntimeException("Missing --pubsubTopic"); + } + String shortTopic; + if (options.getUniqify()) { + // Salt the topic name so we can run multiple jobs in parallel. + shortTopic = String.format("%s_%d_%s_source", baseTopic, now, queryName); + } else { + shortTopic = String.format("%s_%s_source", baseTopic, queryName); + } + String shortSubscription = shortTopic; + // 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); + } else { + subscription = getPubsub().createSubscription(shortTopic, shortSubscription); + } + NexmarkUtils.console("Reading events from Pubsub %s", subscription); + PubsubIO.Read.Bound io = + PubsubIO.Read.named(queryName + ".ReadPubsubEvents(" + subscription + ")") + .subscription(subscription) + .idLabel(NexmarkUtils.PUBSUB_ID) + .withCoder(Event.CODER); if (!configuration.usePubsubPublishTime) { io = io.timestampLabel(NexmarkUtils.PUBSUB_TIMESTAMP); } - return io; + return p.apply(io); } /** - * Return pub/sub source. + * Return Avro source of events from {@code options.getInputFilePrefix}. */ - private PubsubIO.Read.Bound pubsubEventSource(String subscription) { - PubsubIO.Read.Bound io = - PubsubIO.Read.named(queryName + ".Read(" + subscription + ")") - .subscription(subscription) - .idLabel(NexmarkUtils.PUBSUB_ID) - .withCoder(Event.CODER) - .named(queryName + ".PubsubSourceRead"); + private PCollection sourceFromAvro(Pipeline p) { + String filename = options.getInputFilePrefix(); + if (filename == null || filename.isEmpty()) { + throw new RuntimeException("Missing --inputFilePrefix"); + } + NexmarkUtils.console("Reading events from Avro files at %s", filename); + return p + .apply(AvroIO.Read.named(queryName + ".ReadAvroEvents(" + filename + ")") + .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 baseTopic = options.getPubsubTopic(); + if (baseTopic == null || baseTopic.isEmpty()) { + throw new RuntimeException("Missing --pubsubTopic"); + } + String shortTopic; + if (options.getUniqify()) { + // Salt the topic name so we can run multiple jobs in parallel. + shortTopic = String.format("%s_%d_%s_source", baseTopic, now, queryName); + } else { + shortTopic = String.format("%s_%s_source", baseTopic, queryName); + } + // 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); + } else { + // Create a fresh topic to loopback via. It will be destroyed when the + // (necessarily blocking) job is done. + topic = getPubsub().createTopic(shortTopic); + } + NexmarkUtils.console("Writing events to Pubsub %s", topic); + PubsubIO.Write.Bound io = + PubsubIO.Write.named(queryName + ".WritePubsubEvents(" + topic + ")") + .topic(topic) + .idLabel(NexmarkUtils.PUBSUB_ID) + .withCoder(Event.CODER); if (!configuration.usePubsubPublishTime) { io = io.timestampLabel(NexmarkUtils.PUBSUB_TIMESTAMP); } - return io; + events.apply(io); } /** - * Return pub/sub sink for results. + * Send {@code formattedResults} to Pubsub. */ - private PubsubIO.Write.Bound pubsubResultSink(String topic) { + private void sinkResultsToPubsub(PCollection formattedResults, long now) { + String baseTopic = options.getPubsubTopic(); + if (baseTopic == null || baseTopic.isEmpty()) { + throw new RuntimeException("Missing --pubsubTopic"); + } + String shortTopic; + if (options.getUniqify()) { + shortTopic = String.format("%s_%d_%s_sink", baseTopic, now, queryName); + } else { + shortTopic = String.format("%s_%s_sink", baseTopic, queryName); + } + String topic; + if (!options.getManageResources()) { + topic = getPubsub().reuseTopic(shortTopic); + } else { + topic = getPubsub().createTopic(shortTopic); + } + NexmarkUtils.console("Writing results to Pubsub %s", topic); PubsubIO.Write.Bound io = - PubsubIO.Write.named(queryName + ".Write(" + topic + ")") + PubsubIO.Write.named(queryName + ".WritePubsubResults(" + topic + ")") .topic(topic) - .idLabel(NexmarkUtils.PUBSUB_ID) - .named(queryName + ".PubsubSinkWrite"); + .idLabel(NexmarkUtils.PUBSUB_ID); if (!configuration.usePubsubPublishTime) { io = io.timestampLabel(NexmarkUtils.PUBSUB_TIMESTAMP); } - return io; + formattedResults.apply(io); } /** - * Return number of cores for configuration's machine type. + * 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 int coresPerWorker(DataflowPipelineOptions options) { - String machineType = options.getWorkerMachineType(); - if (machineType == null || machineType.isEmpty()) { - return 1; - } - String[] split = machineType.split("-"); - if (split.length != 3) { - return 1; + private void sinkToAvro(PCollection source) { + String filename = options.getOutputPath(); + if (filename == null || filename.isEmpty()) { + throw new RuntimeException("Missing --outputPath"); + } + NexmarkUtils.console("Writing events to Avro files at %s", filename); + source.apply(AvroIO.Write.named(queryName + ".WriteAvroEvents(" + filename + ")") + .to(filename + "/event") + .withSuffix(".avro") + .withSchema(Event.class)); + source.apply(NexmarkQuery.JUST_BIDS) + .apply(AvroIO.Write.named(queryName + ".WriteAvroBids(" + filename + ")") + .to(filename + "/bid") + .withSuffix(".avro") + .withSchema(Bid.class)); + source.apply(NexmarkQuery.JUST_NEW_AUCTIONS) + .apply(AvroIO.Write.named(queryName + ".WriteAvroAuctions(" + filename + ")") + .to(filename + "/auction") + .withSuffix(".avro") + .withSchema(Auction.class)); + source.apply(NexmarkQuery.JUST_NEW_PERSONS) + .apply(AvroIO.Write.named(queryName + ".WriteAvroPeople(" + filename + ")") + .to(filename + "/person") + .withSuffix(".avro") + .withSchema(Person.class)); + } + + private static class StringToTableRow extends DoFn { + @Override + public void processElement(ProcessContext c) { + TableRow row = new TableRow(); + row.set("event", c.element()); + c.output(row); } - try { - return Integer.parseInt(split[2]); - } catch (NumberFormatException ex) { - return 1; + } + + /** + * Send {@code formattedResults} to text files. + */ + private void sinkToText(PCollection formattedResults, long now) { + String filename = options.getOutputPath(); + if (filename == null || filename.isEmpty()) { + throw new RuntimeException("Missing --outputPath"); + } + String fullFilename; + if (options.getUniqify()) { + fullFilename = String.format("%s/nexmark_%d_%s.txt", filename, now, queryName); + } else { + fullFilename = String.format("%s/nexmark_%s.txt", filename, queryName); } + NexmarkUtils.console("Writing results to text files at %s", fullFilename); + formattedResults.apply(TextIO.Write.named(queryName + ".WriteTextResults(" + fullFilename + ")") + .to(fullFilename)); + } + + /** + * Send {@code formattedResults} to BigQuery. + */ + private void sinkToBigQuery(PCollection formattedResults, long now) { + String tableName; + if (options.getUniqify()) { + tableName = String.format("%s:nexmark.table_%d", options.getProject(), now); + } else { + tableName = String.format("%s:nexmark.table", options.getProject()); + } + TableSchema schema = + new TableSchema() + .setFields(ImmutableList.of(new TableFieldSchema().setName("result") + .setType("STRING"))); + NexmarkUtils.console("Writing results to BigQuery table %s", tableName); + BigQueryIO.Write.Bound io = + BigQueryIO.Write.named(queryName + ".WriteBigQueryResults(" + tableName + ")") + .to(tableName) + .withSchema(schema); + formattedResults + .apply(ParDo.of(new StringToTableRow())) + .apply(io); } + // ================================================================================ + // Construct overall pipeline + // ================================================================================ + /** * Return source of events for this run, or null if we are simply publishing events - * to pub/sub. - * - * @throws IOException - * @throws InterruptedException + * to Pubsub. */ - private PCollection createSource(Pipeline p, long now) - throws IOException, InterruptedException { + private PCollection createSource(Pipeline p, final long now) { PCollection source = null; switch (configuration.sourceType) { case DIRECT: - source = createSyntheticSource(p); + source = sourceFromSynthetic(p); break; case AVRO: - if (options.getInputFilePrefix() == null) { - throw new RuntimeException( - "If sourceType is AVRO, --inputFilePrefix must be specified."); - } - PCollection preTimestamp = p.apply(AvroIO.Read.named("ReadFromAvro") - .from(options.getInputFilePrefix() - + "*.avro") - .withSchema(Event.class)); - source = preTimestamp.apply("adjust timestamp", - ParDo.of(NexmarkQuery.EVENT_TIMESTAMP_FROM_DATA)); + source = sourceFromAvro(p); break; case PUBSUB: - if (pubsubTopic == null) { - throw new RuntimeException("must supply a --pubsubTopic if using --sourceType=PUBSUB"); - } - // Check some flags. switch (configuration.pubSubMode) { case SUBSCRIBE_ONLY: break; case PUBLISH_ONLY: - if (manageResources && options.getRunner() != BlockingDataflowPipelineRunner.class) { + if (options.getManageResources() && !isBlocking()) { throw new RuntimeException( "If --manageResources=true and --pubSubMode=PUBLISH_ONLY then " - + "--runner=BlockingDataflowPipelineRunner so that this program " - + "can cleanup the pub/sub topic on exit."); + + "runner must be blocking so that this program " + + "can cleanup the Pubsub topic on exit."); } break; case COMBINED: - if (options.getRunner() != DataflowPipelineRunner.class || !monitorJobs) { + if (!canMonitor() || !options.getMonitorJobs()) { throw new RuntimeException( - "if --pubSubMode=COMBINED then --runner=DataflowPipelineRunner and " - + "--monitorJobs=true so that the two pipelines can be managed."); + "if --pubSubMode=COMBINED then you must use a monitoring runner " + + "and set --monitorJobs=true so that the two pipelines can be coordinated."); } break; } - // Choose a topic name for events, and optionally topics to use to synchronize - // the publisher and subscriber jobs. - String shortTopic; - if (uniqify) { - // Salt the topic name so we can run multiple jobs in parallel. - shortTopic = String.format("%s_%d_%s_source", pubsubTopic, now, queryName); - } else { - shortTopic = String.format("%s_%s_source", pubsubTopic, queryName); - } - String shortSubscription = shortTopic; - - // Create/confirm the topic. - if (!manageResources || configuration.pubSubMode == PubSubMode.SUBSCRIBE_ONLY) { - // The topic should already have been created by the user or - // a companion 'PUBLISH_ONLY' process. - inputTopic = getPubsub().reuseTopic(shortTopic); - } else { - // Create a fresh topic to loopback via. It will be destroyed when the - // (necessarily blocking) job is done. - inputTopic = getPubsub().createTopic(shortTopic); - } - - // Create/confirm the subscription. - String subscription = null; - if (configuration.pubSubMode == PubSubMode.PUBLISH_ONLY) { - // Nothing to consume. - } else if (!manageResources) { - // The subscription should already have been created by the user. - subscription = getPubsub().reuseSubscription(shortTopic, shortSubscription); - } else { - subscription = getPubsub().createSubscription(shortTopic, shortSubscription); - } - // Setup the sink for the publisher. switch (configuration.pubSubMode) { case SUBSCRIBE_ONLY: // Nothing to publish. break; case PUBLISH_ONLY: - // Send synthesized events to pub/sub in this job. - createSyntheticSource(p).apply(NexmarkUtils.snoop(queryName)).apply(pubsubEventSink()); + // Send synthesized events to Pubsub in this job. + sinkEventsToPubsub(sourceFromSynthetic(p).apply(NexmarkUtils.snoop(queryName)), now); break; case COMBINED: - if (options.getRunner() != DataflowPipelineRunner.class || !monitorJobs) { - throw new RuntimeException( - "if --pubSubMode=COMBINED then you must use --runner=DataflowPipelineRunner " - + "and --monitorJobs=true so that the publisher job can be shutdown cleanly"); - } - // Send synthesized events to pub/sub in separate publisher job. + // 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. - String jobName = options.getJobName(); - String appName = options.getAppName(); - options.setJobName("p-" + jobName); - options.setAppName("p-" + appName); - int coresPerWorker = coresPerWorker(options); - 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"); - Pipeline q = Pipeline.create(options); - createSyntheticSource(q) - .apply(publisherMonitor.getTransform()) - .apply(pubsubEventSink()); - PrintStream stdout = System.out; - try { - // Suppress output of publisher job; it makes the output of the command harder to - // interpret. - System.setOut(new PrintStream(new OutputStream() { - @Override - public void write(int b) { - } - })); - publisherResult = q.run(); - } finally { - System.setOut(stdout); - } - System.out.println( - "Publisher job running as " + ((DataflowPipelineJob) publisherResult).getJobId()); - options.setJobName(jobName); - options.setAppName(appName); - options.setMaxNumWorkers(options.getMaxNumWorkers()); - options.setNumWorkers(options.getNumWorkers()); - waitForPublisherPreload(); + 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(sourceFromSynthetic(sp).apply(publisherMonitor.getTransform()), + now); + publisherResult = sp.run(); + } + }); break; } @@ -281,7 +438,7 @@ public void write(int b) { case SUBSCRIBE_ONLY: case COMBINED: // Read events from pubsub. - source = p.apply(pubsubEventSource(subscription)); + source = sourceEventsFromPubsub(p, now); break; } break; @@ -290,20 +447,16 @@ public void write(int b) { } /** - * Consume the given results. - * - * @throws IOException - * @throws InterruptedException + * Consume {@code results}. */ - private void createSink(PCollection> results, long now) - throws IOException, InterruptedException { - - if (NexmarkUtils.SinkType.COUNT_ONLY == configuration.sinkType) { + 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)); + PCollection formattedResults = results.apply(NexmarkUtils.format(queryName)); if (configuration.logResults) { formattedResults = formattedResults.apply(NexmarkUtils.log(queryName)); } @@ -314,59 +467,17 @@ private void createSink(PCollection> results, long n formattedResults.apply(NexmarkUtils.devNull(queryName)); break; case PUBSUB: - // Publish results to pubsub. Don't bother consuming them, the published - // results will be discarded when we delete the 'sink' topic. - if (pubsubTopic == null) { - throw new RuntimeException("must supply a --pubsubTopic if using --sinkType=PUBSUB"); - } - String shortTopic; - if (uniqify) { - shortTopic = String.format("%s_%d_%s_sink", pubsubTopic, now, queryName); - } else { - shortTopic = String.format("%s_%s_sink", pubsubTopic, queryName); - } - String topic; - if (!manageResources) { - topic = getPubsub().reuseTopic(shortTopic); - } else { - topic = getPubsub().createTopic(shortTopic); - } - formattedResults.apply(pubsubResultSink(topic)); + sinkResultsToPubsub(formattedResults, now); break; case TEXT: - // Write results to text. Only works in batch mode. - if (options.isStreaming()) { - throw new RuntimeException("can only use --sinkType=TEXT with --streaming=false"); - } - if (outputPath == null) { - throw new RuntimeException("must supply an --outputPath if using --sinkType=TEXT"); - } - String fullFilename; - if (uniqify) { - fullFilename = String.format("%s/nexmark_%d_%s.txt", outputPath, now, queryName); - } else { - fullFilename = String.format("%s/nexmark_%s.txt", outputPath, queryName); - } - formattedResults.apply(TextIO.Write.to(fullFilename).named(queryName + ".Text")); + sinkToText(formattedResults, now); break; case AVRO: - NexmarkUtils.console(null, "WARNING: with --sinkType=AVRO, actual query results will be " - + "discarded.", outputPath); + NexmarkUtils.console( + "WARNING: with --sinkType=AVRO, actual query results will be discarded."); break; case BIGQUERY: - NexmarkUtils.console(null, "Writing events to BigQuery table %s", outputPath); - formattedResults - .apply(ParDo.of(new StringToTableRow())) - .apply(BigQueryIO.Write.named("WriteBigQuery(Events)") - .to(options.getProject() + ":nexmark.table_" - + new Random().nextInt(Integer.MAX_VALUE)) - .withSchema(new TableSchema().setFields( - new ArrayList() { - { - add(new TableFieldSchema().setName("event") - .setType("STRING")); - } - }))); + sinkToBigQuery(formattedResults, now); break; case COUNT_ONLY: // Short-circuited above. @@ -374,75 +485,38 @@ private void createSink(PCollection> results, long n } } - private static class StringToTableRow extends DoFn { - @Override - public void processElement(ProcessContext c) { - TableRow row = new TableRow(); - row.set("event", c.element()); - c.output(row); - } - } + // ================================================================================ + // Entry point + // ================================================================================ /** - * Sink all raw Events in {@code source} to {@code this.outputPath}. - *

- * 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. + * Calculate the distribution of the expected rate of results per minute (in event time, not + * wallclock time). */ - private void sinkToAvro(final PCollection source) { - if (options.isStreaming()) { - throw new RuntimeException("can only use Avro SinkType with --streaming=false"); - } - if (outputPath == null) { - throw new RuntimeException("Must supply an --outputPath if using --sinkType=AVRO"); + 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)); } - NexmarkUtils.console(null, "Writing events in Avro to %s", outputPath); - source.apply(AvroIO.Write.named("WriteAvro(Events)") - .to(outputPath + "/event") - .withSuffix(".avro") - .withSchema(Event.class)); - source.apply(NexmarkQuery.JUST_BIDS) - .apply(AvroIO.Write.named("WriteAvro(Bids)") - .to(outputPath + "/bid") - .withSuffix(".avro") - .withSchema(Bid.class)); - source.apply(NexmarkQuery.JUST_NEW_AUCTIONS) - .apply(AvroIO.Write.named("WriteAvro(Auctions)") - .to(outputPath + "/auction") - .withSuffix(".avro") - .withSchema(Auction.class)); - source.apply(NexmarkQuery.JUST_NEW_PERSONS) - .apply(AvroIO.Write.named("WriteAvro(People)") - .to(outputPath + "/person") - .withSuffix(".avro") - .withSchema(Person.class)); } /** - * Run {@code configuration} and return its performance (if using DataflowPipelineRunner - * and {@link #monitorJobs} is true). - * - * @throws IOException - * @throws InterruptedException + * Run {@code configuration} and return its performance if possible. */ @Nullable - public NexmarkPerf run(NexmarkConfiguration runConfiguration) - throws IOException, InterruptedException { + public NexmarkPerf run(NexmarkConfiguration runConfiguration) { // // Setup per-run state. // Preconditions.checkState(configuration == null); Preconditions.checkState(pubsub == null); - Preconditions.checkState(publisherResult == null); - Preconditions.checkState(mainResult == null); Preconditions.checkState(queryName == null); configuration = runConfiguration; @@ -450,10 +524,10 @@ public NexmarkPerf run(NexmarkConfiguration runConfiguration) List pathsToDelete = new ArrayList<>(); try { - NexmarkUtils.console(null, "running %s", configuration.toShortString()); + NexmarkUtils.console("Running %s", configuration.toShortString()); if (configuration.numEvents < 0) { - NexmarkUtils.console(null, "skipping since configuration is disabled"); + NexmarkUtils.console("skipping since configuration is disabled"); return null; } @@ -474,13 +548,21 @@ public NexmarkPerf run(NexmarkConfiguration runConfiguration) 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); + 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 (configuration.justModelResultRate) { if (model == null) { throw new RuntimeException(String.format("No model for %s", queryName)); @@ -499,30 +581,29 @@ public NexmarkPerf run(NexmarkConfiguration runConfiguration) // 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) { - if (monitorJobs && options.getRunner() != DataflowPipelineRunner.class) { - throw new RuntimeException("can only use --monitorJobs=true if also " - + "using --runner=DataflowPipelineRunner"); + if (options.getMonitorJobs() && !canMonitor()) { + throw new RuntimeException("Cannot use --monitorJobs=true with this runner"); } - // Optionally sink events in Avro format + // Optionally sink events in Avro format. + // (Query results are ignored). if (configuration.sinkType == NexmarkUtils.SinkType.AVRO) { - this.sinkToAvro(source); + sinkToAvro(source); } // Special hacks for Query 10 (big logger). if (configuration.query == 10) { String path = null; - if (outputPath != null) { - if (uniqify) { - path = String.format("%s/%d_logs", outputPath, now); + if (options.getOutputPath() != null && !options.getOutputPath().isEmpty()) { + if (options.getUniqify()) { + path = String.format("%s/%d_logs", options.getOutputPath(), now); } else { - path = String.format("%s/logs", outputPath); + path = String.format("%s/logs", options.getOutputPath()); } } ((Query10) query).setOutputPath(path); - ((Query10) query).setMaxNumWorkers( - Math.max(options.getMaxNumWorkers(), options.getNumWorkers())); - if (path != null && manageResources) { + ((Query10) query).setMaxNumWorkers(maxNumWorkers()); + if (path != null && options.getManageResources()) { pathsToDelete.add(path + "/**"); } } @@ -535,16 +616,19 @@ public NexmarkPerf run(NexmarkConfiguration runConfiguration) throw new RuntimeException(String.format("No model for %s", queryName)); } // We know all our streams have a finite number of elements. - results.setIsBoundedInternal(IsBounded.BOUNDED); + 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. DataflowAssert.that(results).satisfies(model.assertionFor()); } // Output results. - createSink(results, now); + sink(results, now); } + if (publisherResult != null) { + waitForPublisherPreload(); + } mainResult = p.run(); return monitor(query); } finally { @@ -553,540 +637,12 @@ public NexmarkPerf run(NexmarkConfiguration runConfiguration) // if (pubsub != null) { // Delete any subscriptions and topics we created. - pubsub.cleanup(); + pubsub.close(); pubsub = null; } - // TODO: Find a way to cleanup pathsToDelete robustly without depending on gsutil. configuration = null; - publisherResult = null; - mainResult = null; queryName = null; + // TODO: Cleanup pathsToDelete } } - - /** - * 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(null, "Query%d: only %d samples", model.configuration.query, n); - } else { - NexmarkUtils.console(null, "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)); - } - } - - /** - * Monitor the progress of the publisher job. Return when it has produced at - * least {@code configuration.preloadSeconds} worth of events. - * - * @throws IOException - * @throws InterruptedException - */ - private void waitForPublisherPreload() throws IOException, InterruptedException { - if (!monitorJobs) { - return; - } - if (options.getRunner() != DataflowPipelineRunner.class) { - return; - } - if (!(publisherResult instanceof DataflowPipelineJob)) { - return; - } - if (configuration.preloadSeconds <= 0) { - return; - } - - NexmarkUtils.console(null, "waiting for publisher to pre-load"); - - DataflowPipelineJob job = (DataflowPipelineJob) publisherResult; - - while (true) { - PipelineResult.State state = job.getState(); - long numEvents = getLong(job, publisherMonitor.getElementCounter()); - if (numEvents > 0) { - int waitSeconds = configuration.preloadSeconds; - NexmarkUtils.console(null, "%s publisher (Saw first event, waiting %ds for preload)", state, - configuration.preloadSeconds); - while (waitSeconds > 0) { - NexmarkUtils.console( - null, "%s publisher (%ds until preload is done)", state, waitSeconds); - Thread.sleep(TimeUnit.SECONDS.toMillis(Math.min(60, waitSeconds))); - waitSeconds -= 60; - } - return; - } - - switch (state) { - case UNKNOWN: - case STOPPED: - case RUNNING: - // Keep waiting. - break; - case DONE: - // All done. - NexmarkUtils.console(null, "publisher pipeline done"); - return; - case CANCELLED: - case FAILED: - case UPDATED: - // Something went wrong. - NexmarkUtils.console(null, "publisher pipeline failed", state, numEvents); - return; - } - - Thread.sleep(PERF_DELAY.getMillis()); - } - } - - /** - * Monitor the performance and progress of a running job. Return final performance if - * it was measured. - * - * @throws IOException - * @throws InterruptedException - */ - @Nullable - private NexmarkPerf monitor(NexmarkQuery query) throws IOException, InterruptedException { - if (!monitorJobs) { - return null; - } - if (options.getRunner() != DataflowPipelineRunner.class) { - return null; - } - if (!(mainResult instanceof DataflowPipelineJob)) { - return null; - } - // If we are not in debug mode, we have no event count or result count monitors. - boolean monitorsActive = configuration.debug; - - if (monitorsActive) { - NexmarkUtils.console(null, "waiting for main pipeline to 'finish'"); - } else { - NexmarkUtils.console(null, "debug=false, so job will not self-cancel"); - } - - - DataflowPipelineJob job = (DataflowPipelineJob) mainResult; - DataflowPipelineJob publisherJob = (DataflowPipelineJob) publisherResult; - List snapshots = new ArrayList<>(); - Instant start = Instant.now(); - Instant end = - options.getRunningTimeMinutes() != null - ? start.plus(Duration.standardMinutes(options.getRunningTimeMinutes())) - : new Instant(Long.MAX_VALUE); - if (options.getPreloadSeconds() != null) { - end = end.minus(Duration.standardSeconds(options.getPreloadSeconds())); - } - Instant lastActivity = null; - NexmarkPerf perf = null; - boolean waitingForShutdown = false; - boolean publisherCancelled = false; - List errors = new ArrayList<>(); - - while (true) { - Instant now = Instant.now(); - if (now.isAfter(end) && !waitingForShutdown) { - NexmarkUtils.console(null, "Reached end of test, cancelling job"); - job.cancel(); - if (publisherResult != null) { - publisherJob.cancel(); - publisherCancelled = true; - } - waitingForShutdown = true; - } - - PipelineResult.State state = job.getState(); - NexmarkUtils.console( - now, "%s %s%s", state, queryName, waitingForShutdown ? " (waiting for shutdown)" : ""); - - NexmarkPerf currPerf; - if (monitorsActive) { - currPerf = currentPerf(start, now, job, snapshots, query.eventMonitor, - query.resultMonitor); - } else { - currPerf = null; - } - - if (perf == null || perf.anyActivity(currPerf)) { - lastActivity = now; - } - - if (options.isStreaming() && !waitingForShutdown) { - Duration quietFor = new Duration(lastActivity, now); - if (query.getFatalCount() != null && getLong(job, query.getFatalCount()) > 0) { - NexmarkUtils.console(now, "job has fatal errors, cancelling."); - errors.add(String.format("Pipeline reported %s fatal errors", query.getFatalCount())); - waitingForShutdown = true; - } else if (monitorsActive && configuration.numEvents > 0 - && currPerf.numEvents == configuration.numEvents - && currPerf.numResults >= 0 && quietFor.isLongerThan(DONE_DELAY)) { - NexmarkUtils.console(now, "streaming query appears to have finished, cancelling job."); - waitingForShutdown = true; - } else if (quietFor.isLongerThan(STUCK_TERMINATE_DELAY)) { - NexmarkUtils.console( - now, "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( - now, "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, start)); - - if (waitingForShutdown) { - job.cancel(); - } - } - - 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 (lastActivity.equals(now)) { - NexmarkUtils.console(now, "new perf %s", perf); - } else { - NexmarkUtils.console(now, "no activity"); - } - - Thread.sleep(PERF_DELAY.getMillis()); - } - - perf.errors = errors; - perf.snapshots = snapshots; - NexmarkUtils.console(null, "final perf %s", perf); - - if (publisherResult != null) { - if (publisherCancelled) { - publisherJob.waitToFinish(5, TimeUnit.MINUTES, null); - } else { - publisherJob.cancel(); - } - } - - 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, Instant start) { - Instant now = Instant.now(); - 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.isAfter(start.plus( - Duration.standardSeconds(options.getWatermarkValidationDelaySeconds())))) { - 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(null, msg); - } - } - if (!foundWatermarks) { - NexmarkUtils.console(null, "No known watermarks in update: " + metrics); - if (now.isAfter(start.plus(Duration.standardMinutes(5)))) { - errors.add("No known watermarks found. Metrics were " + metrics); - } - } - } - } - } catch (IOException e) { - NexmarkUtils.console(null, "Warning: failed to get JobMetrics: " + e); - } - - return errors; - } - - /** - * Return the current performance given {@code eventMonitor} and {@code resultMonitor}. - */ - private NexmarkPerf currentPerf( - Instant start, Instant 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 - start.getMillis()) / 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.getMillis() - 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.getMillis() - start.getMillis()) / 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( - null, "%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(null, - "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( - null, "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( - Instant 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.getMillis()) > Duration.standardDays(10000).getMillis()) { - return -1; - } - return value; - } catch (AggregatorRetrievalException e) { - return -1; - } - } - } diff --git a/integration/java/src/main/java/com/google/cloud/dataflow/integration/nexmark/NexmarkUtils.java b/integration/java/src/main/java/com/google/cloud/dataflow/integration/nexmark/NexmarkUtils.java index 003fbabd5589..99a8722108d3 100644 --- a/integration/java/src/main/java/com/google/cloud/dataflow/integration/nexmark/NexmarkUtils.java +++ b/integration/java/src/main/java/com/google/cloud/dataflow/integration/nexmark/NexmarkUtils.java @@ -225,11 +225,8 @@ public static void error(String format, Object... args) { } /** Log message to console. For client side only. */ - public static void console(@Nullable Instant now, String format, Object... args) { - if (now == null) { - now = Instant.now(); - } - System.out.printf("%s %s\n", now, String.format(format, args)); + 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. */ diff --git a/integration/java/src/main/java/com/google/cloud/dataflow/integration/nexmark/Options.java b/integration/java/src/main/java/com/google/cloud/dataflow/integration/nexmark/Options.java index 82081d004f1d..c5495c57a6b7 100644 --- a/integration/java/src/main/java/com/google/cloud/dataflow/integration/nexmark/Options.java +++ b/integration/java/src/main/java/com/google/cloud/dataflow/integration/nexmark/Options.java @@ -18,12 +18,13 @@ import com.google.cloud.dataflow.sdk.options.Default; import com.google.cloud.dataflow.sdk.options.Description; +import com.google.cloud.dataflow.sdk.options.GcpOptions; import javax.annotation.Nullable; /** * Command line flags. */ -public interface Options { +public interface Options extends GcpOptions { @Description("Which suite to run. Default is to use command line arguments for one job.") @Default.Enum("DEFAULT") NexmarkSuite getSuite(); diff --git a/integration/java/src/main/java/com/google/cloud/dataflow/integration/nexmark/PubsubHelper.java b/integration/java/src/main/java/com/google/cloud/dataflow/integration/nexmark/PubsubHelper.java index ac342cd4c6e2..d7bda9781cb8 100644 --- a/integration/java/src/main/java/com/google/cloud/dataflow/integration/nexmark/PubsubHelper.java +++ b/integration/java/src/main/java/com/google/cloud/dataflow/integration/nexmark/PubsubHelper.java @@ -16,6 +16,7 @@ package com.google.cloud.dataflow.integration.nexmark; +import com.google.cloud.dataflow.sdk.io.PubsubClient; import com.google.cloud.dataflow.sdk.io.PubsubGrpcClient; import com.google.cloud.dataflow.sdk.options.GcpOptions; import java.io.IOException; @@ -26,11 +27,11 @@ /** * Helper for working with pubsub. */ -public class PubsubHelper { +public class PubsubHelper implements AutoCloseable { /** * Underlying pub/sub client. */ - private final PubsubGrpcClient pubsubClient; + private final PubsubClient pubsubClient; /** * Project id. @@ -47,7 +48,7 @@ public class PubsubHelper { */ private final List createdSubscriptions; - private PubsubHelper(PubsubGrpcClient pubsubClient, String project) { + private PubsubHelper(PubsubClient pubsubClient, String project) { this.pubsubClient = pubsubClient; this.project = project; createdTopics = new ArrayList<>(); @@ -57,8 +58,13 @@ private PubsubHelper(PubsubGrpcClient pubsubClient, String project) { /** * Create a helper. */ - public static PubsubHelper create(GcpOptions options, String project) throws IOException { - return new PubsubHelper(PubsubGrpcClient.newClient(null, null, options), project); + public static PubsubHelper create(GcpOptions options) { + try { + return new PubsubHelper(PubsubGrpcClient.newClient(null, null, options), + options.getProject()); + } catch (IOException e) { + throw new RuntimeException("Unable to create Pubsub client: ", e); + } } /** @@ -78,46 +84,50 @@ private String fullSubscription(String shortSubscription) { /** * Create a topic from short name. Delete it if it already exists. Ensure the topic will be * deleted on cleanup. Return full topic name. - * - * @throws InterruptedException */ - public String createTopic(String shortTopic) throws IOException, InterruptedException { + public String createTopic(String shortTopic) { String topic = fullTopic(shortTopic); - if (topicExists(topic)) { - NexmarkUtils.console(null, "attempting to cleanup topic %s", topic); - pubsubClient.deleteTopic(topic); + try { + if (topicExists(topic)) { + 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); } - NexmarkUtils.console(null, "create topic %s", topic); - pubsubClient.createTopic(topic); - createdTopics.add(topic); - return topic; } /** * Create a topic from short name if it does not already exist. The topic will not be * deleted on cleanup. Return full topic name. - * - * @throws InterruptedException */ - public String createOrReuseTopic(String shortTopic) throws IOException, InterruptedException { + public String createOrReuseTopic(String shortTopic) { String topic = fullTopic(shortTopic); - if (topicExists(topic)) { - NexmarkUtils.console(null, "topic %s already exists", topic); + try { + if (topicExists(topic)) { + 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); } - NexmarkUtils.console(null, "create topic %s", topic); - pubsubClient.createTopic(topic); - return topic; } /** * 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 String reuseTopic(String shortTopic) throws IOException { + public String reuseTopic(String shortTopic) { String topic = fullTopic(shortTopic); if (topicExists(shortTopic)) { - NexmarkUtils.console(null, "reusing existing topic %s", topic); + NexmarkUtils.console("reusing existing topic %s", topic); return topic; } throw new RuntimeException("topic '" + topic + "' does not already exist"); @@ -126,29 +136,34 @@ public String reuseTopic(String shortTopic) throws IOException { /** * Does topic corresponding to short name exist? */ - public boolean topicExists(String shortTopic) throws IOException { + public boolean topicExists(String shortTopic) { String topic = fullTopic(shortTopic); - Collection existingTopics = pubsubClient.listTopics("projects/" + project); - return existingTopics.contains(topic); + try { + Collection existingTopics = pubsubClient.listTopics("projects/" + 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. - * - * @throws InterruptedException */ - public String createSubscription(String shortTopic, String shortSubscription) - throws IOException, InterruptedException { + public String createSubscription(String shortTopic, String shortSubscription) { String topic = fullTopic(shortTopic); String subscription = fullSubscription(shortSubscription); - if (subscriptionExists(shortTopic, shortSubscription)) { - NexmarkUtils.console(null, "attempting to cleanup subscription %s", subscription); - pubsubClient.deleteSubscription(subscription); + try { + if (subscriptionExists(shortTopic, shortSubscription)) { + NexmarkUtils.console("attempting to cleanup subscription %s", subscription); + pubsubClient.deleteSubscription(subscription); + } + NexmarkUtils.console("create subscription %s", subscription); + pubsubClient.createSubscription(subscription, topic, 60); + createdSubscriptions.add(subscription); + } catch (IOException e) { + throw new RuntimeException("Unable to create Pubsub subscription " + subscription + ": ", e); } - NexmarkUtils.console(null, "create subscription %s", subscription); - pubsubClient.createSubscription(subscription, topic, 60); - createdSubscriptions.add(subscription); return subscription; } @@ -156,10 +171,10 @@ public String createSubscription(String shortTopic, String shortSubscription) * 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 String reuseSubscription(String shortTopic, String shortSubscription) throws IOException { + public String reuseSubscription(String shortTopic, String shortSubscription) { String subscription = fullSubscription(shortSubscription); if (subscriptionExists(shortTopic, shortSubscription)) { - NexmarkUtils.console(null, "reusing existing subscription %s", subscription); + NexmarkUtils.console("reusing existing subscription %s", subscription); return subscription; } throw new RuntimeException("subscription'" + subscription + "' does not already exist"); @@ -168,33 +183,37 @@ public String reuseSubscription(String shortTopic, String shortSubscription) thr /** * Does subscription corresponding to short name exist? */ - public boolean subscriptionExists(String shortTopic, String shortSubscription) - throws IOException { + public boolean subscriptionExists(String shortTopic, String shortSubscription) { String topic = fullTopic(shortTopic); String subscription = fullSubscription(shortSubscription); - Collection existingSubscriptions = - pubsubClient.listSubscriptions("projects/" + project, topic); - return existingSubscriptions.contains(subscription); + try { + Collection existingSubscriptions = + pubsubClient.listSubscriptions("projects/" + 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. */ - public void cleanup() { + @Override + public void close() { for (String subscription : createdSubscriptions) { try { - NexmarkUtils.console(null, "delete subscription %s", subscription); + NexmarkUtils.console("delete subscription %s", subscription); pubsubClient.deleteSubscription(subscription); } catch (IOException ex) { - NexmarkUtils.console(null, "could not delete subscription %s", subscription); + NexmarkUtils.console("could not delete subscription %s", subscription); } } for (String topic : createdTopics) { try { - NexmarkUtils.console(null, "delete topic %s", topic); + NexmarkUtils.console("delete topic %s", topic); pubsubClient.deleteTopic(topic); } catch (IOException ex) { - NexmarkUtils.console(null, "could not delete topic %s", topic); + NexmarkUtils.console("could not delete topic %s", topic); } } } diff --git a/integration/java/src/main/java/com/google/cloud/dataflow/integration/nexmark/WinningBids.java b/integration/java/src/main/java/com/google/cloud/dataflow/integration/nexmark/WinningBids.java index 10370d2578e7..da849815be8f 100644 --- a/integration/java/src/main/java/com/google/cloud/dataflow/integration/nexmark/WinningBids.java +++ b/integration/java/src/main/java/com/google/cloud/dataflow/integration/nexmark/WinningBids.java @@ -306,7 +306,7 @@ public WinningBids(String name, NexmarkConfiguration configuration) { // Adjust for number of in-flight auctions. longestDelayUs = longestDelayUs * configuration.numInFlightAuctions; long expectedAuctionDurationMs = (longestDelayUs + 999) / 1000; - NexmarkUtils.console(null, "Expected auction duration is %d ms", expectedAuctionDurationMs); + NexmarkUtils.console("Expected auction duration is %d ms", expectedAuctionDurationMs); auctionOrBidWindowFn = new AuctionOrBidWindowFn(expectedAuctionDurationMs); } From 0da3274a1bed2863d75d42c14216e45edb82f355 Mon Sep 17 00:00:00 2001 From: Mark Shields Date: Fri, 8 Apr 2016 16:41:20 -0700 Subject: [PATCH 12/14] Update copyright. Checkstyle fixes. --- integration/java/pom.xml | 2 +- .../nexmark/AbstractSimulator.java | 22 ++++---- .../dataflow/integration/nexmark/Auction.java | 22 ++++---- .../integration/nexmark/AuctionBid.java | 22 ++++---- .../integration/nexmark/AuctionCount.java | 22 ++++---- .../integration/nexmark/AuctionPrice.java | 22 ++++---- .../dataflow/integration/nexmark/Bid.java | 22 ++++---- .../integration/nexmark/BidsPerSession.java | 22 ++++---- .../nexmark/BoundedEventSource.java | 22 ++++---- .../integration/nexmark/CategoryPrice.java | 22 ++++---- .../dataflow/integration/nexmark/Done.java | 22 ++++---- .../dataflow/integration/nexmark/Event.java | 22 ++++---- .../integration/nexmark/Generator.java | 22 ++++---- .../integration/nexmark/GeneratorConfig.java | 22 ++++---- .../integration/nexmark/IdNameReserve.java | 22 ++++---- .../integration/nexmark/KnownSize.java | 22 ++++---- .../dataflow/integration/nexmark/Monitor.java | 22 ++++---- .../integration/nexmark/NameCityStateId.java | 22 ++++---- .../nexmark/NexmarkConfiguration.java | 22 ++++---- .../integration/nexmark/NexmarkDriver.java | 22 +++++++- .../nexmark/NexmarkGoogleDriver.java | 22 ++++---- .../nexmark/NexmarkGoogleRunner.java | 51 +++++-------------- .../integration/nexmark/NexmarkPerf.java | 22 ++++---- .../integration/nexmark/NexmarkQuery.java | 22 ++++---- .../nexmark/NexmarkQueryModel.java | 22 ++++---- .../integration/nexmark/NexmarkRunner.java | 24 +++++++-- .../integration/nexmark/NexmarkSuite.java | 22 ++++---- .../integration/nexmark/NexmarkUtils.java | 24 ++++----- .../dataflow/integration/nexmark/Options.java | 22 ++++---- .../dataflow/integration/nexmark/Person.java | 22 ++++---- .../integration/nexmark/PubsubHelper.java | 22 ++++---- .../dataflow/integration/nexmark/Query0.java | 22 ++++---- .../integration/nexmark/Query0Model.java | 22 ++++---- .../dataflow/integration/nexmark/Query1.java | 22 ++++---- .../dataflow/integration/nexmark/Query10.java | 22 ++++---- .../dataflow/integration/nexmark/Query11.java | 22 ++++---- .../dataflow/integration/nexmark/Query12.java | 25 +++++---- .../integration/nexmark/Query1Model.java | 22 ++++---- .../dataflow/integration/nexmark/Query2.java | 22 ++++---- .../integration/nexmark/Query2Model.java | 22 ++++---- .../dataflow/integration/nexmark/Query3.java | 22 ++++---- .../integration/nexmark/Query3Model.java | 22 ++++---- .../dataflow/integration/nexmark/Query4.java | 22 ++++---- .../integration/nexmark/Query4Model.java | 22 ++++---- .../dataflow/integration/nexmark/Query5.java | 22 ++++---- .../integration/nexmark/Query5Model.java | 22 ++++---- .../dataflow/integration/nexmark/Query6.java | 22 ++++---- .../integration/nexmark/Query6Model.java | 22 ++++---- .../dataflow/integration/nexmark/Query7.java | 22 ++++---- .../integration/nexmark/Query7Model.java | 22 ++++---- .../dataflow/integration/nexmark/Query8.java | 22 ++++---- .../integration/nexmark/Query8Model.java | 22 ++++---- .../dataflow/integration/nexmark/Query9.java | 22 ++++---- .../integration/nexmark/Query9Model.java | 22 ++++---- .../integration/nexmark/SellerPrice.java | 22 ++++---- .../nexmark/UnboundedEventSource.java | 22 ++++---- .../integration/nexmark/WinningBids.java | 22 ++++---- .../nexmark/WinningBidsSimulator.java | 22 ++++---- 58 files changed, 704 insertions(+), 588 deletions(-) diff --git a/integration/java/pom.xml b/integration/java/pom.xml index f71534ca80b4..656f8cccf957 100644 --- a/integration/java/pom.xml +++ b/integration/java/pom.xml @@ -63,7 +63,7 @@ - ../../checkstyle.xml + ../../sdks/java/checkstyle.xml true true true diff --git a/integration/java/src/main/java/com/google/cloud/dataflow/integration/nexmark/AbstractSimulator.java b/integration/java/src/main/java/com/google/cloud/dataflow/integration/nexmark/AbstractSimulator.java index 664aa952a02b..6b59886f7cf5 100644 --- a/integration/java/src/main/java/com/google/cloud/dataflow/integration/nexmark/AbstractSimulator.java +++ b/integration/java/src/main/java/com/google/cloud/dataflow/integration/nexmark/AbstractSimulator.java @@ -1,17 +1,19 @@ /* - * Copyright (C) 2015 Google Inc. + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at * - * Licensed under the Apache License, Version 2.0 (the "License"); you may not - * use this file except in compliance with the License. You may obtain a copy of - * the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 + * http://www.apache.org/licenses/LICENSE-2.0 * * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT - * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the - * License for the specific language governing permissions and limitations under - * the License. + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. */ package com.google.cloud.dataflow.integration.nexmark; diff --git a/integration/java/src/main/java/com/google/cloud/dataflow/integration/nexmark/Auction.java b/integration/java/src/main/java/com/google/cloud/dataflow/integration/nexmark/Auction.java index f4c97767fe44..49a2240d2d53 100644 --- a/integration/java/src/main/java/com/google/cloud/dataflow/integration/nexmark/Auction.java +++ b/integration/java/src/main/java/com/google/cloud/dataflow/integration/nexmark/Auction.java @@ -1,17 +1,19 @@ /* - * Copyright (C) 2015 Google Inc. + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at * - * Licensed under the Apache License, Version 2.0 (the "License"); you may not - * use this file except in compliance with the License. You may obtain a copy of - * the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 + * http://www.apache.org/licenses/LICENSE-2.0 * * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT - * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the - * License for the specific language governing permissions and limitations under - * the License. + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. */ package com.google.cloud.dataflow.integration.nexmark; diff --git a/integration/java/src/main/java/com/google/cloud/dataflow/integration/nexmark/AuctionBid.java b/integration/java/src/main/java/com/google/cloud/dataflow/integration/nexmark/AuctionBid.java index 805925c98278..ba4ac5da9d0c 100644 --- a/integration/java/src/main/java/com/google/cloud/dataflow/integration/nexmark/AuctionBid.java +++ b/integration/java/src/main/java/com/google/cloud/dataflow/integration/nexmark/AuctionBid.java @@ -1,17 +1,19 @@ /* - * Copyright (C) 2015 Google Inc. + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at * - * Licensed under the Apache License, Version 2.0 (the "License"); you may not - * use this file except in compliance with the License. You may obtain a copy of - * the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 + * http://www.apache.org/licenses/LICENSE-2.0 * * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT - * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the - * License for the specific language governing permissions and limitations under - * the License. + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. */ package com.google.cloud.dataflow.integration.nexmark; diff --git a/integration/java/src/main/java/com/google/cloud/dataflow/integration/nexmark/AuctionCount.java b/integration/java/src/main/java/com/google/cloud/dataflow/integration/nexmark/AuctionCount.java index 61984659e51c..c009864eb5f4 100644 --- a/integration/java/src/main/java/com/google/cloud/dataflow/integration/nexmark/AuctionCount.java +++ b/integration/java/src/main/java/com/google/cloud/dataflow/integration/nexmark/AuctionCount.java @@ -1,17 +1,19 @@ /* - * Copyright (C) 2015 Google Inc. + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at * - * Licensed under the Apache License, Version 2.0 (the "License"); you may not - * use this file except in compliance with the License. You may obtain a copy of - * the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 + * http://www.apache.org/licenses/LICENSE-2.0 * * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT - * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the - * License for the specific language governing permissions and limitations under - * the License. + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. */ package com.google.cloud.dataflow.integration.nexmark; diff --git a/integration/java/src/main/java/com/google/cloud/dataflow/integration/nexmark/AuctionPrice.java b/integration/java/src/main/java/com/google/cloud/dataflow/integration/nexmark/AuctionPrice.java index 5a5b73d309e6..db4b8944f591 100644 --- a/integration/java/src/main/java/com/google/cloud/dataflow/integration/nexmark/AuctionPrice.java +++ b/integration/java/src/main/java/com/google/cloud/dataflow/integration/nexmark/AuctionPrice.java @@ -1,17 +1,19 @@ /* - * Copyright (C) 2015 Google Inc. + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at * - * Licensed under the Apache License, Version 2.0 (the "License"); you may not - * use this file except in compliance with the License. You may obtain a copy of - * the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 + * http://www.apache.org/licenses/LICENSE-2.0 * * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT - * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the - * License for the specific language governing permissions and limitations under - * the License. + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. */ package com.google.cloud.dataflow.integration.nexmark; diff --git a/integration/java/src/main/java/com/google/cloud/dataflow/integration/nexmark/Bid.java b/integration/java/src/main/java/com/google/cloud/dataflow/integration/nexmark/Bid.java index 543f66506a76..6f47938fee92 100644 --- a/integration/java/src/main/java/com/google/cloud/dataflow/integration/nexmark/Bid.java +++ b/integration/java/src/main/java/com/google/cloud/dataflow/integration/nexmark/Bid.java @@ -1,17 +1,19 @@ /* - * Copyright (C) 2015 Google Inc. + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at * - * Licensed under the Apache License, Version 2.0 (the "License"); you may not - * use this file except in compliance with the License. You may obtain a copy of - * the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 + * http://www.apache.org/licenses/LICENSE-2.0 * * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT - * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the - * License for the specific language governing permissions and limitations under - * the License. + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. */ package com.google.cloud.dataflow.integration.nexmark; diff --git a/integration/java/src/main/java/com/google/cloud/dataflow/integration/nexmark/BidsPerSession.java b/integration/java/src/main/java/com/google/cloud/dataflow/integration/nexmark/BidsPerSession.java index fd9b42f521e3..82f0a557ebcf 100644 --- a/integration/java/src/main/java/com/google/cloud/dataflow/integration/nexmark/BidsPerSession.java +++ b/integration/java/src/main/java/com/google/cloud/dataflow/integration/nexmark/BidsPerSession.java @@ -1,17 +1,19 @@ /* - * Copyright (C) 2015 Google Inc. + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at * - * Licensed under the Apache License, Version 2.0 (the "License"); you may not - * use this file except in compliance with the License. You may obtain a copy of - * the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 + * http://www.apache.org/licenses/LICENSE-2.0 * * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT - * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the - * License for the specific language governing permissions and limitations under - * the License. + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. */ package com.google.cloud.dataflow.integration.nexmark; diff --git a/integration/java/src/main/java/com/google/cloud/dataflow/integration/nexmark/BoundedEventSource.java b/integration/java/src/main/java/com/google/cloud/dataflow/integration/nexmark/BoundedEventSource.java index 4d1688a99517..5141e00928f6 100644 --- a/integration/java/src/main/java/com/google/cloud/dataflow/integration/nexmark/BoundedEventSource.java +++ b/integration/java/src/main/java/com/google/cloud/dataflow/integration/nexmark/BoundedEventSource.java @@ -1,17 +1,19 @@ /* - * Copyright (C) 2015 Google Inc. + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at * - * Licensed under the Apache License, Version 2.0 (the "License"); you may not - * use this file except in compliance with the License. You may obtain a copy of - * the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 + * http://www.apache.org/licenses/LICENSE-2.0 * * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT - * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the - * License for the specific language governing permissions and limitations under - * the License. + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. */ package com.google.cloud.dataflow.integration.nexmark; diff --git a/integration/java/src/main/java/com/google/cloud/dataflow/integration/nexmark/CategoryPrice.java b/integration/java/src/main/java/com/google/cloud/dataflow/integration/nexmark/CategoryPrice.java index 4439b48ab11a..b0adfda7f9ca 100644 --- a/integration/java/src/main/java/com/google/cloud/dataflow/integration/nexmark/CategoryPrice.java +++ b/integration/java/src/main/java/com/google/cloud/dataflow/integration/nexmark/CategoryPrice.java @@ -1,17 +1,19 @@ /* - * Copyright (C) 2015 Google Inc. + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at * - * Licensed under the Apache License, Version 2.0 (the "License"); you may not - * use this file except in compliance with the License. You may obtain a copy of - * the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 + * http://www.apache.org/licenses/LICENSE-2.0 * * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT - * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the - * License for the specific language governing permissions and limitations under - * the License. + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. */ package com.google.cloud.dataflow.integration.nexmark; diff --git a/integration/java/src/main/java/com/google/cloud/dataflow/integration/nexmark/Done.java b/integration/java/src/main/java/com/google/cloud/dataflow/integration/nexmark/Done.java index af3e01485717..35aade3ba439 100644 --- a/integration/java/src/main/java/com/google/cloud/dataflow/integration/nexmark/Done.java +++ b/integration/java/src/main/java/com/google/cloud/dataflow/integration/nexmark/Done.java @@ -1,17 +1,19 @@ /* - * Copyright (C) 2015 Google Inc. + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at * - * Licensed under the Apache License, Version 2.0 (the "License"); you may not - * use this file except in compliance with the License. You may obtain a copy of - * the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 + * http://www.apache.org/licenses/LICENSE-2.0 * * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT - * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the - * License for the specific language governing permissions and limitations under - * the License. + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. */ package com.google.cloud.dataflow.integration.nexmark; diff --git a/integration/java/src/main/java/com/google/cloud/dataflow/integration/nexmark/Event.java b/integration/java/src/main/java/com/google/cloud/dataflow/integration/nexmark/Event.java index da569e49361d..46410bdbf8ae 100644 --- a/integration/java/src/main/java/com/google/cloud/dataflow/integration/nexmark/Event.java +++ b/integration/java/src/main/java/com/google/cloud/dataflow/integration/nexmark/Event.java @@ -1,17 +1,19 @@ /* - * Copyright (C) 2015 Google Inc. + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at * - * Licensed under the Apache License, Version 2.0 (the "License"); you may not - * use this file except in compliance with the License. You may obtain a copy of - * the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 + * http://www.apache.org/licenses/LICENSE-2.0 * * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT - * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the - * License for the specific language governing permissions and limitations under - * the License. + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. */ package com.google.cloud.dataflow.integration.nexmark; diff --git a/integration/java/src/main/java/com/google/cloud/dataflow/integration/nexmark/Generator.java b/integration/java/src/main/java/com/google/cloud/dataflow/integration/nexmark/Generator.java index 27152d29007b..1bb680dd0194 100644 --- a/integration/java/src/main/java/com/google/cloud/dataflow/integration/nexmark/Generator.java +++ b/integration/java/src/main/java/com/google/cloud/dataflow/integration/nexmark/Generator.java @@ -1,17 +1,19 @@ /* - * Copyright (C) 2015 Google Inc. + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at * - * Licensed under the Apache License, Version 2.0 (the "License"); you may not - * use this file except in compliance with the License. You may obtain a copy of - * the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 + * http://www.apache.org/licenses/LICENSE-2.0 * * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT - * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the - * License for the specific language governing permissions and limitations under - * the License. + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. */ package com.google.cloud.dataflow.integration.nexmark; diff --git a/integration/java/src/main/java/com/google/cloud/dataflow/integration/nexmark/GeneratorConfig.java b/integration/java/src/main/java/com/google/cloud/dataflow/integration/nexmark/GeneratorConfig.java index 4da3533a5964..33635b71d313 100644 --- a/integration/java/src/main/java/com/google/cloud/dataflow/integration/nexmark/GeneratorConfig.java +++ b/integration/java/src/main/java/com/google/cloud/dataflow/integration/nexmark/GeneratorConfig.java @@ -1,17 +1,19 @@ /* - * Copyright (C) 2015 Google Inc. + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at * - * Licensed under the Apache License, Version 2.0 (the "License"); you may not - * use this file except in compliance with the License. You may obtain a copy of - * the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 + * http://www.apache.org/licenses/LICENSE-2.0 * * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT - * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the - * License for the specific language governing permissions and limitations under - * the License. + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. */ package com.google.cloud.dataflow.integration.nexmark; diff --git a/integration/java/src/main/java/com/google/cloud/dataflow/integration/nexmark/IdNameReserve.java b/integration/java/src/main/java/com/google/cloud/dataflow/integration/nexmark/IdNameReserve.java index 70fcf019fd28..4049399c57f4 100644 --- a/integration/java/src/main/java/com/google/cloud/dataflow/integration/nexmark/IdNameReserve.java +++ b/integration/java/src/main/java/com/google/cloud/dataflow/integration/nexmark/IdNameReserve.java @@ -1,17 +1,19 @@ /* - * Copyright (C) 2015 Google Inc. + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at * - * Licensed under the Apache License, Version 2.0 (the "License"); you may not - * use this file except in compliance with the License. You may obtain a copy of - * the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 + * http://www.apache.org/licenses/LICENSE-2.0 * * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT - * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the - * License for the specific language governing permissions and limitations under - * the License. + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. */ package com.google.cloud.dataflow.integration.nexmark; diff --git a/integration/java/src/main/java/com/google/cloud/dataflow/integration/nexmark/KnownSize.java b/integration/java/src/main/java/com/google/cloud/dataflow/integration/nexmark/KnownSize.java index 1d32a8377c1a..81b9a879f5b4 100644 --- a/integration/java/src/main/java/com/google/cloud/dataflow/integration/nexmark/KnownSize.java +++ b/integration/java/src/main/java/com/google/cloud/dataflow/integration/nexmark/KnownSize.java @@ -1,17 +1,19 @@ /* - * Copyright (C) 2015 Google Inc. + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at * - * Licensed under the Apache License, Version 2.0 (the "License"); you may not - * use this file except in compliance with the License. You may obtain a copy of - * the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 + * http://www.apache.org/licenses/LICENSE-2.0 * * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT - * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the - * License for the specific language governing permissions and limitations under - * the License. + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. */ package com.google.cloud.dataflow.integration.nexmark; diff --git a/integration/java/src/main/java/com/google/cloud/dataflow/integration/nexmark/Monitor.java b/integration/java/src/main/java/com/google/cloud/dataflow/integration/nexmark/Monitor.java index b499e408ba91..d3a17ad40fdc 100644 --- a/integration/java/src/main/java/com/google/cloud/dataflow/integration/nexmark/Monitor.java +++ b/integration/java/src/main/java/com/google/cloud/dataflow/integration/nexmark/Monitor.java @@ -1,17 +1,19 @@ /* - * Copyright (C) 2015 Google Inc. + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at * - * Licensed under the Apache License, Version 2.0 (the "License"); you may not - * use this file except in compliance with the License. You may obtain a copy of - * the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 + * http://www.apache.org/licenses/LICENSE-2.0 * * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT - * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the - * License for the specific language governing permissions and limitations under - * the License. + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. */ package com.google.cloud.dataflow.integration.nexmark; diff --git a/integration/java/src/main/java/com/google/cloud/dataflow/integration/nexmark/NameCityStateId.java b/integration/java/src/main/java/com/google/cloud/dataflow/integration/nexmark/NameCityStateId.java index bf5a27d2d628..c16473c7ff85 100644 --- a/integration/java/src/main/java/com/google/cloud/dataflow/integration/nexmark/NameCityStateId.java +++ b/integration/java/src/main/java/com/google/cloud/dataflow/integration/nexmark/NameCityStateId.java @@ -1,17 +1,19 @@ /* - * Copyright (C) 2015 Google Inc. + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at * - * Licensed under the Apache License, Version 2.0 (the "License"); you may not - * use this file except in compliance with the License. You may obtain a copy of - * the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 + * http://www.apache.org/licenses/LICENSE-2.0 * * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT - * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the - * License for the specific language governing permissions and limitations under - * the License. + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. */ package com.google.cloud.dataflow.integration.nexmark; diff --git a/integration/java/src/main/java/com/google/cloud/dataflow/integration/nexmark/NexmarkConfiguration.java b/integration/java/src/main/java/com/google/cloud/dataflow/integration/nexmark/NexmarkConfiguration.java index 6152b43c38a6..0ab230f65fcd 100644 --- a/integration/java/src/main/java/com/google/cloud/dataflow/integration/nexmark/NexmarkConfiguration.java +++ b/integration/java/src/main/java/com/google/cloud/dataflow/integration/nexmark/NexmarkConfiguration.java @@ -1,17 +1,19 @@ /* - * Copyright (C) 2015 Google Inc. + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at * - * Licensed under the Apache License, Version 2.0 (the "License"); you may not - * use this file except in compliance with the License. You may obtain a copy of - * the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 + * http://www.apache.org/licenses/LICENSE-2.0 * * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT - * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the - * License for the specific language governing permissions and limitations under - * the License. + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. */ package com.google.cloud.dataflow.integration.nexmark; diff --git a/integration/java/src/main/java/com/google/cloud/dataflow/integration/nexmark/NexmarkDriver.java b/integration/java/src/main/java/com/google/cloud/dataflow/integration/nexmark/NexmarkDriver.java index 39ae84f3a329..b1c9d82b971c 100644 --- a/integration/java/src/main/java/com/google/cloud/dataflow/integration/nexmark/NexmarkDriver.java +++ b/integration/java/src/main/java/com/google/cloud/dataflow/integration/nexmark/NexmarkDriver.java @@ -1,3 +1,21 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package com.google.cloud.dataflow.integration.nexmark; import org.joda.time.Duration; @@ -33,12 +51,12 @@ * * http://datalab.cs.pdx.edu/niagaraST/NEXMark/ */ -public class NexmarkDriver { +public class NexmarkDriver { /** * Entry point. */ - public void runAll(O options, NexmarkRunner runner) { + public void runAll(OptionT options, NexmarkRunner runner) { Instant start = Instant.now(); Map baseline = loadBaseline(options.getBaselineFilename()); Map actual = new LinkedHashMap<>(); diff --git a/integration/java/src/main/java/com/google/cloud/dataflow/integration/nexmark/NexmarkGoogleDriver.java b/integration/java/src/main/java/com/google/cloud/dataflow/integration/nexmark/NexmarkGoogleDriver.java index 47baa9a231a3..322ed2eff84d 100644 --- a/integration/java/src/main/java/com/google/cloud/dataflow/integration/nexmark/NexmarkGoogleDriver.java +++ b/integration/java/src/main/java/com/google/cloud/dataflow/integration/nexmark/NexmarkGoogleDriver.java @@ -1,17 +1,19 @@ /* - * Copyright (C) 2015 Google Inc. + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at * - * Licensed under the Apache License, Version 2.0 (the "License"); you may not - * use this file except in compliance with the License. You may obtain a copy of - * the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 + * http://www.apache.org/licenses/LICENSE-2.0 * * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT - * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the - * License for the specific language governing permissions and limitations under - * the License. + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. */ package com.google.cloud.dataflow.integration.nexmark; diff --git a/integration/java/src/main/java/com/google/cloud/dataflow/integration/nexmark/NexmarkGoogleRunner.java b/integration/java/src/main/java/com/google/cloud/dataflow/integration/nexmark/NexmarkGoogleRunner.java index e54ff0d4f0dd..091d4e30d877 100644 --- a/integration/java/src/main/java/com/google/cloud/dataflow/integration/nexmark/NexmarkGoogleRunner.java +++ b/integration/java/src/main/java/com/google/cloud/dataflow/integration/nexmark/NexmarkGoogleRunner.java @@ -1,63 +1,42 @@ /* - * Copyright (C) 2015 Google Inc. + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at * - * Licensed under the Apache License, Version 2.0 (the "License"); you may not - * use this file except in compliance with the License. You may obtain a copy of - * the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 + * http://www.apache.org/licenses/LICENSE-2.0 * * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT - * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the - * License for the specific language governing permissions and limitations under - * the License. + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. */ package com.google.cloud.dataflow.integration.nexmark; -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.api.services.dataflow.model.JobMetrics; import com.google.api.services.dataflow.model.MetricUpdate; -import com.google.cloud.dataflow.integration.nexmark.NexmarkUtils.PubSubMode; -import com.google.cloud.dataflow.sdk.Pipeline; import com.google.cloud.dataflow.sdk.PipelineResult; -import com.google.cloud.dataflow.sdk.io.AvroIO; -import com.google.cloud.dataflow.sdk.io.BigQueryIO; -import com.google.cloud.dataflow.sdk.io.PubsubIO; -import com.google.cloud.dataflow.sdk.io.TextIO; -import com.google.cloud.dataflow.sdk.options.DataflowPipelineOptions; import com.google.cloud.dataflow.sdk.runners.AggregatorRetrievalException; import com.google.cloud.dataflow.sdk.runners.BlockingDataflowPipelineRunner; import com.google.cloud.dataflow.sdk.runners.DataflowPipelineJob; import com.google.cloud.dataflow.sdk.runners.DataflowPipelineRunner; -import com.google.cloud.dataflow.sdk.runners.DirectPipelineRunner; -import com.google.cloud.dataflow.sdk.testing.DataflowAssert; import com.google.cloud.dataflow.sdk.transforms.Aggregator; -import com.google.cloud.dataflow.sdk.transforms.DoFn; -import com.google.cloud.dataflow.sdk.transforms.ParDo; -import com.google.cloud.dataflow.sdk.values.PCollection; -import com.google.cloud.dataflow.sdk.values.PCollection.IsBounded; -import com.google.cloud.dataflow.sdk.values.TimestampedValue; import com.google.common.base.Preconditions; import com.google.common.collect.Iterables; -import com.google.common.collect.Lists; import org.joda.time.Duration; import org.joda.time.Instant; import java.io.IOException; -import java.io.OutputStream; -import java.io.PrintStream; import java.math.BigDecimal; import java.util.ArrayList; -import java.util.Arrays; import java.util.Collection; -import java.util.Collections; import java.util.List; -import java.util.Random; import java.util.concurrent.TimeUnit; import javax.annotation.Nullable; @@ -267,10 +246,8 @@ protected NexmarkPerf monitor(NexmarkQuery query) { long endMsSinceEpoch = -1; if (options.getRunningTimeMinutes() != null) { endMsSinceEpoch = startMsSinceEpoch - + Duration.standardMinutes(options.getRunningTimeMinutes()).getMillis(); - } - if (options.getPreloadSeconds() != null) { - endMsSinceEpoch -= Duration.standardSeconds(options.getPreloadSeconds()).getMillis(); + + Duration.standardMinutes(options.getRunningTimeMinutes()).getMillis() + - Duration.standardSeconds(configuration.preloadSeconds).getMillis(); } long lastActivityMsSinceEpoch = -1; NexmarkPerf perf = null; diff --git a/integration/java/src/main/java/com/google/cloud/dataflow/integration/nexmark/NexmarkPerf.java b/integration/java/src/main/java/com/google/cloud/dataflow/integration/nexmark/NexmarkPerf.java index 22cc18ebdd0c..4581a58cf85d 100644 --- a/integration/java/src/main/java/com/google/cloud/dataflow/integration/nexmark/NexmarkPerf.java +++ b/integration/java/src/main/java/com/google/cloud/dataflow/integration/nexmark/NexmarkPerf.java @@ -1,17 +1,19 @@ /* - * Copyright (C) 2015 Google Inc. + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at * - * Licensed under the Apache License, Version 2.0 (the "License"); you may not - * use this file except in compliance with the License. You may obtain a copy of - * the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 + * http://www.apache.org/licenses/LICENSE-2.0 * * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT - * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the - * License for the specific language governing permissions and limitations under - * the License. + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. */ package com.google.cloud.dataflow.integration.nexmark; diff --git a/integration/java/src/main/java/com/google/cloud/dataflow/integration/nexmark/NexmarkQuery.java b/integration/java/src/main/java/com/google/cloud/dataflow/integration/nexmark/NexmarkQuery.java index 8f77ab102165..44ba2d4ef8cc 100644 --- a/integration/java/src/main/java/com/google/cloud/dataflow/integration/nexmark/NexmarkQuery.java +++ b/integration/java/src/main/java/com/google/cloud/dataflow/integration/nexmark/NexmarkQuery.java @@ -1,17 +1,19 @@ /* - * Copyright (C) 2015 Google Inc. + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at * - * Licensed under the Apache License, Version 2.0 (the "License"); you may not - * use this file except in compliance with the License. You may obtain a copy of - * the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 + * http://www.apache.org/licenses/LICENSE-2.0 * * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT - * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the - * License for the specific language governing permissions and limitations under - * the License. + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. */ package com.google.cloud.dataflow.integration.nexmark; diff --git a/integration/java/src/main/java/com/google/cloud/dataflow/integration/nexmark/NexmarkQueryModel.java b/integration/java/src/main/java/com/google/cloud/dataflow/integration/nexmark/NexmarkQueryModel.java index 7484672bd887..c85e24fe5320 100644 --- a/integration/java/src/main/java/com/google/cloud/dataflow/integration/nexmark/NexmarkQueryModel.java +++ b/integration/java/src/main/java/com/google/cloud/dataflow/integration/nexmark/NexmarkQueryModel.java @@ -1,17 +1,19 @@ /* - * Copyright (C) 2015 Google Inc. + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at * - * Licensed under the Apache License, Version 2.0 (the "License"); you may not - * use this file except in compliance with the License. You may obtain a copy of - * the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 + * http://www.apache.org/licenses/LICENSE-2.0 * * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT - * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the - * License for the specific language governing permissions and limitations under - * the License. + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. */ package com.google.cloud.dataflow.integration.nexmark; diff --git a/integration/java/src/main/java/com/google/cloud/dataflow/integration/nexmark/NexmarkRunner.java b/integration/java/src/main/java/com/google/cloud/dataflow/integration/nexmark/NexmarkRunner.java index b616ac2e8b5f..6c344bf6dd1d 100644 --- a/integration/java/src/main/java/com/google/cloud/dataflow/integration/nexmark/NexmarkRunner.java +++ b/integration/java/src/main/java/com/google/cloud/dataflow/integration/nexmark/NexmarkRunner.java @@ -1,3 +1,21 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package com.google.cloud.dataflow.integration.nexmark; import com.google.api.services.bigquery.model.TableFieldSchema; @@ -28,11 +46,11 @@ /** * Run a single Nexmark query using a given configuration. */ -public abstract class NexmarkRunner { +public abstract class NexmarkRunner { /** * Options shared by all runs. */ - protected final O options; + protected final OptionT options; /** * Which configuration we are running. @@ -70,7 +88,7 @@ public abstract class NexmarkRunner { @Nullable protected String queryName; - public NexmarkRunner(O options) { + public NexmarkRunner(OptionT options) { this.options = options; } diff --git a/integration/java/src/main/java/com/google/cloud/dataflow/integration/nexmark/NexmarkSuite.java b/integration/java/src/main/java/com/google/cloud/dataflow/integration/nexmark/NexmarkSuite.java index 2c2bc8b8bc4f..7a76e0dc0c50 100644 --- a/integration/java/src/main/java/com/google/cloud/dataflow/integration/nexmark/NexmarkSuite.java +++ b/integration/java/src/main/java/com/google/cloud/dataflow/integration/nexmark/NexmarkSuite.java @@ -1,17 +1,19 @@ /* - * Copyright (C) 2015 Google Inc. + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at * - * Licensed under the Apache License, Version 2.0 (the "License"); you may not - * use this file except in compliance with the License. You may obtain a copy of - * the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 + * http://www.apache.org/licenses/LICENSE-2.0 * * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT - * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the - * License for the specific language governing permissions and limitations under - * the License. + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. */ package com.google.cloud.dataflow.integration.nexmark; diff --git a/integration/java/src/main/java/com/google/cloud/dataflow/integration/nexmark/NexmarkUtils.java b/integration/java/src/main/java/com/google/cloud/dataflow/integration/nexmark/NexmarkUtils.java index 99a8722108d3..54e29fbe9702 100644 --- a/integration/java/src/main/java/com/google/cloud/dataflow/integration/nexmark/NexmarkUtils.java +++ b/integration/java/src/main/java/com/google/cloud/dataflow/integration/nexmark/NexmarkUtils.java @@ -1,17 +1,19 @@ /* - * Copyright (C) 2015 Google Inc. + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at * - * Licensed under the Apache License, Version 2.0 (the "License"); you may not - * use this file except in compliance with the License. You may obtain a copy of - * the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 + * http://www.apache.org/licenses/LICENSE-2.0 * * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT - * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the - * License for the specific language governing permissions and limitations under - * the License. + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. */ package com.google.cloud.dataflow.integration.nexmark; @@ -61,8 +63,6 @@ import java.util.Iterator; import java.util.List; -import javax.annotation.Nullable; - /** * Odd's 'n Ends used throughout queries and driver. */ diff --git a/integration/java/src/main/java/com/google/cloud/dataflow/integration/nexmark/Options.java b/integration/java/src/main/java/com/google/cloud/dataflow/integration/nexmark/Options.java index c5495c57a6b7..8ddcbb22b08f 100644 --- a/integration/java/src/main/java/com/google/cloud/dataflow/integration/nexmark/Options.java +++ b/integration/java/src/main/java/com/google/cloud/dataflow/integration/nexmark/Options.java @@ -1,17 +1,19 @@ /* - * Copyright (C) 2016 Google Inc. + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at * - * Licensed under the Apache License, Version 2.0 (the "License"); you may not - * use this file except in compliance with the License. You may obtain a copy of - * the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 + * http://www.apache.org/licenses/LICENSE-2.0 * * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT - * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the - * License for the specific language governing permissions and limitations under - * the License. + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. */ package com.google.cloud.dataflow.integration.nexmark; diff --git a/integration/java/src/main/java/com/google/cloud/dataflow/integration/nexmark/Person.java b/integration/java/src/main/java/com/google/cloud/dataflow/integration/nexmark/Person.java index 1f53f7d45823..e042386129e5 100644 --- a/integration/java/src/main/java/com/google/cloud/dataflow/integration/nexmark/Person.java +++ b/integration/java/src/main/java/com/google/cloud/dataflow/integration/nexmark/Person.java @@ -1,17 +1,19 @@ /* - * Copyright (C) 2015 Google Inc. + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at * - * Licensed under the Apache License, Version 2.0 (the "License"); you may not - * use this file except in compliance with the License. You may obtain a copy of - * the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 + * http://www.apache.org/licenses/LICENSE-2.0 * * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT - * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the - * License for the specific language governing permissions and limitations under - * the License. + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. */ package com.google.cloud.dataflow.integration.nexmark; diff --git a/integration/java/src/main/java/com/google/cloud/dataflow/integration/nexmark/PubsubHelper.java b/integration/java/src/main/java/com/google/cloud/dataflow/integration/nexmark/PubsubHelper.java index d7bda9781cb8..0f642b9df85c 100644 --- a/integration/java/src/main/java/com/google/cloud/dataflow/integration/nexmark/PubsubHelper.java +++ b/integration/java/src/main/java/com/google/cloud/dataflow/integration/nexmark/PubsubHelper.java @@ -1,17 +1,19 @@ /* - * Copyright (C) 2015 Google Inc. + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at * - * Licensed under the Apache License, Version 2.0 (the "License"); you may not - * use this file except in compliance with the License. You may obtain a copy of - * the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 + * http://www.apache.org/licenses/LICENSE-2.0 * * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT - * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the - * License for the specific language governing permissions and limitations under - * the License. + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. */ package com.google.cloud.dataflow.integration.nexmark; diff --git a/integration/java/src/main/java/com/google/cloud/dataflow/integration/nexmark/Query0.java b/integration/java/src/main/java/com/google/cloud/dataflow/integration/nexmark/Query0.java index 90f97c9807ac..a660633a0b0c 100644 --- a/integration/java/src/main/java/com/google/cloud/dataflow/integration/nexmark/Query0.java +++ b/integration/java/src/main/java/com/google/cloud/dataflow/integration/nexmark/Query0.java @@ -1,17 +1,19 @@ /* - * Copyright (C) 2015 Google Inc. + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at * - * Licensed under the Apache License, Version 2.0 (the "License"); you may not - * use this file except in compliance with the License. You may obtain a copy of - * the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 + * http://www.apache.org/licenses/LICENSE-2.0 * * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT - * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the - * License for the specific language governing permissions and limitations under - * the License. + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. */ package com.google.cloud.dataflow.integration.nexmark; diff --git a/integration/java/src/main/java/com/google/cloud/dataflow/integration/nexmark/Query0Model.java b/integration/java/src/main/java/com/google/cloud/dataflow/integration/nexmark/Query0Model.java index 2afe6b09351c..b1360ca63b1c 100644 --- a/integration/java/src/main/java/com/google/cloud/dataflow/integration/nexmark/Query0Model.java +++ b/integration/java/src/main/java/com/google/cloud/dataflow/integration/nexmark/Query0Model.java @@ -1,17 +1,19 @@ /* - * Copyright (C) 2015 Google Inc. + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at * - * Licensed under the Apache License, Version 2.0 (the "License"); you may not - * use this file except in compliance with the License. You may obtain a copy of - * the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 + * http://www.apache.org/licenses/LICENSE-2.0 * * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT - * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the - * License for the specific language governing permissions and limitations under - * the License. + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. */ package com.google.cloud.dataflow.integration.nexmark; diff --git a/integration/java/src/main/java/com/google/cloud/dataflow/integration/nexmark/Query1.java b/integration/java/src/main/java/com/google/cloud/dataflow/integration/nexmark/Query1.java index b1a096a124d4..3b79867229df 100644 --- a/integration/java/src/main/java/com/google/cloud/dataflow/integration/nexmark/Query1.java +++ b/integration/java/src/main/java/com/google/cloud/dataflow/integration/nexmark/Query1.java @@ -1,17 +1,19 @@ /* - * Copyright (C) 2015 Google Inc. + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at * - * Licensed under the Apache License, Version 2.0 (the "License"); you may not - * use this file except in compliance with the License. You may obtain a copy of - * the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 + * http://www.apache.org/licenses/LICENSE-2.0 * * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT - * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the - * License for the specific language governing permissions and limitations under - * the License. + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. */ package com.google.cloud.dataflow.integration.nexmark; diff --git a/integration/java/src/main/java/com/google/cloud/dataflow/integration/nexmark/Query10.java b/integration/java/src/main/java/com/google/cloud/dataflow/integration/nexmark/Query10.java index eb098c2b1e18..9660f9bc743b 100644 --- a/integration/java/src/main/java/com/google/cloud/dataflow/integration/nexmark/Query10.java +++ b/integration/java/src/main/java/com/google/cloud/dataflow/integration/nexmark/Query10.java @@ -1,17 +1,19 @@ /* - * Copyright (C) 2015 Google Inc. + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at * - * Licensed under the Apache License, Version 2.0 (the "License"); you may not - * use this file except in compliance with the License. You may obtain a copy of - * the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 + * http://www.apache.org/licenses/LICENSE-2.0 * * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT - * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the - * License for the specific language governing permissions and limitations under - * the License. + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. */ package com.google.cloud.dataflow.integration.nexmark; diff --git a/integration/java/src/main/java/com/google/cloud/dataflow/integration/nexmark/Query11.java b/integration/java/src/main/java/com/google/cloud/dataflow/integration/nexmark/Query11.java index 06d4f6991bdf..e5052fe8e9c8 100644 --- a/integration/java/src/main/java/com/google/cloud/dataflow/integration/nexmark/Query11.java +++ b/integration/java/src/main/java/com/google/cloud/dataflow/integration/nexmark/Query11.java @@ -1,17 +1,19 @@ /* - * Copyright (C) 2015 Google Inc. + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at * - * Licensed under the Apache License, Version 2.0 (the "License"); you may not - * use this file except in compliance with the License. You may obtain a copy of - * the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 + * http://www.apache.org/licenses/LICENSE-2.0 * * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT - * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the - * License for the specific language governing permissions and limitations under - * the License. + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. */ package com.google.cloud.dataflow.integration.nexmark; diff --git a/integration/java/src/main/java/com/google/cloud/dataflow/integration/nexmark/Query12.java b/integration/java/src/main/java/com/google/cloud/dataflow/integration/nexmark/Query12.java index 6f1a6088ef55..56619db3e40a 100644 --- a/integration/java/src/main/java/com/google/cloud/dataflow/integration/nexmark/Query12.java +++ b/integration/java/src/main/java/com/google/cloud/dataflow/integration/nexmark/Query12.java @@ -1,17 +1,19 @@ /* - * Copyright (C) 2015 Google Inc. + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at * - * Licensed under the Apache License, Version 2.0 (the "License"); you may not - * use this file except in compliance with the License. You may obtain a copy of - * the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 + * http://www.apache.org/licenses/LICENSE-2.0 * * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT - * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the - * License for the specific language governing permissions and limitations under - * the License. + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. */ package com.google.cloud.dataflow.integration.nexmark; @@ -19,15 +21,12 @@ import com.google.cloud.dataflow.sdk.transforms.Count; import com.google.cloud.dataflow.sdk.transforms.DoFn; import com.google.cloud.dataflow.sdk.transforms.ParDo; -import com.google.cloud.dataflow.sdk.transforms.windowing.AfterPane; import com.google.cloud.dataflow.sdk.transforms.windowing.AfterProcessingTime; import com.google.cloud.dataflow.sdk.transforms.windowing.GlobalWindows; import com.google.cloud.dataflow.sdk.transforms.windowing.Repeatedly; -import com.google.cloud.dataflow.sdk.transforms.windowing.Sessions; import com.google.cloud.dataflow.sdk.transforms.windowing.Window; import com.google.cloud.dataflow.sdk.values.KV; import com.google.cloud.dataflow.sdk.values.PCollection; - import org.joda.time.Duration; /** diff --git a/integration/java/src/main/java/com/google/cloud/dataflow/integration/nexmark/Query1Model.java b/integration/java/src/main/java/com/google/cloud/dataflow/integration/nexmark/Query1Model.java index 47dc098de600..a7db42a8e26c 100644 --- a/integration/java/src/main/java/com/google/cloud/dataflow/integration/nexmark/Query1Model.java +++ b/integration/java/src/main/java/com/google/cloud/dataflow/integration/nexmark/Query1Model.java @@ -1,17 +1,19 @@ /* - * Copyright (C) 2015 Google Inc. + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at * - * Licensed under the Apache License, Version 2.0 (the "License"); you may not - * use this file except in compliance with the License. You may obtain a copy of - * the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 + * http://www.apache.org/licenses/LICENSE-2.0 * * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT - * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the - * License for the specific language governing permissions and limitations under - * the License. + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. */ package com.google.cloud.dataflow.integration.nexmark; diff --git a/integration/java/src/main/java/com/google/cloud/dataflow/integration/nexmark/Query2.java b/integration/java/src/main/java/com/google/cloud/dataflow/integration/nexmark/Query2.java index 47b521980863..cecddf0f3775 100644 --- a/integration/java/src/main/java/com/google/cloud/dataflow/integration/nexmark/Query2.java +++ b/integration/java/src/main/java/com/google/cloud/dataflow/integration/nexmark/Query2.java @@ -1,17 +1,19 @@ /* - * Copyright (C) 2015 Google Inc. + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at * - * Licensed under the Apache License, Version 2.0 (the "License"); you may not - * use this file except in compliance with the License. You may obtain a copy of - * the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 + * http://www.apache.org/licenses/LICENSE-2.0 * * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT - * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the - * License for the specific language governing permissions and limitations under - * the License. + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. */ package com.google.cloud.dataflow.integration.nexmark; diff --git a/integration/java/src/main/java/com/google/cloud/dataflow/integration/nexmark/Query2Model.java b/integration/java/src/main/java/com/google/cloud/dataflow/integration/nexmark/Query2Model.java index bf30b01d1516..bcff600e1efe 100644 --- a/integration/java/src/main/java/com/google/cloud/dataflow/integration/nexmark/Query2Model.java +++ b/integration/java/src/main/java/com/google/cloud/dataflow/integration/nexmark/Query2Model.java @@ -1,17 +1,19 @@ /* - * Copyright (C) 2015 Google Inc. + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at * - * Licensed under the Apache License, Version 2.0 (the "License"); you may not - * use this file except in compliance with the License. You may obtain a copy of - * the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 + * http://www.apache.org/licenses/LICENSE-2.0 * * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT - * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the - * License for the specific language governing permissions and limitations under - * the License. + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. */ package com.google.cloud.dataflow.integration.nexmark; diff --git a/integration/java/src/main/java/com/google/cloud/dataflow/integration/nexmark/Query3.java b/integration/java/src/main/java/com/google/cloud/dataflow/integration/nexmark/Query3.java index cbb2fa161582..75e117cb179c 100644 --- a/integration/java/src/main/java/com/google/cloud/dataflow/integration/nexmark/Query3.java +++ b/integration/java/src/main/java/com/google/cloud/dataflow/integration/nexmark/Query3.java @@ -1,17 +1,19 @@ /* - * Copyright (C) 2015 Google Inc. + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at * - * Licensed under the Apache License, Version 2.0 (the "License"); you may not - * use this file except in compliance with the License. You may obtain a copy of - * the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 + * http://www.apache.org/licenses/LICENSE-2.0 * * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT - * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the - * License for the specific language governing permissions and limitations under - * the License. + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. */ package com.google.cloud.dataflow.integration.nexmark; diff --git a/integration/java/src/main/java/com/google/cloud/dataflow/integration/nexmark/Query3Model.java b/integration/java/src/main/java/com/google/cloud/dataflow/integration/nexmark/Query3Model.java index 6ea152a1e79d..03a4f492f787 100644 --- a/integration/java/src/main/java/com/google/cloud/dataflow/integration/nexmark/Query3Model.java +++ b/integration/java/src/main/java/com/google/cloud/dataflow/integration/nexmark/Query3Model.java @@ -1,17 +1,19 @@ /* - * Copyright (C) 2015 Google Inc. + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at * - * Licensed under the Apache License, Version 2.0 (the "License"); you may not - * use this file except in compliance with the License. You may obtain a copy of - * the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 + * http://www.apache.org/licenses/LICENSE-2.0 * * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT - * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the - * License for the specific language governing permissions and limitations under - * the License. + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. */ package com.google.cloud.dataflow.integration.nexmark; diff --git a/integration/java/src/main/java/com/google/cloud/dataflow/integration/nexmark/Query4.java b/integration/java/src/main/java/com/google/cloud/dataflow/integration/nexmark/Query4.java index 332be51db079..a83221315a25 100644 --- a/integration/java/src/main/java/com/google/cloud/dataflow/integration/nexmark/Query4.java +++ b/integration/java/src/main/java/com/google/cloud/dataflow/integration/nexmark/Query4.java @@ -1,17 +1,19 @@ /* - * Copyright (C) 2015 Google Inc. + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at * - * Licensed under the Apache License, Version 2.0 (the "License"); you may not - * use this file except in compliance with the License. You may obtain a copy of - * the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 + * http://www.apache.org/licenses/LICENSE-2.0 * * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT - * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the - * License for the specific language governing permissions and limitations under - * the License. + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. */ package com.google.cloud.dataflow.integration.nexmark; diff --git a/integration/java/src/main/java/com/google/cloud/dataflow/integration/nexmark/Query4Model.java b/integration/java/src/main/java/com/google/cloud/dataflow/integration/nexmark/Query4Model.java index 3f3359ed2a8f..c378861baa62 100644 --- a/integration/java/src/main/java/com/google/cloud/dataflow/integration/nexmark/Query4Model.java +++ b/integration/java/src/main/java/com/google/cloud/dataflow/integration/nexmark/Query4Model.java @@ -1,17 +1,19 @@ /* - * Copyright (C) 2015 Google Inc. + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at * - * Licensed under the Apache License, Version 2.0 (the "License"); you may not - * use this file except in compliance with the License. You may obtain a copy of - * the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 + * http://www.apache.org/licenses/LICENSE-2.0 * * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT - * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the - * License for the specific language governing permissions and limitations under - * the License. + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. */ package com.google.cloud.dataflow.integration.nexmark; diff --git a/integration/java/src/main/java/com/google/cloud/dataflow/integration/nexmark/Query5.java b/integration/java/src/main/java/com/google/cloud/dataflow/integration/nexmark/Query5.java index f42221e5cc5e..30a1703f511a 100644 --- a/integration/java/src/main/java/com/google/cloud/dataflow/integration/nexmark/Query5.java +++ b/integration/java/src/main/java/com/google/cloud/dataflow/integration/nexmark/Query5.java @@ -1,17 +1,19 @@ /* - * Copyright (C) 2015 Google Inc. + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at * - * Licensed under the Apache License, Version 2.0 (the "License"); you may not - * use this file except in compliance with the License. You may obtain a copy of - * the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 + * http://www.apache.org/licenses/LICENSE-2.0 * * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT - * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the - * License for the specific language governing permissions and limitations under - * the License. + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. */ package com.google.cloud.dataflow.integration.nexmark; diff --git a/integration/java/src/main/java/com/google/cloud/dataflow/integration/nexmark/Query5Model.java b/integration/java/src/main/java/com/google/cloud/dataflow/integration/nexmark/Query5Model.java index 4265c31067aa..1f0bbd5124d1 100644 --- a/integration/java/src/main/java/com/google/cloud/dataflow/integration/nexmark/Query5Model.java +++ b/integration/java/src/main/java/com/google/cloud/dataflow/integration/nexmark/Query5Model.java @@ -1,17 +1,19 @@ /* - * Copyright (C) 2015 Google Inc. + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at * - * Licensed under the Apache License, Version 2.0 (the "License"); you may not - * use this file except in compliance with the License. You may obtain a copy of - * the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 + * http://www.apache.org/licenses/LICENSE-2.0 * * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT - * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the - * License for the specific language governing permissions and limitations under - * the License. + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. */ package com.google.cloud.dataflow.integration.nexmark; diff --git a/integration/java/src/main/java/com/google/cloud/dataflow/integration/nexmark/Query6.java b/integration/java/src/main/java/com/google/cloud/dataflow/integration/nexmark/Query6.java index 4706b346d362..adf4da207320 100644 --- a/integration/java/src/main/java/com/google/cloud/dataflow/integration/nexmark/Query6.java +++ b/integration/java/src/main/java/com/google/cloud/dataflow/integration/nexmark/Query6.java @@ -1,17 +1,19 @@ /* - * Copyright (C) 2015 Google Inc. + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at * - * Licensed under the Apache License, Version 2.0 (the "License"); you may not - * use this file except in compliance with the License. You may obtain a copy of - * the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 + * http://www.apache.org/licenses/LICENSE-2.0 * * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT - * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the - * License for the specific language governing permissions and limitations under - * the License. + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. */ package com.google.cloud.dataflow.integration.nexmark; diff --git a/integration/java/src/main/java/com/google/cloud/dataflow/integration/nexmark/Query6Model.java b/integration/java/src/main/java/com/google/cloud/dataflow/integration/nexmark/Query6Model.java index 98493fdfe42e..af7bbc12cdf6 100644 --- a/integration/java/src/main/java/com/google/cloud/dataflow/integration/nexmark/Query6Model.java +++ b/integration/java/src/main/java/com/google/cloud/dataflow/integration/nexmark/Query6Model.java @@ -1,17 +1,19 @@ /* - * Copyright (C) 2015 Google Inc. + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at * - * Licensed under the Apache License, Version 2.0 (the "License"); you may not - * use this file except in compliance with the License. You may obtain a copy of - * the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 + * http://www.apache.org/licenses/LICENSE-2.0 * * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT - * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the - * License for the specific language governing permissions and limitations under - * the License. + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. */ package com.google.cloud.dataflow.integration.nexmark; diff --git a/integration/java/src/main/java/com/google/cloud/dataflow/integration/nexmark/Query7.java b/integration/java/src/main/java/com/google/cloud/dataflow/integration/nexmark/Query7.java index 12b958bde998..146a43f88387 100644 --- a/integration/java/src/main/java/com/google/cloud/dataflow/integration/nexmark/Query7.java +++ b/integration/java/src/main/java/com/google/cloud/dataflow/integration/nexmark/Query7.java @@ -1,17 +1,19 @@ /* - * Copyright (C) 2015 Google Inc. + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at * - * Licensed under the Apache License, Version 2.0 (the "License"); you may not - * use this file except in compliance with the License. You may obtain a copy of - * the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 + * http://www.apache.org/licenses/LICENSE-2.0 * * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT - * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the - * License for the specific language governing permissions and limitations under - * the License. + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. */ package com.google.cloud.dataflow.integration.nexmark; diff --git a/integration/java/src/main/java/com/google/cloud/dataflow/integration/nexmark/Query7Model.java b/integration/java/src/main/java/com/google/cloud/dataflow/integration/nexmark/Query7Model.java index 7437e0130c70..354bb22c8b20 100644 --- a/integration/java/src/main/java/com/google/cloud/dataflow/integration/nexmark/Query7Model.java +++ b/integration/java/src/main/java/com/google/cloud/dataflow/integration/nexmark/Query7Model.java @@ -1,17 +1,19 @@ /* - * Copyright (C) 2015 Google Inc. + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at * - * Licensed under the Apache License, Version 2.0 (the "License"); you may not - * use this file except in compliance with the License. You may obtain a copy of - * the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 + * http://www.apache.org/licenses/LICENSE-2.0 * * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT - * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the - * License for the specific language governing permissions and limitations under - * the License. + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. */ package com.google.cloud.dataflow.integration.nexmark; diff --git a/integration/java/src/main/java/com/google/cloud/dataflow/integration/nexmark/Query8.java b/integration/java/src/main/java/com/google/cloud/dataflow/integration/nexmark/Query8.java index 7b5acb3ec6b1..1da726cab565 100644 --- a/integration/java/src/main/java/com/google/cloud/dataflow/integration/nexmark/Query8.java +++ b/integration/java/src/main/java/com/google/cloud/dataflow/integration/nexmark/Query8.java @@ -1,17 +1,19 @@ /* - * Copyright (C) 2015 Google Inc. + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at * - * Licensed under the Apache License, Version 2.0 (the "License"); you may not - * use this file except in compliance with the License. You may obtain a copy of - * the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 + * http://www.apache.org/licenses/LICENSE-2.0 * * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT - * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the - * License for the specific language governing permissions and limitations under - * the License. + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. */ package com.google.cloud.dataflow.integration.nexmark; diff --git a/integration/java/src/main/java/com/google/cloud/dataflow/integration/nexmark/Query8Model.java b/integration/java/src/main/java/com/google/cloud/dataflow/integration/nexmark/Query8Model.java index 9fd110fe1ddd..f96b1f9afb61 100644 --- a/integration/java/src/main/java/com/google/cloud/dataflow/integration/nexmark/Query8Model.java +++ b/integration/java/src/main/java/com/google/cloud/dataflow/integration/nexmark/Query8Model.java @@ -1,17 +1,19 @@ /* - * Copyright (C) 2015 Google Inc. + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at * - * Licensed under the Apache License, Version 2.0 (the "License"); you may not - * use this file except in compliance with the License. You may obtain a copy of - * the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 + * http://www.apache.org/licenses/LICENSE-2.0 * * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT - * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the - * License for the specific language governing permissions and limitations under - * the License. + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. */ package com.google.cloud.dataflow.integration.nexmark; diff --git a/integration/java/src/main/java/com/google/cloud/dataflow/integration/nexmark/Query9.java b/integration/java/src/main/java/com/google/cloud/dataflow/integration/nexmark/Query9.java index a497951c5a3c..9d846f71c070 100644 --- a/integration/java/src/main/java/com/google/cloud/dataflow/integration/nexmark/Query9.java +++ b/integration/java/src/main/java/com/google/cloud/dataflow/integration/nexmark/Query9.java @@ -1,17 +1,19 @@ /* - * Copyright (C) 2015 Google Inc. + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at * - * Licensed under the Apache License, Version 2.0 (the "License"); you may not - * use this file except in compliance with the License. You may obtain a copy of - * the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 + * http://www.apache.org/licenses/LICENSE-2.0 * * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT - * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the - * License for the specific language governing permissions and limitations under - * the License. + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. */ package com.google.cloud.dataflow.integration.nexmark; diff --git a/integration/java/src/main/java/com/google/cloud/dataflow/integration/nexmark/Query9Model.java b/integration/java/src/main/java/com/google/cloud/dataflow/integration/nexmark/Query9Model.java index c64f5a70cf8e..0fb678b0325d 100644 --- a/integration/java/src/main/java/com/google/cloud/dataflow/integration/nexmark/Query9Model.java +++ b/integration/java/src/main/java/com/google/cloud/dataflow/integration/nexmark/Query9Model.java @@ -1,17 +1,19 @@ /* - * Copyright (C) 2015 Google Inc. + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at * - * Licensed under the Apache License, Version 2.0 (the "License"); you may not - * use this file except in compliance with the License. You may obtain a copy of - * the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 + * http://www.apache.org/licenses/LICENSE-2.0 * * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT - * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the - * License for the specific language governing permissions and limitations under - * the License. + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. */ package com.google.cloud.dataflow.integration.nexmark; diff --git a/integration/java/src/main/java/com/google/cloud/dataflow/integration/nexmark/SellerPrice.java b/integration/java/src/main/java/com/google/cloud/dataflow/integration/nexmark/SellerPrice.java index f85e9eb694be..fb6c931b0ccd 100644 --- a/integration/java/src/main/java/com/google/cloud/dataflow/integration/nexmark/SellerPrice.java +++ b/integration/java/src/main/java/com/google/cloud/dataflow/integration/nexmark/SellerPrice.java @@ -1,17 +1,19 @@ /* - * Copyright (C) 2015 Google Inc. + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at * - * Licensed under the Apache License, Version 2.0 (the "License"); you may not - * use this file except in compliance with the License. You may obtain a copy of - * the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 + * http://www.apache.org/licenses/LICENSE-2.0 * * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT - * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the - * License for the specific language governing permissions and limitations under - * the License. + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. */ package com.google.cloud.dataflow.integration.nexmark; diff --git a/integration/java/src/main/java/com/google/cloud/dataflow/integration/nexmark/UnboundedEventSource.java b/integration/java/src/main/java/com/google/cloud/dataflow/integration/nexmark/UnboundedEventSource.java index be88f0712d83..57d7df866cf7 100644 --- a/integration/java/src/main/java/com/google/cloud/dataflow/integration/nexmark/UnboundedEventSource.java +++ b/integration/java/src/main/java/com/google/cloud/dataflow/integration/nexmark/UnboundedEventSource.java @@ -1,17 +1,19 @@ /* - * Copyright (C) 2015 Google Inc. + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at * - * Licensed under the Apache License, Version 2.0 (the "License"); you may not - * use this file except in compliance with the License. You may obtain a copy of - * the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 + * http://www.apache.org/licenses/LICENSE-2.0 * * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT - * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the - * License for the specific language governing permissions and limitations under - * the License. + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. */ package com.google.cloud.dataflow.integration.nexmark; diff --git a/integration/java/src/main/java/com/google/cloud/dataflow/integration/nexmark/WinningBids.java b/integration/java/src/main/java/com/google/cloud/dataflow/integration/nexmark/WinningBids.java index da849815be8f..b8939aa2b68a 100644 --- a/integration/java/src/main/java/com/google/cloud/dataflow/integration/nexmark/WinningBids.java +++ b/integration/java/src/main/java/com/google/cloud/dataflow/integration/nexmark/WinningBids.java @@ -1,17 +1,19 @@ /* - * Copyright (C) 2015 Google Inc. + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at * - * Licensed under the Apache License, Version 2.0 (the "License"); you may not - * use this file except in compliance with the License. You may obtain a copy of - * the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 + * http://www.apache.org/licenses/LICENSE-2.0 * * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT - * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the - * License for the specific language governing permissions and limitations under - * the License. + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. */ package com.google.cloud.dataflow.integration.nexmark; diff --git a/integration/java/src/main/java/com/google/cloud/dataflow/integration/nexmark/WinningBidsSimulator.java b/integration/java/src/main/java/com/google/cloud/dataflow/integration/nexmark/WinningBidsSimulator.java index 489c82ace374..803d84b5a334 100644 --- a/integration/java/src/main/java/com/google/cloud/dataflow/integration/nexmark/WinningBidsSimulator.java +++ b/integration/java/src/main/java/com/google/cloud/dataflow/integration/nexmark/WinningBidsSimulator.java @@ -1,17 +1,19 @@ /* - * Copyright (C) 2015 Google Inc. + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at * - * Licensed under the Apache License, Version 2.0 (the "License"); you may not - * use this file except in compliance with the License. You may obtain a copy of - * the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 + * http://www.apache.org/licenses/LICENSE-2.0 * * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT - * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the - * License for the specific language governing permissions and limitations under - * the License. + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. */ package com.google.cloud.dataflow.integration.nexmark; From c734feef1d368d994f723bf4980d53c577b3338c Mon Sep 17 00:00:00 2001 From: Mark Shields Date: Fri, 8 Apr 2016 16:43:05 -0700 Subject: [PATCH 13/14] wibble --- .../cloud/dataflow/integration/nexmark/NexmarkRunner.java | 7 +++++-- 1 file changed, 5 insertions(+), 2 deletions(-) diff --git a/integration/java/src/main/java/com/google/cloud/dataflow/integration/nexmark/NexmarkRunner.java b/integration/java/src/main/java/com/google/cloud/dataflow/integration/nexmark/NexmarkRunner.java index 6c344bf6dd1d..5def022c4ddf 100644 --- a/integration/java/src/main/java/com/google/cloud/dataflow/integration/nexmark/NexmarkRunner.java +++ b/integration/java/src/main/java/com/google/cloud/dataflow/integration/nexmark/NexmarkRunner.java @@ -131,8 +131,11 @@ private PubsubHelper getPubsub() { */ protected abstract boolean canMonitor(); - protected interface PipelineBuilder { - void build(O publishOnlyOptions); + /** + * Build and run a pipeline using specified options. + */ + protected interface PipelineBuilder { + void build(OptionT publishOnlyOptions); } /** From 01ace068e938b7d9f281f6fa1858b54990223a3d Mon Sep 17 00:00:00 2001 From: Mark Shields Date: Fri, 8 Apr 2016 20:50:58 -0700 Subject: [PATCH 14/14] Misc bugs. Make netty work. --- integration/java/pom.xml | 16 +++++++ .../nexmark/NexmarkGoogleRunner.java | 11 ++--- .../integration/nexmark/NexmarkRunner.java | 46 +++++++++++-------- .../integration/nexmark/PubsubHelper.java | 2 +- 4 files changed, 48 insertions(+), 27 deletions(-) diff --git a/integration/java/pom.xml b/integration/java/pom.xml index 656f8cccf957..fcc9b5d35abd 100644 --- a/integration/java/pom.xml +++ b/integration/java/pom.xml @@ -36,6 +36,15 @@ jar + + + + kr.motd.maven + os-maven-plugin + 1.4.0.Final + + + maven-compiler-plugin @@ -232,5 +241,12 @@ junit ${junit.version} + + + io.netty + netty-tcnative-boringssl-static + 1.1.33.Fork13 + ${os.detected.classifier} + diff --git a/integration/java/src/main/java/com/google/cloud/dataflow/integration/nexmark/NexmarkGoogleRunner.java b/integration/java/src/main/java/com/google/cloud/dataflow/integration/nexmark/NexmarkGoogleRunner.java index 091d4e30d877..79883e0773ff 100644 --- a/integration/java/src/main/java/com/google/cloud/dataflow/integration/nexmark/NexmarkGoogleRunner.java +++ b/integration/java/src/main/java/com/google/cloud/dataflow/integration/nexmark/NexmarkGoogleRunner.java @@ -234,7 +234,7 @@ protected NexmarkPerf monitor(NexmarkQuery query) { boolean monitorsActive = configuration.debug; if (monitorsActive) { - NexmarkUtils.console("waiting for main pipeline to 'finish'"); + NexmarkUtils.console("Waiting for main pipeline to 'finish'"); } else { NexmarkUtils.console("--debug=false, so job will not self-cancel"); } @@ -257,7 +257,7 @@ protected NexmarkPerf monitor(NexmarkQuery query) { while (true) { long now = System.currentTimeMillis(); - if (now > endMsSinceEpoch && !waitingForShutdown) { + if (endMsSinceEpoch >= 0 && now > endMsSinceEpoch && !waitingForShutdown) { NexmarkUtils.console("Reached end of test, cancelling job"); try { job.cancel(); @@ -371,15 +371,14 @@ protected NexmarkPerf monitor(NexmarkQuery query) { perf.errors = errors; perf.snapshots = snapshots; - NexmarkUtils.console("final perf %s", perf); if (publisherResult != null) { + NexmarkUtils.console("Shutting down publisher pipeline."); try { - if (publisherCancelled) { - publisherJob.waitToFinish(5, TimeUnit.MINUTES, null); - } else { + 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) { diff --git a/integration/java/src/main/java/com/google/cloud/dataflow/integration/nexmark/NexmarkRunner.java b/integration/java/src/main/java/com/google/cloud/dataflow/integration/nexmark/NexmarkRunner.java index 5def022c4ddf..3b2a9d5331c9 100644 --- a/integration/java/src/main/java/com/google/cloud/dataflow/integration/nexmark/NexmarkRunner.java +++ b/integration/java/src/main/java/com/google/cloud/dataflow/integration/nexmark/NexmarkRunner.java @@ -164,11 +164,11 @@ protected interface PipelineBuilder { */ private PCollection sourceFromSynthetic(Pipeline p) { if (isStreaming()) { - NexmarkUtils.console("Generating %d events in batch mode", configuration.numEvents); - return p.apply(NexmarkUtils.batchEventsSource(queryName, configuration)); - } else { 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)); } } @@ -198,7 +198,7 @@ private PCollection sourceEventsFromPubsub(Pipeline p, long now) { } NexmarkUtils.console("Reading events from Pubsub %s", subscription); PubsubIO.Read.Bound io = - PubsubIO.Read.named(queryName + ".ReadPubsubEvents(" + subscription + ")") + PubsubIO.Read.named(queryName + ".ReadPubsubEvents(" + subscription.replace('/', '.') + ")") .subscription(subscription) .idLabel(NexmarkUtils.PUBSUB_ID) .withCoder(Event.CODER); @@ -218,7 +218,7 @@ private PCollection sourceFromAvro(Pipeline p) { } NexmarkUtils.console("Reading events from Avro files at %s", filename); return p - .apply(AvroIO.Read.named(queryName + ".ReadAvroEvents(" + filename + ")") + .apply(AvroIO.Read.named(queryName + ".ReadAvroEvents(" + filename.replace('/', '.') + ")") .from(filename + "*.avro") .withSchema(Event.class)) .apply(NexmarkQuery.EVENT_TIMESTAMP_FROM_DATA); @@ -253,7 +253,7 @@ private void sinkEventsToPubsub(PCollection events, long now) { } NexmarkUtils.console("Writing events to Pubsub %s", topic); PubsubIO.Write.Bound io = - PubsubIO.Write.named(queryName + ".WritePubsubEvents(" + topic + ")") + PubsubIO.Write.named(queryName + ".WritePubsubEvents(" + topic.replace('/', '.') + ")") .topic(topic) .idLabel(NexmarkUtils.PUBSUB_ID) .withCoder(Event.CODER); @@ -285,7 +285,7 @@ private void sinkResultsToPubsub(PCollection formattedResults, long now) } NexmarkUtils.console("Writing results to Pubsub %s", topic); PubsubIO.Write.Bound io = - PubsubIO.Write.named(queryName + ".WritePubsubResults(" + topic + ")") + PubsubIO.Write.named(queryName + ".WritePubsubResults(" + topic.replace('/', '.') + ")") .topic(topic) .idLabel(NexmarkUtils.PUBSUB_ID); if (!configuration.usePubsubPublishTime) { @@ -312,25 +312,29 @@ private void sinkToAvro(PCollection source) { throw new RuntimeException("Missing --outputPath"); } NexmarkUtils.console("Writing events to Avro files at %s", filename); - source.apply(AvroIO.Write.named(queryName + ".WriteAvroEvents(" + filename + ")") + source.apply(AvroIO.Write.named(queryName + ".WriteAvroEvents(" + filename.replace('/', '.') + + ")") .to(filename + "/event") .withSuffix(".avro") .withSchema(Event.class)); source.apply(NexmarkQuery.JUST_BIDS) - .apply(AvroIO.Write.named(queryName + ".WriteAvroBids(" + filename + ")") - .to(filename + "/bid") - .withSuffix(".avro") - .withSchema(Bid.class)); + .apply( + AvroIO.Write.named(queryName + ".WriteAvroBids(" + filename.replace('/', '.') + ")") + .to(filename + "/bid") + .withSuffix(".avro") + .withSchema(Bid.class)); source.apply(NexmarkQuery.JUST_NEW_AUCTIONS) - .apply(AvroIO.Write.named(queryName + ".WriteAvroAuctions(" + filename + ")") + .apply(AvroIO.Write.named( + queryName + ".WriteAvroAuctions(" + filename.replace('/', '.') + ")") .to(filename + "/auction") .withSuffix(".avro") .withSchema(Auction.class)); source.apply(NexmarkQuery.JUST_NEW_PERSONS) - .apply(AvroIO.Write.named(queryName + ".WriteAvroPeople(" + filename + ")") - .to(filename + "/person") - .withSuffix(".avro") - .withSchema(Person.class)); + .apply( + AvroIO.Write.named(queryName + ".WriteAvroPeople(" + filename.replace('/', '.') + ")") + .to(filename + "/person") + .withSuffix(".avro") + .withSchema(Person.class)); } private static class StringToTableRow extends DoFn { @@ -357,8 +361,9 @@ private void sinkToText(PCollection formattedResults, long now) { fullFilename = String.format("%s/nexmark_%s.txt", filename, queryName); } NexmarkUtils.console("Writing results to text files at %s", fullFilename); - formattedResults.apply(TextIO.Write.named(queryName + ".WriteTextResults(" + fullFilename + ")") - .to(fullFilename)); + formattedResults.apply( + TextIO.Write.named(queryName + ".WriteTextResults(" + fullFilename.replace('/', '.') + ")") + .to(fullFilename)); } /** @@ -377,7 +382,8 @@ private void sinkToBigQuery(PCollection formattedResults, long now) { .setType("STRING"))); NexmarkUtils.console("Writing results to BigQuery table %s", tableName); BigQueryIO.Write.Bound io = - BigQueryIO.Write.named(queryName + ".WriteBigQueryResults(" + tableName + ")") + BigQueryIO.Write.named( + queryName + ".WriteBigQueryResults(" + tableName.replace('/', '.') + ")") .to(tableName) .withSchema(schema); formattedResults diff --git a/integration/java/src/main/java/com/google/cloud/dataflow/integration/nexmark/PubsubHelper.java b/integration/java/src/main/java/com/google/cloud/dataflow/integration/nexmark/PubsubHelper.java index 0f642b9df85c..ba60bc9531d8 100644 --- a/integration/java/src/main/java/com/google/cloud/dataflow/integration/nexmark/PubsubHelper.java +++ b/integration/java/src/main/java/com/google/cloud/dataflow/integration/nexmark/PubsubHelper.java @@ -161,7 +161,7 @@ public String createSubscription(String shortTopic, String shortSubscription) { pubsubClient.deleteSubscription(subscription); } NexmarkUtils.console("create subscription %s", subscription); - pubsubClient.createSubscription(subscription, topic, 60); + pubsubClient.createSubscription(topic, subscription, 60); createdSubscriptions.add(subscription); } catch (IOException e) { throw new RuntimeException("Unable to create Pubsub subscription " + subscription + ": ", e);