From d396c5e5502696ed83065be5f787288c06393859 Mon Sep 17 00:00:00 2001 From: Mark Shields Date: Mon, 28 Mar 2016 13:30:37 -0700 Subject: [PATCH 01/14] Initial import --- sdks/java/core/pom.xml | 14 +++++++++++ .../dataflow/sdk/io/UnboundedSource.java | 25 ++++++++++++++----- 2 files changed, 33 insertions(+), 6 deletions(-) diff --git a/sdks/java/core/pom.xml b/sdks/java/core/pom.xml index 6595a214e4f8..41cf64a4f194 100644 --- a/sdks/java/core/pom.xml +++ b/sdks/java/core/pom.xml @@ -389,6 +389,20 @@ 0.12.0 + + com.google.api.grpc + grpc-pubsub-v1 + 0.0.1 + + + + 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/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: + *

*/ 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 4d4ecb97219e7ed124740ec999667698c568aac7 Mon Sep 17 00:00:00 2001 From: Mark Shields Date: Mon, 28 Mar 2016 13:37:23 -0700 Subject: [PATCH 02/14] Initial import II --- .../dataflow/sdk/io/BucketingFunction.java | 133 ++ .../cloud/dataflow/sdk/io/MovingFunction.java | 149 +++ .../dataflow/sdk/io/PubsubGrpcClient.java | 151 +++ .../dataflow/sdk/io/PubsubUnboundedSink.java | 359 ++++++ .../sdk/io/PubsubUnboundedSource.java | 1141 +++++++++++++++++ .../cloud/dataflow/sdk/io/SimpleFunction.java | 46 + .../sdk/io/BucketingFunctionTest.java | 86 ++ .../dataflow/sdk/io/MovingFunctionTest.java | 97 ++ 8 files changed, 2162 insertions(+) create mode 100644 sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/io/BucketingFunction.java create mode 100644 sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/io/MovingFunction.java create mode 100644 sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/io/PubsubGrpcClient.java create mode 100644 sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/io/PubsubUnboundedSink.java 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/io/SimpleFunction.java create mode 100644 sdks/java/core/src/test/java/com/google/cloud/dataflow/sdk/io/BucketingFunctionTest.java create mode 100644 sdks/java/core/src/test/java/com/google/cloud/dataflow/sdk/io/MovingFunctionTest.java diff --git a/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/io/BucketingFunction.java b/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/io/BucketingFunction.java new file mode 100644 index 000000000000..eae0069bac13 --- /dev/null +++ b/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/io/BucketingFunction.java @@ -0,0 +1,133 @@ +package com.google.cloud.dataflow.sdk.io; + +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.zero(); + } + + public void add(BucketingFunction outer, long value) { + combinedValue = outer.function.f(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 SimpleFunction function; + + /** + * Active buckets. + */ + private final Map buckets; + + public BucketingFunction( + long bucketWidthMs, + int numSignificantBuckets, + int numSignificantSamples, + SimpleFunction 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.zero(); + for (Bucket bucket : buckets.values()) { + result = function.f(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/io/MovingFunction.java b/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/io/MovingFunction.java new file mode 100644 index 000000000000..aadc5aaa7f46 --- /dev/null +++ b/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/io/MovingFunction.java @@ -0,0 +1,149 @@ +/* + * 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.sdk.io; + +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}. + */ +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 SimpleFunction 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, + SimpleFunction 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.zero()); + 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.zero(); + numSamples[currentIndex] = 0; + newBuckets--; + currentMsSinceEpoch += sampleUpdateMs; + } + } + + /** + * Add {@code value} at {@code nowMsSinceEpoch}. + */ + public void add(long nowMsSinceEpoch, long value) { + flush(nowMsSinceEpoch); + buckets[currentIndex] = function.f(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.zero(); + for (int i = 0; i < buckets.length; i++) { + result = function.f(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/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..753289218481 --- /dev/null +++ b/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/io/PubsubGrpcClient.java @@ -0,0 +1,151 @@ +package com.google.cloud.dataflow.sdk.io; + +import com.google.auth.oauth2.GoogleCredentials; +import com.google.cloud.dataflow.sdk.options.GcpOptions; +import com.google.common.base.Preconditions; +import com.google.protobuf.Empty; +import com.google.pubsub.v1.AcknowledgeRequest; +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.PullRequest; +import com.google.pubsub.v1.PullResponse; +import com.google.pubsub.v1.SubscriberGrpc; +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 io.grpc.stub.AbstractStub; +import java.io.IOException; +import java.security.GeneralSecurityException; +import java.util.Arrays; +import java.util.List; +import java.util.concurrent.Executors; +import java.util.concurrent.TimeUnit; +import javax.annotation.Nullable; + +/** + * A helper class for talking to pub/sub via grpc. + */ +public class PubsubGrpcClient implements AutoCloseable { + private static final String PUBSUB_ADDRESS = "pubsub.googleapis.com"; + private static final int PUBSUB_PORT = 443; + private static final List PUBSUB_SCOPES = + Arrays.asList("https://www.googleapis.com/auth/pubsub"); + + /** + * Timeout for grpc calls (in s). + */ + public static final int TIMEOUT_S = 15; + + /** + * Underlying netty channel, or null if closed. + */ + @Nullable + private ManagedChannel publisherChannel; + + /** + * Credentials determined from options and environment. + */ + private GoogleCredentials credentials; + + /** + * Cached stubs, or null if not cached. + */ + @Nullable + private PublisherGrpc.PublisherBlockingStub cachedPublisherStub; + private SubscriberGrpc.SubscriberBlockingStub cachedSubscriberStub; + + private PubsubGrpcClient(ManagedChannel publisherChannel, GoogleCredentials credentials) { + this.publisherChannel = publisherChannel; + this.credentials = credentials; + } + + /** + * Construct a new pub/sub grpc client. It should be closed via {@link #close} in order + * to ensure tidy cleanup of underlying netty resources. + */ + public static PubsubGrpcClient newClient(GcpOptions options) + throws IOException, GeneralSecurityException { + 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(channel, credentials); + } + + /** + * Gracefully close the underlying netty channel. + */ + @Override + public void close() { + 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); + } + + /** + * The following are pass-through. + */ + public PublishResponse publish(PublishRequest request) throws IOException { + return publisherStub().publish(request); + } + + public Empty acknowledge(AcknowledgeRequest request) throws IOException { + return subscriberStub().acknowledge(request); + } + + public Empty modifyAckDeadline(ModifyAckDeadlineRequest request) throws IOException { + return subscriberStub().modifyAckDeadline(request); + } + + public PullResponse pull(PullRequest request) throws IOException { + return subscriberStub().pull(request); + } +} 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..a2f970aa97a8 --- /dev/null +++ b/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/io/PubsubUnboundedSink.java @@ -0,0 +1,359 @@ +/* + * 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.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 com.google.common.hash.Hashing; +import com.google.protobuf.ByteString; +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 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 java.util.concurrent.TimeUnit; +import javax.annotation.Nullable; +import org.joda.time.Duration; +import org.joda.time.Instant; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * A PTransform which streams messages to pub/sub. + *

+ */ +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; + + // ================================================================================ + // Message + // ================================================================================ + + /** + * A message to be sent to pub/sub. + */ + private static class Message { + public static final Coder CODER = new AtomicCoder() { + @Override + public void encode( + Message 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 Message 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 Message(elementBytes, timestampMsSinceEpoch); + } + }; + + /** + * Underlying (encoded) element. + */ + private final byte[] elementBytes; + + /** + * Timestamp for element (ms since epoch). + */ + private final long timestampMsSinceEpoch; + + public Message(byte[] elementBytes, long timestampMsSinceEpoch) { + this.elementBytes = elementBytes; + this.timestampMsSinceEpoch = timestampMsSinceEpoch; + } + + /** + * Return message for element. + */ + public static Message forElement( + PubsubUnboundedSink outer, T element, Instant timestamp) throws CoderException { + byte[] elementBytes = CoderUtils.encodeToByteArray(outer.elementCoder, element); + long timestampMsSinceEpoch = timestamp.getMillis(); + return new Message(elementBytes, timestampMsSinceEpoch); + } + + /** + * Return pub/sub message representing this message. + */ + public PubsubMessage toMessage(PubsubUnboundedSink outer) { + PubsubMessage.Builder message = + PubsubMessage.newBuilder().setData(ByteString.copyFrom(elementBytes)); + + if (outer.timestampLabel != null) { + message.getMutableAttributes().put(outer.timestampLabel, String.valueOf + (timestampMsSinceEpoch)); + } + + if (outer.idLabel != null) { + message.getMutableAttributes().put(outer.idLabel, + Hashing.murmur3_128() + .hashBytes(elementBytes) + .toString()); + } + + return message.build(); + } + } + + // ================================================================================ + // 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); + c.output(KV.of(ThreadLocalRandom.current().nextInt(numCores * SCALE_OUT), + Message.forElement(PubsubUnboundedSink.this, c.element(), c.timestamp()))); + } + } + + // ================================================================================ + // 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 PubsubGrpcClient 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(); + + PublishRequest request = PublishRequest.newBuilder() + .setTopic(topic) + .addAllMessages(messages).build(); + PublishResponse response = pubsubClient.publish(request); + Preconditions.checkState(response.getMessageIdsCount() == 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(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 (Message 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.toMessage(PubsubUnboundedSink.this)); + 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(), Message.CODER)) + .apply(GroupByKey.create()) + .apply(ParDo.named(label + ".Writer").of(new WriterFn())); + return PDone.in(input.getPipeline()); + } +} 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..4d78c9046613 --- /dev/null +++ b/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/io/PubsubUnboundedSource.java @@ -0,0 +1,1141 @@ +/* + * 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.sdk.io; + +import com.google.api.client.util.DateTime; +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.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.CoderUtils; +import com.google.cloud.dataflow.sdk.values.PBegin; +import com.google.cloud.dataflow.sdk.values.PCollection; +import com.google.protobuf.Empty; +import com.google.protobuf.Timestamp; +import com.google.pubsub.v1.AcknowledgeRequest; +import com.google.pubsub.v1.ModifyAckDeadlineRequest; +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 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.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; +import org.joda.time.Duration; +import org.joda.time.Instant; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * 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.Reader} 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; + + // ================================================================================ + // Message + // ================================================================================ + + /** + * A message received from pub/sub. + */ + private static class Message { + /** + * Underlying (encoded) element. + */ + private final byte[] elementBytes; + + /** + * Timestamp for element (ms since epoch). Either pub/sub's processing time, + * or the custom timestamp associated with the message. + */ + private final long timestampMsSinceEpoch; + + /** + * Timestamp (in system time) at which we requested the message (ms since epoch). + */ + private final long requestTimeMsSinceEpoch; + + /** + * Id to pass back to pub/sub to acknowledge receipt of this message. + */ + private final String ackId; + + /** + * Id to pass to the runner to distinguish this message from all others. + */ + private final byte[] recordId; + + public Message( + 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; + } + + /* + * Return the size of (encoded) element, in bytes. + */ + public long getElementByteSize() { + return elementBytes.length; + } + + /** + * Decode and return the element. + */ + public T getElement(PubsubUnboundedSource outer) throws CoderException { + return CoderUtils.decodeFromByteArray(outer.elementCoder, elementBytes); + } + + /** + * Return the element's timestamp. + */ + public Instant getTimestamp() { + return new Instant(timestampMsSinceEpoch); + } + + /** + * Return the record id to use to distinguish this element from all others. + */ + public byte[] getRecordId() { + return recordId; + } + + /** + * Decode message from a pub/sub message. + */ + public static Message fromMessage( + long requestTimeMsSinceEpoch, PubsubUnboundedSource outer, + ReceivedMessage receivedMessage) + throws CoderException { + PubsubMessage pubsubMessage = receivedMessage.getMessage(); + Map attributes = pubsubMessage.getAttributes(); + + // Payload. + byte[] elementBytes = pubsubMessage.getData().toByteArray(); + + // Timestamp. + // Start with pub/sub processing time. + Timestamp timestampProto = pubsubMessage.getPublishTime(); + long timestampMsSinceEpoch = timestampProto.getSeconds() + timestampProto.getNanos() / 1000L; + if (outer.timestampLabel != null && attributes != null) { + String timestampString = attributes.get(outer.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 _1) { + 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 _2) { + // Fallback to pub/sub processing time. + } + } + } + // else: fallback to pub/sub processing time. + } + // else: fallback to pub/sub processing time. + + // Ack id. + String ackId = receivedMessage.getAckId(); + Preconditions.checkState(ackId != null && !ackId.isEmpty()); + + // Record id, if any. + @Nullable byte[] recordId = null; + if (outer.idLabel != null && attributes != null) { + String recordIdString = attributes.get(outer.idLabel); + if (recordIdString != null && !recordIdString.isEmpty()) { + recordId = recordIdString.getBytes(); + } + } + if (recordId == null) { + recordId = pubsubMessage.getMessageId().getBytes(); + } + + return new Message(elementBytes, timestampMsSinceEpoch, requestTimeMsSinceEpoch, ackId, + recordId); + } + } + + // ================================================================================ + // 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> { + 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 Message 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(SimpleFunction 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(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, + SimpleFunction.MIN); + minReadTimestampMsSinceEpoch = newFun(SimpleFunction.MIN); + lastReceivedMsSinceEpoch = -1; + lastWatermarkMsSinceEpoch = BoundedWindow.TIMESTAMP_MIN_VALUE.getMillis(); + current = null; + lastLogTimestampMsSinceEpoch = -1; + numReceived = 0L; + numReceivedRecently = newFun(SimpleFunction.SUM); + numExtendedDeadlines = newFun(SimpleFunction.SUM); + numLateDeadlines = newFun(SimpleFunction.SUM); + numAcked = newFun(SimpleFunction.SUM); + numNacked = newFun(SimpleFunction.SUM); + numReadBytes = newFun(SimpleFunction.SUM); + minReceivedTimestampMsSinceEpoch = newFun(SimpleFunction.MIN); + maxReceivedTimestampMsSinceEpoch = newFun(SimpleFunction.MAX); + minWatermarkMsSinceEpoch = newFun(SimpleFunction.MIN); + maxWatermarkMsSinceEpoch = newFun(SimpleFunction.MAX); + numLateMessages = newFun(SimpleFunction.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 { + AcknowledgeRequest request = AcknowledgeRequest.newBuilder() + .setSubscription(outer.outer.subscription) + .addAllAckIds(ackIds) + .build(); + Empty response = pubsubClient.acknowledge(request); + 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 { + ModifyAckDeadlineRequest request = + ModifyAckDeadlineRequest.newBuilder() + .setSubscription(outer.outer.subscription) + .addAllAckIds(ackIds) + .setAckDeadlineSeconds(0) + .build(); + Empty response = pubsubClient.modifyAckDeadline(request); + 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 { + ModifyAckDeadlineRequest request = + ModifyAckDeadlineRequest.newBuilder() + .setSubscription(outer.outer.subscription) + .addAllAckIds(ackIds) + .setAckDeadlineSeconds((int) ACK_EXTENSION.getStandardSeconds()) + .build(); + Empty response = pubsubClient.modifyAckDeadline(request); + 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. + PullRequest request = PullRequest.newBuilder() + .setSubscription(outer.outer.subscription) + .setReturnImmediately(true) + .setMaxMessages(PULL_BATCH_SIZE) + .build(); + PullResponse response = pubsubClient.pull(request); + List receivedMessages = response.getReceivedMessagesList(); + if (receivedMessages == null || receivedMessages.isEmpty()) { + // Nothing available yet. Try again later. + return; + } + + lastReceivedMsSinceEpoch = requestTimeMsSinceEpoch; + + // Capture the received messages. + for (ReceivedMessage receivedMessage : receivedMessages) { + Message message = + Message.fromMessage(requestTimeMsSinceEpoch, outer.outer, receivedMessage); + notYetRead.add(message); + notYetReadBytes += message.getElementByteSize(); + inFlight.put(message.ackId, new Instant(deadlineMsSinceEpoch)); + numReceived++; + numReceivedRecently.add(requestTimeMsSinceEpoch, 1L); + minReceivedTimestampMsSinceEpoch.add(requestTimeMsSinceEpoch, + message.timestampMsSinceEpoch); + maxReceivedTimestampMsSinceEpoch.add(requestTimeMsSinceEpoch, + message.timestampMsSinceEpoch); + minUnreadTimestampMsSinceEpoch.add(requestTimeMsSinceEpoch, message.timestampMsSinceEpoch); + if (message.timestampMsSinceEpoch < lastWatermarkMsSinceEpoch) { + numLateMessages.add(requestTimeMsSinceEpoch, 1L); + } + } + } + + /** + * 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.getElementByteSize(); + Preconditions.checkState(notYetReadBytes >= 0); + long nowMsSinceEpoch = System.currentTimeMillis(); + numReadBytes.add(nowMsSinceEpoch, current.getElementByteSize()); + minReadTimestampMsSinceEpoch.add(nowMsSinceEpoch, current.timestampMsSinceEpoch); + + // 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 current.getElement(outer.outer); + } 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 current.getTimestamp(); + } + + @Override + public byte[] getCurrentRecordId() throws NoSuchElementException { + if (current == null) { + throw new NoSuchElementException(); + } + return current.getRecordId(); + } + + @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 (Message message : notYetRead) { + snapshotNotYetReadIds.add(message.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/io/SimpleFunction.java b/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/io/SimpleFunction.java new file mode 100644 index 000000000000..172d84b1213f --- /dev/null +++ b/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/io/SimpleFunction.java @@ -0,0 +1,46 @@ +package com.google.cloud.dataflow.sdk.io; + +/** + * Some simple functions (commutative and associative with a zero) + * usable with {@link MovingFunction} and {@link BucketingFunction}. + */ +public enum SimpleFunction { + MIN { + @Override + public long f(long l, long r) { + return Math.min(l, r); + } + + @Override + public long zero() { + return Long.MAX_VALUE; + } + }, + MAX { + @Override + public long f(long l, long r) { + return Math.max(l, r); + } + + @Override + public long zero() { + return Long.MIN_VALUE; + } + }, + SUM { + @Override + public long f(long l, long r) { + return l + r; + } + + @Override + public long zero() { + return 0; + } + }; + + public abstract long f(long l, long r); + + public abstract long zero(); +} + diff --git a/sdks/java/core/src/test/java/com/google/cloud/dataflow/sdk/io/BucketingFunctionTest.java b/sdks/java/core/src/test/java/com/google/cloud/dataflow/sdk/io/BucketingFunctionTest.java new file mode 100644 index 000000000000..2877c8c8e5bf --- /dev/null +++ b/sdks/java/core/src/test/java/com/google/cloud/dataflow/sdk/io/BucketingFunctionTest.java @@ -0,0 +1,86 @@ +/* + * Copyright (C) 2014 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.sdk.io; + +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 BucketingFunction newFunc() { + return new + BucketingFunction(BUCKET_WIDTH, SIGNIFICANT_BUCKETS, + SIGNIFICANT_SAMPLES, SimpleFunction.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/io/MovingFunctionTest.java b/sdks/java/core/src/test/java/com/google/cloud/dataflow/sdk/io/MovingFunctionTest.java new file mode 100644 index 000000000000..7095b10ddffa --- /dev/null +++ b/sdks/java/core/src/test/java/com/google/cloud/dataflow/sdk/io/MovingFunctionTest.java @@ -0,0 +1,97 @@ +/* + * Copyright (C) 2014 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.sdk.io; + +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 MovingFunction newFunc() { + return new + MovingFunction(SAMPLE_PERIOD, SAMPLE_UPDATE, SIGNIFICANT_BUCKETS, + SIGNIFICANT_SAMPLES, SimpleFunction.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 a05030890378d7c690fad0807a0fe39fb2a5b201 Mon Sep 17 00:00:00 2001 From: Mark Shields Date: Mon, 28 Mar 2016 14:58:49 -0700 Subject: [PATCH 03/14] Formatting busywork --- pom.xml | 1 + sdks/java/core/pom.xml | 22 +++++++++++- .../dataflow/sdk/io/BucketingFunction.java | 16 +++++++++ .../cloud/dataflow/sdk/io/MovingFunction.java | 8 ++--- .../dataflow/sdk/io/PubsubGrpcClient.java | 36 +++++++++++++------ .../dataflow/sdk/io/PubsubUnboundedSink.java | 15 ++++---- .../sdk/io/PubsubUnboundedSource.java | 22 ++++++------ .../cloud/dataflow/sdk/io/SimpleFunction.java | 16 +++++++++ .../sdk/io/BucketingFunctionTest.java | 4 +-- .../dataflow/sdk/io/MovingFunctionTest.java | 6 ++-- 10 files changed, 107 insertions(+), 39 deletions(-) diff --git a/pom.xml b/pom.xml index 3cae1a45d65a..ea7578254b8f 100644 --- a/pom.xml +++ b/pom.xml @@ -103,6 +103,7 @@ 1.7.7 v2-rev248-1.21.0 0.2.3 + 0.0.1 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 41cf64a4f194..97621362cbf8 100644 --- a/sdks/java/core/pom.xml +++ b/sdks/java/core/pom.xml @@ -389,10 +389,30 @@ 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 - 0.0.1 + ${pubsubgrpc.version} diff --git a/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/io/BucketingFunction.java b/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/io/BucketingFunction.java index eae0069bac13..36a681858544 100644 --- a/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/io/BucketingFunction.java +++ b/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/io/BucketingFunction.java @@ -1,3 +1,19 @@ +/* + * 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.sdk.io; import com.google.common.base.Preconditions; diff --git a/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/io/MovingFunction.java b/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/io/MovingFunction.java index aadc5aaa7f46..96ce6433cbb2 100644 --- a/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/io/MovingFunction.java +++ b/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/io/MovingFunction.java @@ -1,5 +1,5 @@ /* - * Copyright (C) 2015 Google Inc. + * 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 @@ -136,11 +136,11 @@ public long get(long nowMsSinceEpoch) { * or from enough samples? */ public boolean isSignificant() { - int totalSamples =0; + int totalSamples = 0; int activeBuckets = 0; - for (int i =0; i < buckets.length; i++) { + for (int i = 0; i < buckets.length; i++) { totalSamples += numSamples[i]; - if (numSamples[i]> 0) { + if (numSamples[i] > 0) { activeBuckets++; } } 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 index 753289218481..cfb6a2a582d4 100644 --- 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 @@ -1,3 +1,19 @@ +/* + * 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.sdk.io; import com.google.auth.oauth2.GoogleCredentials; @@ -19,10 +35,9 @@ import io.grpc.netty.GrpcSslContexts; import io.grpc.netty.NegotiationType; import io.grpc.netty.NettyChannelBuilder; -import io.grpc.stub.AbstractStub; import java.io.IOException; import java.security.GeneralSecurityException; -import java.util.Arrays; +import java.util.Collections; import java.util.List; import java.util.concurrent.Executors; import java.util.concurrent.TimeUnit; @@ -31,16 +46,16 @@ /** * A helper class for talking to pub/sub via grpc. */ -public class PubsubGrpcClient implements AutoCloseable { +class PubsubGrpcClient implements AutoCloseable { private static final String PUBSUB_ADDRESS = "pubsub.googleapis.com"; private static final int PUBSUB_PORT = 443; private static final List PUBSUB_SCOPES = - Arrays.asList("https://www.googleapis.com/auth/pubsub"); + Collections.singletonList("https://www.googleapis.com/auth/pubsub"); /** * Timeout for grpc calls (in s). */ - public static final int TIMEOUT_S = 15; + private static final int TIMEOUT_S = 15; /** * Underlying netty channel, or null if closed. @@ -69,7 +84,7 @@ private PubsubGrpcClient(ManagedChannel publisherChannel, GoogleCredentials cred * Construct a new pub/sub grpc client. It should be closed via {@link #close} in order * to ensure tidy cleanup of underlying netty resources. */ - public static PubsubGrpcClient newClient(GcpOptions options) + static PubsubGrpcClient newClient(GcpOptions options) throws IOException, GeneralSecurityException { ManagedChannel channel = NettyChannelBuilder .forAddress(PUBSUB_ADDRESS, PUBSUB_PORT) @@ -88,6 +103,7 @@ public static PubsubGrpcClient newClient(GcpOptions options) */ @Override public void close() { + Preconditions.checkState(publisherChannel != null, "Client has already been closed"); publisherChannel.shutdown(); try { publisherChannel.awaitTermination(TIMEOUT_S, TimeUnit.SECONDS); @@ -133,19 +149,19 @@ private SubscriberGrpc.SubscriberBlockingStub subscriberStub() throws IOExceptio /** * The following are pass-through. */ - public PublishResponse publish(PublishRequest request) throws IOException { + PublishResponse publish(PublishRequest request) throws IOException { return publisherStub().publish(request); } - public Empty acknowledge(AcknowledgeRequest request) throws IOException { + Empty acknowledge(AcknowledgeRequest request) throws IOException { return subscriberStub().acknowledge(request); } - public Empty modifyAckDeadline(ModifyAckDeadlineRequest request) throws IOException { + Empty modifyAckDeadline(ModifyAckDeadlineRequest request) throws IOException { return subscriberStub().modifyAckDeadline(request); } - public PullResponse pull(PullRequest request) throws IOException { + PullResponse pull(PullRequest request) throws IOException { return subscriberStub().pull(request); } } 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 index a2f970aa97a8..6696ea8b439d 100644 --- 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 @@ -1,5 +1,5 @@ /* - * Copyright (C) 2015 Google Inc. + * 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 @@ -45,20 +45,19 @@ import com.google.protobuf.ByteString; 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 org.joda.time.Duration; +import org.joda.time.Instant; + 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 java.util.concurrent.TimeUnit; + import javax.annotation.Nullable; -import org.joda.time.Duration; -import org.joda.time.Instant; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; /** * A PTransform which streams messages to pub/sub. @@ -73,8 +72,6 @@ * */ public class PubsubUnboundedSink extends PTransform, PDone> { - private static final Logger LOG = LoggerFactory.getLogger(PubsubUnboundedSink.class); - /** * Maximum number of messages per publish. */ 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 index 4d78c9046613..df837cd9db21 100644 --- 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 @@ -1,5 +1,5 @@ /* - * Copyright (C) 2015 Google Inc. + * 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 @@ -42,7 +42,12 @@ 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 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; @@ -56,11 +61,8 @@ import java.util.Queue; import java.util.concurrent.ConcurrentLinkedQueue; import java.util.concurrent.atomic.AtomicInteger; + import javax.annotation.Nullable; -import org.joda.time.Duration; -import org.joda.time.Instant; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; /** * A PTransform which streams messages from pub/sub. @@ -82,7 +84,7 @@ *

  • 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.Reader} instance to execute concurrently and thus hide latency. + * {@link UnboundedSource#UnboundedReader} instance to execute concurrently and thus hide latency. * */ public class PubsubUnboundedSource extends PTransform> { @@ -260,12 +262,12 @@ public static Message fromMessage( // Expected IllegalArgumentException if parsing fails; we use that to fall back // to RFC 3339. timestampMsSinceEpoch = Long.parseLong(timestampString); - } catch (IllegalArgumentException _1) { + } 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 _2) { + } catch (IllegalArgumentException e2) { // Fallback to pub/sub processing time. } } @@ -407,7 +409,7 @@ public void nackAll() throws IOException { * The coder for our checkpoints. */ private static class CheckpointCoder extends AtomicCoder> { - Coder> LIST_CODER = ListCoder.of(StringUtf8Coder.of()); + private static final Coder> LIST_CODER = ListCoder.of(StringUtf8Coder.of()); @Override public void encode(Checkpoint value, OutputStream outStream, Context context) diff --git a/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/io/SimpleFunction.java b/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/io/SimpleFunction.java index 172d84b1213f..4b735d14192a 100644 --- a/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/io/SimpleFunction.java +++ b/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/io/SimpleFunction.java @@ -1,3 +1,19 @@ +/* + * 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.sdk.io; /** diff --git a/sdks/java/core/src/test/java/com/google/cloud/dataflow/sdk/io/BucketingFunctionTest.java b/sdks/java/core/src/test/java/com/google/cloud/dataflow/sdk/io/BucketingFunctionTest.java index 2877c8c8e5bf..0e43739c2b30 100644 --- a/sdks/java/core/src/test/java/com/google/cloud/dataflow/sdk/io/BucketingFunctionTest.java +++ b/sdks/java/core/src/test/java/com/google/cloud/dataflow/sdk/io/BucketingFunctionTest.java @@ -1,5 +1,5 @@ /* - * Copyright (C) 2014 Google Inc. + * 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 @@ -76,7 +76,7 @@ public void movingSum() { f.add(i, 1); if (i >= 100) { f.remove(i - 100); - if (i % BUCKET_WIDTH == BUCKET_WIDTH -1) { + if (i % BUCKET_WIDTH == BUCKET_WIDTH - 1) { lost += BUCKET_WIDTH; } } diff --git a/sdks/java/core/src/test/java/com/google/cloud/dataflow/sdk/io/MovingFunctionTest.java b/sdks/java/core/src/test/java/com/google/cloud/dataflow/sdk/io/MovingFunctionTest.java index 7095b10ddffa..11e5edddc98e 100644 --- a/sdks/java/core/src/test/java/com/google/cloud/dataflow/sdk/io/MovingFunctionTest.java +++ b/sdks/java/core/src/test/java/com/google/cloud/dataflow/sdk/io/MovingFunctionTest.java @@ -1,5 +1,5 @@ /* - * Copyright (C) 2014 Google Inc. + * 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 @@ -89,8 +89,8 @@ public void movingSum() { public void jumpingSum() { MovingFunction f = newFunc(); f.add(0, 1); - f.add(SAMPLE_PERIOD -1, 1); - assertEquals(2, f.get(SAMPLE_PERIOD -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 f95466ea8bf72e33512ca46e6acc413d55ffda5d Mon Sep 17 00:00:00 2001 From: Mark Shields Date: Wed, 30 Mar 2016 11:52:34 -0700 Subject: [PATCH 04/14] pub/sub source/sink and grpc client refactor Move some pubsub source/sink machinery into PubSubGrpcClient so that it's api can be grpc and protoc neutral. --- .../dataflow/sdk/io/PubsubGrpcClient.java | 353 +++++++++++++++++- .../dataflow/sdk/io/PubsubUnboundedSink.java | 137 ++----- .../sdk/io/PubsubUnboundedSource.java | 230 ++---------- 3 files changed, 410 insertions(+), 310 deletions(-) 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 index cfb6a2a582d4..f5bec9355364 100644 --- 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 @@ -16,18 +16,32 @@ 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.protobuf.Empty; +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; @@ -35,22 +49,27 @@ import io.grpc.netty.GrpcSslContexts; import io.grpc.netty.NegotiationType; import io.grpc.netty.NettyChannelBuilder; + import java.io.IOException; -import java.security.GeneralSecurityException; +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 pub/sub via grpc. */ -class PubsubGrpcClient implements AutoCloseable { +public class PubsubGrpcClient implements AutoCloseable { 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). @@ -66,7 +85,20 @@ class PubsubGrpcClient implements AutoCloseable { /** * Credentials determined from options and environment. */ - private GoogleCredentials credentials; + private final GoogleCredentials credentials; + + /** + * Label to use for custom timestamps, or {@literal null} if should use pub/sub publish time + * instead. + */ + @Nullable + private final String timestampLabel; + + /** + * Label to use for custom ids, or {@literal null} if should use pub/sub provided ids. + */ + @Nullable + private final String idLabel; /** * Cached stubs, or null if not cached. @@ -75,17 +107,24 @@ class PubsubGrpcClient implements AutoCloseable { private PublisherGrpc.PublisherBlockingStub cachedPublisherStub; private SubscriberGrpc.SubscriberBlockingStub cachedSubscriberStub; - private PubsubGrpcClient(ManagedChannel publisherChannel, GoogleCredentials credentials) { + 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 pub/sub grpc client. It should be closed via {@link #close} in order - * to ensure tidy cleanup of underlying netty resources. + * to ensure tidy cleanup of underlying netty resources. If non-{@literal null}, use + * {@code timestampLabel} and {@code idLabel} to store custom timestamps/ids within + * message metadata. */ - static PubsubGrpcClient newClient(GcpOptions options) - throws IOException, GeneralSecurityException { + 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) @@ -95,7 +134,7 @@ static PubsubGrpcClient newClient(GcpOptions options) // various command line options. It currently only supports the older // com.google.api.client.auth.oauth2.Credentials. GoogleCredentials credentials = GoogleCredentials.getApplicationDefault(); - return new PubsubGrpcClient(channel, credentials); + return new PubsubGrpcClient(timestampLabel, idLabel, channel, credentials); } /** @@ -147,21 +186,299 @@ private SubscriberGrpc.SubscriberBlockingStub subscriberStub() throws IOExceptio } /** - * The following are pass-through. + * A message to be sent to pub/sub. */ - PublishResponse publish(PublishRequest request) throws IOException { - return publisherStub().publish(request); + public static 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; + } } - Empty acknowledge(AcknowledgeRequest request) throws IOException { - return subscriberStub().acknowledge(request); + /** + * A message received from pub/sub. + */ + public static class IncomingMessage { + /** + * Underlying (encoded) element. + */ + public final byte[] elementBytes; + + /** + * Timestamp for element (ms since epoch). Either pub/sub'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 pub/sub 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; + } } - Empty modifyAckDeadline(ModifyAckDeadlineRequest request) throws IOException { - return subscriberStub().modifyAckDeadline(request); + /** + * Publish {@code outgoingMessages} to pub/sub {@code topic}. Return number of messages + * published. + * + * @throws IOException + */ + 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(); } - PullResponse pull(PullRequest request) throws IOException { - return subscriberStub().pull(request); + /** + * Request the next batch of up to {@code batchSize} messages from {@code subscription}. + * + * @throws IOException + */ + 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 pub/sub 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 pub/sub processing time. + } + } + } + // else: fallback to pub/sub processing time. + } + // else: fallback to pub/sub 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; + } + + /** + * Acknowldege messages from {@code subscription} with {@code ackIds}. + * + * @throws IOException + */ + public void acknowledge(String subscription, Iterable ackIds) throws IOException { + AcknowledgeRequest request = AcknowledgeRequest.newBuilder() + .setSubscription(subscription) + .addAllAckIds(ackIds) + .build(); + subscriberStub().acknowledge(request); // ignore Empty result. + } + + /** + * Modify the ack deadline for messages from {@code subscription} with {@code ackIds}. + * + * @throws IOException + */ + 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. + } + + + /** + * Create {@code topic}. + */ + public void createTopic(String topic) throws IOException { + Topic request = Topic.newBuilder() + .setName(topic) + .build(); + publisherStub().createTopic(request); // ignore Topic result. + } + + /* + * Delete {@code topic}. + */ + public void deleteTopic(String topic) throws IOException { + DeleteTopicRequest request = DeleteTopicRequest.newBuilder().setTopic(topic).build(); + publisherStub().deleteTopic(request); // ignore Empty result. + } + + /** + * Return a list of topics for {@code project}. + */ + 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; + } + + /** + * Create {@code subscription} to {@code topic}. + * + * @throws IOException + */ + 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. + } + + /** + * Delete {@code subscription}. + */ + public void deleteSubscription(String subscription) throws IOException { + DeleteSubscriptionRequest request = + DeleteSubscriptionRequest.newBuilder().setSubscription(subscription).build(); + subscriberStub().deleteSubscription(request); // ignore Empty result. + } + + /** + * Return a list of subscriptions for {@code topic} in {@code project}. + */ + 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; } } 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 index 6696ea8b439d..004e1d620b2e 100644 --- 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 @@ -1,5 +1,5 @@ /* - * Copyright (C) 2016 Google Inc. + * 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 @@ -41,23 +41,19 @@ import com.google.cloud.dataflow.sdk.values.PCollection; import com.google.cloud.dataflow.sdk.values.PDone; import com.google.common.base.Preconditions; -import com.google.common.hash.Hashing; -import com.google.protobuf.ByteString; import com.google.pubsub.v1.PublishRequest; import com.google.pubsub.v1.PublishResponse; import com.google.pubsub.v1.PubsubMessage; - -import org.joda.time.Duration; -import org.joda.time.Instant; - 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; +import org.joda.time.Duration; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; /** * A PTransform which streams messages to pub/sub. @@ -72,6 +68,8 @@ * */ public class PubsubUnboundedSink extends PTransform, PDone> { + private static final Logger LOG = LoggerFactory.getLogger(PubsubUnboundedSink.class); + /** * Maximum number of messages per publish. */ @@ -107,79 +105,24 @@ public class PubsubUnboundedSink extends PTransform, PDone> { */ private static final int SCALE_OUT = 4; - // ================================================================================ - // Message - // ================================================================================ - - /** - * A message to be sent to pub/sub. - */ - private static class Message { - public static final Coder CODER = new AtomicCoder() { - @Override - public void encode( - Message 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 Message 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 Message(elementBytes, timestampMsSinceEpoch); - } - }; - - /** - * Underlying (encoded) element. - */ - private final byte[] elementBytes; - - /** - * Timestamp for element (ms since epoch). - */ - private final long timestampMsSinceEpoch; - - public Message(byte[] elementBytes, long timestampMsSinceEpoch) { - this.elementBytes = elementBytes; - this.timestampMsSinceEpoch = timestampMsSinceEpoch; - } - - /** - * Return message for element. - */ - public static Message forElement( - PubsubUnboundedSink outer, T element, Instant timestamp) throws CoderException { - byte[] elementBytes = CoderUtils.encodeToByteArray(outer.elementCoder, element); - long timestampMsSinceEpoch = timestamp.getMillis(); - return new Message(elementBytes, timestampMsSinceEpoch); - } - - /** - * Return pub/sub message representing this message. - */ - public PubsubMessage toMessage(PubsubUnboundedSink outer) { - PubsubMessage.Builder message = - PubsubMessage.newBuilder().setData(ByteString.copyFrom(elementBytes)); - - if (outer.timestampLabel != null) { - message.getMutableAttributes().put(outer.timestampLabel, String.valueOf - (timestampMsSinceEpoch)); - } - - if (outer.idLabel != null) { - message.getMutableAttributes().put(outer.idLabel, - Hashing.murmur3_128() - .hashBytes(elementBytes) - .toString()); - } + public static final Coder CODER = new + AtomicCoder() { + @Override + public void encode( + PubsubGrpcClient.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); + } - return message.build(); - } - } + @Override + public PubsubGrpcClient.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 PubsubGrpcClient.OutgoingMessage(elementBytes, timestampMsSinceEpoch); + } + }; // ================================================================================ // ShardFv @@ -188,7 +131,7 @@ public PubsubMessage toMessage(PubsubUnboundedSink outer) { /** * Convert elements to messages and shard them. */ - private class ShardFn extends DoFn> { + private class ShardFn extends DoFn> { /** * Number of cores available for publishing. */ @@ -204,8 +147,10 @@ public ShardFn(int 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), - Message.forElement(PubsubUnboundedSink.this, c.element(), c.timestamp()))); + new PubsubGrpcClient.OutgoingMessage(elementBytes, timestampMsSinceEpoch))); } } @@ -216,7 +161,8 @@ public void processElement(ProcessContext c) throws Exception { /** * Publish messages to pub/sub in batches. */ - private class WriterFn extends DoFn>, Void> { + private class WriterFn + extends DoFn>, Void> { /** * Client on which to talk to pub/sub. Null until created by {@link #startBundle}. @@ -235,14 +181,11 @@ private class WriterFn extends DoFn>, Void> { * BLOCKING * Send {@code messages} as a batch to pub/sub. */ - private void publishBatch(List messages, int bytes) throws IOException { + private void publishBatch(List messages, int bytes) + throws IOException { long nowMsSinceEpoch = System.currentTimeMillis(); - - PublishRequest request = PublishRequest.newBuilder() - .setTopic(topic) - .addAllMessages(messages).build(); - PublishResponse response = pubsubClient.publish(request); - Preconditions.checkState(response.getMessageIdsCount() == messages.size()); + int n = pubsubClient.publish(topic, messages); + Preconditions.checkState(n == messages.size()); batchCounter.addValue(1L); elementCounter.addValue((long) messages.size()); byteCounter.addValue((long) bytes); @@ -251,15 +194,16 @@ private void publishBatch(List messages, int bytes) throws IOExce @Override public void startBundle(Context c) throws Exception { Preconditions.checkState(pubsubClient == null); - pubsubClient = PubsubGrpcClient.newClient(c.getPipelineOptions().as(GcpOptions.class)); + 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); + List pubsubMessages = new ArrayList<>(PUBLISH_BATCH_SIZE); int bytes = 0; - for (Message message : c.element().getValue()) { + for (PubsubGrpcClient.OutgoingMessage message : c.element().getValue()) { if (!pubsubMessages.isEmpty() && bytes + message.elementBytes.length > PUBLISH_BATCH_BYTES) { // Break large (in bytes) batches into smaller. @@ -270,7 +214,7 @@ public void processElement(ProcessContext c) throws Exception { pubsubMessages.clear(); bytes = 0; } - pubsubMessages.add(message.toMessage(PubsubUnboundedSink.this)); + pubsubMessages.add(message); bytes += message.elementBytes.length; } if (!pubsubMessages.isEmpty()) { @@ -287,6 +231,7 @@ public void finishBundle(Context c) throws Exception { } } + // ================================================================================ // PubsubUnboundedSink // ================================================================================ @@ -348,8 +293,8 @@ public PDone apply(PCollection input) { .discardingFiredPanes() .withAllowedLateness(Duration.ZERO)) .apply(ParDo.named(label + ".Shard").of(new ShardFn(numCores))) - .setCoder(KvCoder.of(VarIntCoder.of(), Message.CODER)) - .apply(GroupByKey.create()) + .setCoder(KvCoder.of(VarIntCoder.of(), CODER)) + .apply(GroupByKey.create()) .apply(ParDo.named(label + ".Writer").of(new WriterFn())); return PDone.in(input.getPipeline()); } 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 index df837cd9db21..f819ea46ab2d 100644 --- 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 @@ -16,7 +16,6 @@ package com.google.cloud.dataflow.sdk.io; -import com.google.api.client.util.DateTime; import com.google.api.client.util.Preconditions; import com.google.cloud.dataflow.sdk.coders.AtomicCoder; import com.google.cloud.dataflow.sdk.coders.Coder; @@ -34,15 +33,6 @@ import com.google.cloud.dataflow.sdk.util.CoderUtils; import com.google.cloud.dataflow.sdk.values.PBegin; import com.google.cloud.dataflow.sdk.values.PCollection; -import com.google.protobuf.Empty; -import com.google.protobuf.Timestamp; -import com.google.pubsub.v1.AcknowledgeRequest; -import com.google.pubsub.v1.ModifyAckDeadlineRequest; -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 org.joda.time.Duration; import org.joda.time.Instant; import org.slf4j.Logger; @@ -54,6 +44,7 @@ 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; @@ -84,7 +75,7 @@ *
  • 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. + * {@link UnboundedSource.UnboundedReader} instance to execute concurrently and thus hide latency. * */ public class PubsubUnboundedSource extends PTransform> { @@ -161,142 +152,6 @@ public class PubsubUnboundedSource extends PTransform> */ private static final int SCALE_OUT = 4; - // ================================================================================ - // Message - // ================================================================================ - - /** - * A message received from pub/sub. - */ - private static class Message { - /** - * Underlying (encoded) element. - */ - private final byte[] elementBytes; - - /** - * Timestamp for element (ms since epoch). Either pub/sub's processing time, - * or the custom timestamp associated with the message. - */ - private final long timestampMsSinceEpoch; - - /** - * Timestamp (in system time) at which we requested the message (ms since epoch). - */ - private final long requestTimeMsSinceEpoch; - - /** - * Id to pass back to pub/sub to acknowledge receipt of this message. - */ - private final String ackId; - - /** - * Id to pass to the runner to distinguish this message from all others. - */ - private final byte[] recordId; - - public Message( - 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; - } - - /* - * Return the size of (encoded) element, in bytes. - */ - public long getElementByteSize() { - return elementBytes.length; - } - - /** - * Decode and return the element. - */ - public T getElement(PubsubUnboundedSource outer) throws CoderException { - return CoderUtils.decodeFromByteArray(outer.elementCoder, elementBytes); - } - - /** - * Return the element's timestamp. - */ - public Instant getTimestamp() { - return new Instant(timestampMsSinceEpoch); - } - - /** - * Return the record id to use to distinguish this element from all others. - */ - public byte[] getRecordId() { - return recordId; - } - - /** - * Decode message from a pub/sub message. - */ - public static Message fromMessage( - long requestTimeMsSinceEpoch, PubsubUnboundedSource outer, - ReceivedMessage receivedMessage) - throws CoderException { - PubsubMessage pubsubMessage = receivedMessage.getMessage(); - Map attributes = pubsubMessage.getAttributes(); - - // Payload. - byte[] elementBytes = pubsubMessage.getData().toByteArray(); - - // Timestamp. - // Start with pub/sub processing time. - Timestamp timestampProto = pubsubMessage.getPublishTime(); - long timestampMsSinceEpoch = timestampProto.getSeconds() + timestampProto.getNanos() / 1000L; - if (outer.timestampLabel != null && attributes != null) { - String timestampString = attributes.get(outer.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 pub/sub processing time. - } - } - } - // else: fallback to pub/sub processing time. - } - // else: fallback to pub/sub processing time. - - // Ack id. - String ackId = receivedMessage.getAckId(); - Preconditions.checkState(ackId != null && !ackId.isEmpty()); - - // Record id, if any. - @Nullable byte[] recordId = null; - if (outer.idLabel != null && attributes != null) { - String recordIdString = attributes.get(outer.idLabel); - if (recordIdString != null && !recordIdString.isEmpty()) { - recordId = recordIdString.getBytes(); - } - } - if (recordId == null) { - recordId = pubsubMessage.getMessageId().getBytes(); - } - - return new Message(elementBytes, timestampMsSinceEpoch, requestTimeMsSinceEpoch, ackId, - recordId); - } - } - // ================================================================================ // Checkpoint // ================================================================================ @@ -453,7 +308,7 @@ private static class Reader extends UnboundedSource.UnboundedReader { * Messages we have received from pub/sub and not yet delivered downstream. * We preserve their order. */ - private final Queue notYetRead; + private final Queue notYetRead; /** * Map from ack ids of messages we have received from pub/sub but not yet acked to their @@ -500,7 +355,7 @@ private static class Reader extends UnboundedSource.UnboundedReader { * The current message, or {@literal null} if none. */ @Nullable - private Message current; + private PubsubGrpcClient.IncomingMessage current; /** * Stats only: System time (ms since epoch) we last logs stats, or -1 if never. @@ -597,7 +452,9 @@ private static MovingFunction newFun(SimpleFunction function) { public Reader(GcpOptions options, Source outer) throws IOException, GeneralSecurityException { this.outer = outer; - pubsubClient = PubsubGrpcClient.newClient(options); + pubsubClient = PubsubGrpcClient.newClient(outer.outer.timestampLabel, + outer.outer.idLabel, + options); safeToAckIds = new ArrayList<>(); notYetRead = new ArrayDeque<>(); inFlight = new LinkedHashMap<>(); @@ -635,11 +492,7 @@ public Reader(GcpOptions options, Source outer) throws IOException, * CAUTION: Retains {@code ackIds}. */ public void ackBatch(List ackIds) throws IOException { - AcknowledgeRequest request = AcknowledgeRequest.newBuilder() - .setSubscription(outer.outer.subscription) - .addAllAckIds(ackIds) - .build(); - Empty response = pubsubClient.acknowledge(request); + pubsubClient.acknowledge(outer.outer.subscription, ackIds); ackedIds.add(ackIds); } @@ -649,13 +502,7 @@ public void ackBatch(List ackIds) throws IOException { * with the given {@code ockIds}. Does not retain {@code ackIds}. */ public void nackBatch(long nowMsSinceEpoch, List ackIds) throws IOException { - ModifyAckDeadlineRequest request = - ModifyAckDeadlineRequest.newBuilder() - .setSubscription(outer.outer.subscription) - .addAllAckIds(ackIds) - .setAckDeadlineSeconds(0) - .build(); - Empty response = pubsubClient.modifyAckDeadline(request); + pubsubClient.modifyAckDeadline(outer.outer.subscription, ackIds, 0); numNacked.add(nowMsSinceEpoch, ackIds.size()); } @@ -665,13 +512,8 @@ public void nackBatch(long nowMsSinceEpoch, List ackIds) throws IOExcept * Does not retain {@code ackIds}. */ private void extendBatch(long nowMsSinceEpoch, List ackIds) throws IOException { - ModifyAckDeadlineRequest request = - ModifyAckDeadlineRequest.newBuilder() - .setSubscription(outer.outer.subscription) - .addAllAckIds(ackIds) - .setAckDeadlineSeconds((int) ACK_EXTENSION.getStandardSeconds()) - .build(); - Empty response = pubsubClient.modifyAckDeadline(request); + pubsubClient.modifyAckDeadline(outer.outer.subscription, ackIds, + (int) ACK_EXTENSION.getStandardSeconds()); numExtendedDeadlines.add(nowMsSinceEpoch, ackIds.size()); } @@ -756,14 +598,11 @@ private void pull() throws IOException { // Pull the next batch. // BLOCKs until received. - PullRequest request = PullRequest.newBuilder() - .setSubscription(outer.outer.subscription) - .setReturnImmediately(true) - .setMaxMessages(PULL_BATCH_SIZE) - .build(); - PullResponse response = pubsubClient.pull(request); - List receivedMessages = response.getReceivedMessagesList(); - if (receivedMessages == null || receivedMessages.isEmpty()) { + Collection receivedMessages = + pubsubClient.pull(requestTimeMsSinceEpoch, + outer.outer.subscription, + PULL_BATCH_SIZE); + if (receivedMessages.isEmpty()) { // Nothing available yet. Try again later. return; } @@ -771,22 +610,18 @@ private void pull() throws IOException { lastReceivedMsSinceEpoch = requestTimeMsSinceEpoch; // Capture the received messages. - for (ReceivedMessage receivedMessage : receivedMessages) { - Message message = - Message.fromMessage(requestTimeMsSinceEpoch, outer.outer, receivedMessage); - notYetRead.add(message); - notYetReadBytes += message.getElementByteSize(); - inFlight.put(message.ackId, new Instant(deadlineMsSinceEpoch)); + for (PubsubGrpcClient.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, - message.timestampMsSinceEpoch); + incomingMessage.timestampMsSinceEpoch); maxReceivedTimestampMsSinceEpoch.add(requestTimeMsSinceEpoch, - message.timestampMsSinceEpoch); - minUnreadTimestampMsSinceEpoch.add(requestTimeMsSinceEpoch, message.timestampMsSinceEpoch); - if (message.timestampMsSinceEpoch < lastWatermarkMsSinceEpoch) { - numLateMessages.add(requestTimeMsSinceEpoch, 1L); - } + incomingMessage.timestampMsSinceEpoch); + minUnreadTimestampMsSinceEpoch.add(requestTimeMsSinceEpoch, + incomingMessage.timestampMsSinceEpoch); } } @@ -901,11 +736,14 @@ public boolean advance() throws IOException { // Try again later. return false; } - notYetReadBytes -= current.getElementByteSize(); + notYetReadBytes -= current.elementBytes.length; Preconditions.checkState(notYetReadBytes >= 0); long nowMsSinceEpoch = System.currentTimeMillis(); - numReadBytes.add(nowMsSinceEpoch, current.getElementByteSize()); + 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); @@ -918,7 +756,7 @@ public T getCurrent() throws NoSuchElementException { throw new NoSuchElementException(); } try { - return current.getElement(outer.outer); + return CoderUtils.decodeFromByteArray(outer.outer.elementCoder, current.elementBytes); } catch (CoderException e) { throw new RuntimeException("Unable to decode element from pub/sub message: ", e); } @@ -929,7 +767,7 @@ public Instant getCurrentTimestamp() throws NoSuchElementException { if (current == null) { throw new NoSuchElementException(); } - return current.getTimestamp(); + return new Instant(current.timestampMsSinceEpoch); } @Override @@ -937,7 +775,7 @@ public byte[] getCurrentRecordId() throws NoSuchElementException { if (current == null) { throw new NoSuchElementException(); } - return current.getRecordId(); + return current.recordId; } @Override @@ -991,8 +829,8 @@ public UnboundedSource.CheckpointMark getCheckpointMark() { List snapshotSafeToAckIds = safeToAckIds; safeToAckIds = new ArrayList<>(); List snapshotNotYetReadIds = new ArrayList<>(notYetRead.size()); - for (Message message : notYetRead) { - snapshotNotYetReadIds.add(message.ackId); + for (PubsubGrpcClient.IncomingMessage incomingMessage : notYetRead) { + snapshotNotYetReadIds.add(incomingMessage.ackId); } return new Checkpoint<>(this, snapshotSafeToAckIds, snapshotNotYetReadIds); } From 08bdb4dd6ebc0ba4d29d42ba5fb8d7c91404a138 Mon Sep 17 00:00:00 2001 From: Mark Shields Date: Wed, 30 Mar 2016 12:22:20 -0700 Subject: [PATCH 05/14] Unused imports --- .../com/google/cloud/dataflow/sdk/io/PubsubUnboundedSink.java | 3 --- 1 file changed, 3 deletions(-) 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 index 004e1d620b2e..d06230bd95ce 100644 --- 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 @@ -41,9 +41,6 @@ import com.google.cloud.dataflow.sdk.values.PCollection; import com.google.cloud.dataflow.sdk.values.PDone; import com.google.common.base.Preconditions; -import com.google.pubsub.v1.PublishRequest; -import com.google.pubsub.v1.PublishResponse; -import com.google.pubsub.v1.PubsubMessage; import java.io.IOException; import java.io.InputStream; import java.io.OutputStream; From 170412eaa185bf4278da3443dea8863d386dc744 Mon Sep 17 00:00:00 2001 From: Mark Shields Date: Wed, 30 Mar 2016 12:39:21 -0700 Subject: [PATCH 06/14] Import order --- .../cloud/dataflow/sdk/io/PubsubUnboundedSink.java | 9 ++++++--- 1 file changed, 6 insertions(+), 3 deletions(-) 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 index d06230bd95ce..b1d7602899c3 100644 --- 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 @@ -41,16 +41,19 @@ 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; -import org.joda.time.Duration; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; /** * A PTransform which streams messages to pub/sub. From 8f82b1d9cabff768b728d0f499f1fa41efb081ff Mon Sep 17 00:00:00 2001 From: Mark Shields Date: Thu, 31 Mar 2016 14:41:00 -0700 Subject: [PATCH 07/14] Upgrade pubsub-grpc-v1 dep --- pom.xml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/pom.xml b/pom.xml index ea7578254b8f..0ad044f56df7 100644 --- a/pom.xml +++ b/pom.xml @@ -103,7 +103,7 @@ 1.7.7 v2-rev248-1.21.0 0.2.3 - 0.0.1 + 0.0.2 v2-rev6-1.21.0 v1b3-rev22-1.21.0 0.5.160222 From 94d0676b65e9fb147ec0e58714836b067bfaa40b Mon Sep 17 00:00:00 2001 From: Mark Shields Date: Thu, 31 Mar 2016 15:15:24 -0700 Subject: [PATCH 08/14] Switch from 'SimpleFunction' to CombineLongsFn. Factor PubsubClient iface out of PubsubGrpcClient impl. --- .../dataflow/sdk/io/BucketingFunction.java | 13 +- .../cloud/dataflow/sdk/io/MovingFunction.java | 15 +- .../cloud/dataflow/sdk/io/PubsubClient.java | 174 ++++++++++++++++++ .../dataflow/sdk/io/PubsubGrpcClient.java | 118 ++---------- .../dataflow/sdk/io/PubsubUnboundedSink.java | 26 +-- .../sdk/io/PubsubUnboundedSource.java | 79 ++++++-- .../cloud/dataflow/sdk/io/SimpleFunction.java | 62 ------- .../sdk/io/BucketingFunctionTest.java | 16 +- .../dataflow/sdk/io/MovingFunctionTest.java | 16 +- 9 files changed, 303 insertions(+), 216 deletions(-) create mode 100644 sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/io/PubsubClient.java delete mode 100644 sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/io/SimpleFunction.java diff --git a/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/io/BucketingFunction.java b/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/io/BucketingFunction.java index 36a681858544..a74564fc1ae7 100644 --- a/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/io/BucketingFunction.java +++ b/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/io/BucketingFunction.java @@ -16,6 +16,7 @@ package com.google.cloud.dataflow.sdk.io; +import com.google.cloud.dataflow.sdk.transforms.Combine; import com.google.common.base.Preconditions; import java.util.HashMap; import java.util.Map; @@ -31,11 +32,11 @@ private static class Bucket { public Bucket(BucketingFunction outer) { numSamples = 0; - combinedValue = outer.function.zero(); + combinedValue = outer.function.identity(); } public void add(BucketingFunction outer, long value) { - combinedValue = outer.function.f(combinedValue, value); + combinedValue = outer.function.apply(combinedValue, value); numSamples++; } @@ -68,7 +69,7 @@ public long get() { /** * Function for combining sample values. */ - private final SimpleFunction function; + private final Combine.BinaryCombineLongFn function; /** * Active buckets. @@ -79,7 +80,7 @@ public BucketingFunction( long bucketWidthMs, int numSignificantBuckets, int numSignificantSamples, - SimpleFunction function) { + Combine.BinaryCombineLongFn function) { this.bucketWidthMs = bucketWidthMs; this.numSignificantBuckets = numSignificantBuckets; this.numSignificantSamples = numSignificantSamples; @@ -125,9 +126,9 @@ public void remove(long timeMsSinceEpoch) { * Return the (bucketized) combined value of all samples. */ public long get() { - long result = function.zero(); + long result = function.identity(); for (Bucket bucket : buckets.values()) { - result = function.f(result, bucket.get()); + result = function.apply(result, bucket.get()); } return result; } diff --git a/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/io/MovingFunction.java b/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/io/MovingFunction.java index 96ce6433cbb2..d0fa67aaa699 100644 --- a/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/io/MovingFunction.java +++ b/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/io/MovingFunction.java @@ -16,6 +16,7 @@ package com.google.cloud.dataflow.sdk.io; +import com.google.cloud.dataflow.sdk.transforms.Combine; import com.google.common.base.Preconditions; import java.util.Arrays; @@ -48,7 +49,7 @@ class MovingFunction { /** * Function for combining sample values. */ - private final SimpleFunction function; + private final Combine.BinaryCombineLongFn function; /** * Minimum/maximum/sum of all values per bucket. @@ -72,7 +73,7 @@ class MovingFunction { public MovingFunction(long samplePeriodMs, long sampleUpdateMs, int numSignificantBuckets, int numSignificantSamples, - SimpleFunction function) { + Combine.BinaryCombineLongFn function) { this.samplePeriodMs = samplePeriodMs; this.sampleUpdateMs = sampleUpdateMs; this.numSignificantBuckets = numSignificantBuckets; @@ -80,7 +81,7 @@ public MovingFunction(long samplePeriodMs, long sampleUpdateMs, this.function = function; int n = (int) (samplePeriodMs / sampleUpdateMs); buckets = new long[n]; - Arrays.fill(buckets, function.zero()); + Arrays.fill(buckets, function.identity()); numSamples = new int[n]; Arrays.fill(numSamples, 0); currentMsSinceEpoch = -1; @@ -102,7 +103,7 @@ private void flush(long nowMsSinceEpoch) { buckets.length); while (newBuckets > 0) { currentIndex = (currentIndex + 1) % buckets.length; - buckets[currentIndex] = function.zero(); + buckets[currentIndex] = function.identity(); numSamples[currentIndex] = 0; newBuckets--; currentMsSinceEpoch += sampleUpdateMs; @@ -114,7 +115,7 @@ private void flush(long nowMsSinceEpoch) { */ public void add(long nowMsSinceEpoch, long value) { flush(nowMsSinceEpoch); - buckets[currentIndex] = function.f(buckets[currentIndex], value); + buckets[currentIndex] = function.apply(buckets[currentIndex], value); numSamples[currentIndex]++; } @@ -124,9 +125,9 @@ public void add(long nowMsSinceEpoch, long value) { */ public long get(long nowMsSinceEpoch) { flush(nowMsSinceEpoch); - long result = function.zero(); + long result = function.identity(); for (int i = 0; i < buckets.length; i++) { - result = function.f(result, buckets[i]); + result = function.apply(result, buckets[i]); } return result; } 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..2fdd70b8d473 --- /dev/null +++ b/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/io/PubsubClient.java @@ -0,0 +1,174 @@ +/* + * 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.sdk.io; + +import java.io.IOException; +import java.util.Collection; + +/** + * A helper interface for talking to pub/sub via an underlying transport. + */ +public interface PubsubClient extends AutoCloseable { + /** + * Gracefully close the underlying transport. + */ + @Override + void close(); + + /** + * A message to be sent to pub/sub. + */ + 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 pub/sub. + */ + class IncomingMessage { + /** + * Underlying (encoded) element. + */ + public final byte[] elementBytes; + + /** + * Timestamp for element (ms since epoch). Either pub/sub'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 pub/sub 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 pub/sub {@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 index f5bec9355364..05807e35d3de 100644 --- 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 @@ -64,7 +64,7 @@ /** * A helper class for talking to pub/sub via grpc. */ -public class PubsubGrpcClient implements AutoCloseable { +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 = @@ -185,76 +185,7 @@ private SubscriberGrpc.SubscriberBlockingStub subscriberStub() throws IOExceptio return cachedSubscriberStub.withDeadlineAfter(TIMEOUT_S, TimeUnit.SECONDS); } - /** - * A message to be sent to pub/sub. - */ - public static 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 pub/sub. - */ - public static class IncomingMessage { - /** - * Underlying (encoded) element. - */ - public final byte[] elementBytes; - - /** - * Timestamp for element (ms since epoch). Either pub/sub'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 pub/sub 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 pub/sub {@code topic}. Return number of messages - * published. - * - * @throws IOException - */ + @Override public int publish(String topic, Iterable outgoingMessages) throws IOException { PublishRequest.Builder request = PublishRequest.newBuilder() .setTopic(topic); @@ -281,11 +212,7 @@ public int publish(String topic, Iterable outgoingMessages) thr return response.getMessageIdsCount(); } - /** - * Request the next batch of up to {@code batchSize} messages from {@code subscription}. - * - * @throws IOException - */ + @Override public Collection pull( long requestTimeMsSinceEpoch, String subscription, @@ -356,11 +283,7 @@ public Collection pull( return incomingMessages; } - /** - * Acknowldege messages from {@code subscription} with {@code ackIds}. - * - * @throws IOException - */ + @Override public void acknowledge(String subscription, Iterable ackIds) throws IOException { AcknowledgeRequest request = AcknowledgeRequest.newBuilder() .setSubscription(subscription) @@ -369,11 +292,7 @@ public void acknowledge(String subscription, Iterable ackIds) throws IOE subscriberStub().acknowledge(request); // ignore Empty result. } - /** - * Modify the ack deadline for messages from {@code subscription} with {@code ackIds}. - * - * @throws IOException - */ + @Override public void modifyAckDeadline(String subscription, Iterable ackIds, int deadlineSeconds) throws IOException { ModifyAckDeadlineRequest request = @@ -385,10 +304,7 @@ public void modifyAckDeadline(String subscription, Iterable ackIds, int subscriberStub().modifyAckDeadline(request); // ignore Empty result. } - - /** - * Create {@code topic}. - */ + @Override public void createTopic(String topic) throws IOException { Topic request = Topic.newBuilder() .setName(topic) @@ -396,17 +312,13 @@ public void createTopic(String topic) throws IOException { publisherStub().createTopic(request); // ignore Topic result. } - /* - * Delete {@code topic}. - */ + @Override public void deleteTopic(String topic) throws IOException { DeleteTopicRequest request = DeleteTopicRequest.newBuilder().setTopic(topic).build(); publisherStub().deleteTopic(request); // ignore Empty result. } - /** - * Return a list of topics for {@code project}. - */ + @Override public Collection listTopics(String project) throws IOException { ListTopicsRequest.Builder request = ListTopicsRequest.newBuilder() @@ -430,11 +342,7 @@ public Collection listTopics(String project) throws IOException { return topics; } - /** - * Create {@code subscription} to {@code topic}. - * - * @throws IOException - */ + @Override public void createSubscription(String topic, String subscription, int ackDeadlineSeconds) throws IOException { Subscription request = Subscription.newBuilder() @@ -445,18 +353,14 @@ public void createSubscription(String topic, String subscription, int ackDeadlin subscriberStub().createSubscription(request); // ignore Subscription result. } - /** - * Delete {@code subscription}. - */ + @Override public void deleteSubscription(String subscription) throws IOException { DeleteSubscriptionRequest request = DeleteSubscriptionRequest.newBuilder().setSubscription(subscription).build(); subscriberStub().deleteSubscription(request); // ignore Empty result. } - /** - * Return a list of subscriptions for {@code topic} in {@code project}. - */ + @Override public Collection listSubscriptions(String project, String topic) throws IOException { ListSubscriptionsRequest.Builder request = ListSubscriptionsRequest.newBuilder() 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 index b1d7602899c3..b83b367ee81b 100644 --- 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 @@ -105,22 +105,22 @@ public class PubsubUnboundedSink extends PTransform, PDone> { */ private static final int SCALE_OUT = 4; - public static final Coder CODER = new - AtomicCoder() { + public static final Coder CODER = new + AtomicCoder() { @Override public void encode( - PubsubGrpcClient.OutgoingMessage value, OutputStream outStream, Context context) + 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 PubsubGrpcClient.OutgoingMessage decode( + 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 PubsubGrpcClient.OutgoingMessage(elementBytes, timestampMsSinceEpoch); + return new PubsubClient.OutgoingMessage(elementBytes, timestampMsSinceEpoch); } }; @@ -131,7 +131,7 @@ public PubsubGrpcClient.OutgoingMessage decode( /** * Convert elements to messages and shard them. */ - private class ShardFn extends DoFn> { + private class ShardFn extends DoFn> { /** * Number of cores available for publishing. */ @@ -150,7 +150,7 @@ public void processElement(ProcessContext c) throws Exception { byte[] elementBytes = CoderUtils.encodeToByteArray(elementCoder, c.element()); long timestampMsSinceEpoch = c.timestamp().getMillis(); c.output(KV.of(ThreadLocalRandom.current().nextInt(numCores * SCALE_OUT), - new PubsubGrpcClient.OutgoingMessage(elementBytes, timestampMsSinceEpoch))); + new PubsubClient.OutgoingMessage(elementBytes, timestampMsSinceEpoch))); } } @@ -162,13 +162,13 @@ public void processElement(ProcessContext c) throws Exception { * Publish messages to pub/sub in batches. */ private class WriterFn - extends DoFn>, Void> { + extends DoFn>, Void> { /** * Client on which to talk to pub/sub. Null until created by {@link #startBundle}. */ @Nullable - private transient PubsubGrpcClient pubsubClient; + private transient PubsubClient pubsubClient; private final Aggregator batchCounter = createAggregator("batches", new Sum.SumLongFn()); @@ -181,7 +181,7 @@ private class WriterFn * BLOCKING * Send {@code messages} as a batch to pub/sub. */ - private void publishBatch(List messages, int bytes) + private void publishBatch(List messages, int bytes) throws IOException { long nowMsSinceEpoch = System.currentTimeMillis(); int n = pubsubClient.publish(topic, messages); @@ -201,9 +201,9 @@ public void startBundle(Context c) throws Exception { @Override public void processElement(ProcessContext c) throws Exception { - List pubsubMessages = new ArrayList<>(PUBLISH_BATCH_SIZE); + List pubsubMessages = new ArrayList<>(PUBLISH_BATCH_SIZE); int bytes = 0; - for (PubsubGrpcClient.OutgoingMessage message : c.element().getValue()) { + for (PubsubClient.OutgoingMessage message : c.element().getValue()) { if (!pubsubMessages.isEmpty() && bytes + message.elementBytes.length > PUBLISH_BATCH_BYTES) { // Break large (in bytes) batches into smaller. @@ -294,7 +294,7 @@ public PDone apply(PCollection input) { .withAllowedLateness(Duration.ZERO)) .apply(ParDo.named(label + ".Shard").of(new ShardFn(numCores))) .setCoder(KvCoder.of(VarIntCoder.of(), CODER)) - .apply(GroupByKey.create()) + .apply(GroupByKey.create()) .apply(ParDo.named(label + ".Writer").of(new WriterFn())); return PDone.in(input.getPipeline()); } 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 index f819ea46ab2d..4da0f9bb614b 100644 --- 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 @@ -25,6 +25,7 @@ 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; @@ -152,6 +153,46 @@ public class PubsubUnboundedSource extends PTransform> */ 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 // ================================================================================ @@ -308,7 +349,7 @@ private static class Reader extends UnboundedSource.UnboundedReader { * Messages we have received from pub/sub and not yet delivered downstream. * We preserve their order. */ - private final Queue notYetRead; + private final Queue notYetRead; /** * Map from ack ids of messages we have received from pub/sub but not yet acked to their @@ -355,7 +396,7 @@ private static class Reader extends UnboundedSource.UnboundedReader { * The current message, or {@literal null} if none. */ @Nullable - private PubsubGrpcClient.IncomingMessage current; + private PubsubClient.IncomingMessage current; /** * Stats only: System time (ms since epoch) we last logs stats, or -1 if never. @@ -438,7 +479,7 @@ private static class Reader extends UnboundedSource.UnboundedReader { */ private int maxInFlightCheckpoints; - private static MovingFunction newFun(SimpleFunction function) { + private static MovingFunction newFun(Combine.BinaryCombineLongFn function) { return new MovingFunction(SAMPLE_PERIOD.getMillis(), SAMPLE_UPDATE.getMillis(), MIN_WATERMARK_SPREAD, @@ -463,24 +504,24 @@ public Reader(GcpOptions options, Source outer) throws IOException, minUnreadTimestampMsSinceEpoch = new BucketingFunction(SAMPLE_UPDATE.getMillis(), MIN_WATERMARK_SPREAD, MIN_WATERMARK_MESSAGES, - SimpleFunction.MIN); - minReadTimestampMsSinceEpoch = newFun(SimpleFunction.MIN); + MIN); + minReadTimestampMsSinceEpoch = newFun(MIN); lastReceivedMsSinceEpoch = -1; lastWatermarkMsSinceEpoch = BoundedWindow.TIMESTAMP_MIN_VALUE.getMillis(); current = null; lastLogTimestampMsSinceEpoch = -1; numReceived = 0L; - numReceivedRecently = newFun(SimpleFunction.SUM); - numExtendedDeadlines = newFun(SimpleFunction.SUM); - numLateDeadlines = newFun(SimpleFunction.SUM); - numAcked = newFun(SimpleFunction.SUM); - numNacked = newFun(SimpleFunction.SUM); - numReadBytes = newFun(SimpleFunction.SUM); - minReceivedTimestampMsSinceEpoch = newFun(SimpleFunction.MIN); - maxReceivedTimestampMsSinceEpoch = newFun(SimpleFunction.MAX); - minWatermarkMsSinceEpoch = newFun(SimpleFunction.MIN); - maxWatermarkMsSinceEpoch = newFun(SimpleFunction.MAX); - numLateMessages = newFun(SimpleFunction.SUM); + 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; } @@ -598,7 +639,7 @@ private void pull() throws IOException { // Pull the next batch. // BLOCKs until received. - Collection receivedMessages = + Collection receivedMessages = pubsubClient.pull(requestTimeMsSinceEpoch, outer.outer.subscription, PULL_BATCH_SIZE); @@ -610,7 +651,7 @@ private void pull() throws IOException { lastReceivedMsSinceEpoch = requestTimeMsSinceEpoch; // Capture the received messages. - for (PubsubGrpcClient.IncomingMessage incomingMessage : receivedMessages) { + for (PubsubClient.IncomingMessage incomingMessage : receivedMessages) { notYetRead.add(incomingMessage); notYetReadBytes += incomingMessage.elementBytes.length; inFlight.put(incomingMessage.ackId, new Instant(deadlineMsSinceEpoch)); @@ -829,7 +870,7 @@ public UnboundedSource.CheckpointMark getCheckpointMark() { List snapshotSafeToAckIds = safeToAckIds; safeToAckIds = new ArrayList<>(); List snapshotNotYetReadIds = new ArrayList<>(notYetRead.size()); - for (PubsubGrpcClient.IncomingMessage incomingMessage : notYetRead) { + for (PubsubClient.IncomingMessage incomingMessage : notYetRead) { snapshotNotYetReadIds.add(incomingMessage.ackId); } return new Checkpoint<>(this, snapshotSafeToAckIds, snapshotNotYetReadIds); diff --git a/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/io/SimpleFunction.java b/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/io/SimpleFunction.java deleted file mode 100644 index 4b735d14192a..000000000000 --- a/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/io/SimpleFunction.java +++ /dev/null @@ -1,62 +0,0 @@ -/* - * 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.sdk.io; - -/** - * Some simple functions (commutative and associative with a zero) - * usable with {@link MovingFunction} and {@link BucketingFunction}. - */ -public enum SimpleFunction { - MIN { - @Override - public long f(long l, long r) { - return Math.min(l, r); - } - - @Override - public long zero() { - return Long.MAX_VALUE; - } - }, - MAX { - @Override - public long f(long l, long r) { - return Math.max(l, r); - } - - @Override - public long zero() { - return Long.MIN_VALUE; - } - }, - SUM { - @Override - public long f(long l, long r) { - return l + r; - } - - @Override - public long zero() { - return 0; - } - }; - - public abstract long f(long l, long r); - - public abstract long zero(); -} - diff --git a/sdks/java/core/src/test/java/com/google/cloud/dataflow/sdk/io/BucketingFunctionTest.java b/sdks/java/core/src/test/java/com/google/cloud/dataflow/sdk/io/BucketingFunctionTest.java index 0e43739c2b30..f5e5e8d5ac18 100644 --- a/sdks/java/core/src/test/java/com/google/cloud/dataflow/sdk/io/BucketingFunctionTest.java +++ b/sdks/java/core/src/test/java/com/google/cloud/dataflow/sdk/io/BucketingFunctionTest.java @@ -14,6 +14,7 @@ package com.google.cloud.dataflow.sdk.io; +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; @@ -31,10 +32,23 @@ public class BucketingFunctionTest { 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, SimpleFunction.SUM); + SIGNIFICANT_SAMPLES, SUM); } @Test diff --git a/sdks/java/core/src/test/java/com/google/cloud/dataflow/sdk/io/MovingFunctionTest.java b/sdks/java/core/src/test/java/com/google/cloud/dataflow/sdk/io/MovingFunctionTest.java index 11e5edddc98e..6bcb2ec90dd3 100644 --- a/sdks/java/core/src/test/java/com/google/cloud/dataflow/sdk/io/MovingFunctionTest.java +++ b/sdks/java/core/src/test/java/com/google/cloud/dataflow/sdk/io/MovingFunctionTest.java @@ -14,6 +14,7 @@ package com.google.cloud.dataflow.sdk.io; +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; @@ -32,10 +33,23 @@ public class MovingFunctionTest { 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, SimpleFunction.SUM); + SIGNIFICANT_SAMPLES, SUM); } From 9b71aa0df8af9f109da0828268f42214b56ec2f5 Mon Sep 17 00:00:00 2001 From: Mark Shields Date: Thu, 31 Mar 2016 15:24:16 -0700 Subject: [PATCH 09/14] Copyright message busywork. --- .../dataflow/sdk/io/BucketingFunction.java | 22 ++++++++++--------- .../cloud/dataflow/sdk/io/MovingFunction.java | 22 ++++++++++--------- .../cloud/dataflow/sdk/io/PubsubClient.java | 22 ++++++++++--------- .../dataflow/sdk/io/PubsubGrpcClient.java | 22 ++++++++++--------- .../dataflow/sdk/io/PubsubUnboundedSink.java | 22 ++++++++++--------- .../sdk/io/PubsubUnboundedSource.java | 22 ++++++++++--------- .../sdk/io/BucketingFunctionTest.java | 22 +++++++++++-------- .../dataflow/sdk/io/MovingFunctionTest.java | 22 +++++++++++-------- 8 files changed, 98 insertions(+), 78 deletions(-) diff --git a/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/io/BucketingFunction.java b/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/io/BucketingFunction.java index a74564fc1ae7..52cc53770cb2 100644 --- a/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/io/BucketingFunction.java +++ b/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/io/BucketingFunction.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.sdk.io; diff --git a/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/io/MovingFunction.java b/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/io/MovingFunction.java index d0fa67aaa699..80fe5d24b623 100644 --- a/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/io/MovingFunction.java +++ b/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/io/MovingFunction.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.sdk.io; 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 index 2fdd70b8d473..4c79cb8ade7c 100644 --- 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 @@ -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.sdk.io; 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 index 05807e35d3de..ce947bf7dfd7 100644 --- 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 @@ -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.sdk.io; 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 index b83b367ee81b..e90d7bbc9903 100644 --- 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 @@ -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.sdk.io; 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 index 4da0f9bb614b..a72f1698f9a5 100644 --- 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 @@ -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.sdk.io; diff --git a/sdks/java/core/src/test/java/com/google/cloud/dataflow/sdk/io/BucketingFunctionTest.java b/sdks/java/core/src/test/java/com/google/cloud/dataflow/sdk/io/BucketingFunctionTest.java index f5e5e8d5ac18..62ff64705e0d 100644 --- a/sdks/java/core/src/test/java/com/google/cloud/dataflow/sdk/io/BucketingFunctionTest.java +++ b/sdks/java/core/src/test/java/com/google/cloud/dataflow/sdk/io/BucketingFunctionTest.java @@ -1,15 +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. + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES 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; diff --git a/sdks/java/core/src/test/java/com/google/cloud/dataflow/sdk/io/MovingFunctionTest.java b/sdks/java/core/src/test/java/com/google/cloud/dataflow/sdk/io/MovingFunctionTest.java index 6bcb2ec90dd3..7775c53d801e 100644 --- a/sdks/java/core/src/test/java/com/google/cloud/dataflow/sdk/io/MovingFunctionTest.java +++ b/sdks/java/core/src/test/java/com/google/cloud/dataflow/sdk/io/MovingFunctionTest.java @@ -1,15 +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. + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES 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; From 0befb344bd7451e4fc0e3caebb6f20a58a573cb1 Mon Sep 17 00:00:00 2001 From: Mark Shields Date: Thu, 31 Mar 2016 15:30:01 -0700 Subject: [PATCH 10/14] Make util functions package only. --- .../dataflow/sdk/io/BucketingFunction.java | 20 +++++++++---------- .../cloud/dataflow/sdk/io/MovingFunction.java | 8 ++++---- 2 files changed, 14 insertions(+), 14 deletions(-) diff --git a/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/io/BucketingFunction.java b/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/io/BucketingFunction.java index 52cc53770cb2..fd660bd5afe0 100644 --- a/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/io/BucketingFunction.java +++ b/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/io/BucketingFunction.java @@ -27,28 +27,28 @@ * Keep track of the minimum/maximum/sum of a set of timestamped long values. * For efficiency, bucket values by their timestamp. */ -public class BucketingFunction { +class BucketingFunction { private static class Bucket { private int numSamples; private long combinedValue; - public Bucket(BucketingFunction outer) { + Bucket(BucketingFunction outer) { numSamples = 0; combinedValue = outer.function.identity(); } - public void add(BucketingFunction outer, long value) { + void add(BucketingFunction outer, long value) { combinedValue = outer.function.apply(combinedValue, value); numSamples++; } - public boolean remove() { + boolean remove() { numSamples--; Preconditions.checkState(numSamples >= 0); return numSamples == 0; } - public long get() { + long get() { return combinedValue; } } @@ -78,7 +78,7 @@ public long get() { */ private final Map buckets; - public BucketingFunction( + BucketingFunction( long bucketWidthMs, int numSignificantBuckets, int numSignificantSamples, @@ -100,7 +100,7 @@ private long key(long timeMsSinceEpoch) { /** * Add one sample of {@code value} (to bucket) at {@code timeMsSinceEpoch}. */ - public void add(long timeMsSinceEpoch, long value) { + void add(long timeMsSinceEpoch, long value) { long key = key(timeMsSinceEpoch); Bucket bucket = buckets.get(key); if (bucket == null) { @@ -113,7 +113,7 @@ public void add(long timeMsSinceEpoch, long value) { /** * Remove one sample (from bucket) at {@code timeMsSinceEpoch}. */ - public void remove(long timeMsSinceEpoch) { + void remove(long timeMsSinceEpoch) { long key = key(timeMsSinceEpoch); Bucket bucket = buckets.get(key); if (bucket == null) { @@ -127,7 +127,7 @@ public void remove(long timeMsSinceEpoch) { /** * Return the (bucketized) combined value of all samples. */ - public long get() { + long get() { long result = function.identity(); for (Bucket bucket : buckets.values()) { result = function.apply(result, bucket.get()); @@ -139,7 +139,7 @@ public long get() { * Is the current result 'significant'? Ie is it drawn from enough buckets * or from enough samples? */ - public boolean isSignificant() { + boolean isSignificant() { if (buckets.size() >= numSignificantBuckets) { return true; } diff --git a/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/io/MovingFunction.java b/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/io/MovingFunction.java index 80fe5d24b623..922112428447 100644 --- a/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/io/MovingFunction.java +++ b/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/io/MovingFunction.java @@ -73,7 +73,7 @@ class MovingFunction { */ private int currentIndex; - public MovingFunction(long samplePeriodMs, long sampleUpdateMs, + MovingFunction(long samplePeriodMs, long sampleUpdateMs, int numSignificantBuckets, int numSignificantSamples, Combine.BinaryCombineLongFn function) { this.samplePeriodMs = samplePeriodMs; @@ -115,7 +115,7 @@ private void flush(long nowMsSinceEpoch) { /** * Add {@code value} at {@code nowMsSinceEpoch}. */ - public void add(long nowMsSinceEpoch, long value) { + void add(long nowMsSinceEpoch, long value) { flush(nowMsSinceEpoch); buckets[currentIndex] = function.apply(buckets[currentIndex], value); numSamples[currentIndex]++; @@ -125,7 +125,7 @@ public void add(long nowMsSinceEpoch, long value) { * Return the minimum/maximum/sum of all retained values within {@link #samplePeriodMs} * of {@code nowMsSinceEpoch}. */ - public long get(long nowMsSinceEpoch) { + long get(long nowMsSinceEpoch) { flush(nowMsSinceEpoch); long result = function.identity(); for (int i = 0; i < buckets.length; i++) { @@ -138,7 +138,7 @@ public long get(long nowMsSinceEpoch) { * Is the current result 'significant'? Ie is it drawn from enough buckets * or from enough samples? */ - public boolean isSignificant() { + boolean isSignificant() { int totalSamples = 0; int activeBuckets = 0; for (int i = 0; i < buckets.length; i++) { From 238b43acda8d40e1704a9cb01fc6fbc166321c00 Mon Sep 17 00:00:00 2001 From: Mark Shields Date: Thu, 31 Mar 2016 15:35:15 -0700 Subject: [PATCH 11/14] Better idea: move them to util... --- .../google/cloud/dataflow/sdk/io/PubsubUnboundedSource.java | 2 ++ .../cloud/dataflow/sdk/{io => util}/BucketingFunction.java | 2 +- .../google/cloud/dataflow/sdk/{io => util}/MovingFunction.java | 2 +- .../cloud/dataflow/sdk/{io => util}/BucketingFunctionTest.java | 3 ++- .../cloud/dataflow/sdk/{io => util}/MovingFunctionTest.java | 3 ++- 5 files changed, 8 insertions(+), 4 deletions(-) rename sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/{io => util}/BucketingFunction.java (98%) rename sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/{io => util}/MovingFunction.java (99%) rename sdks/java/core/src/test/java/com/google/cloud/dataflow/sdk/{io => util}/BucketingFunctionTest.java (96%) rename sdks/java/core/src/test/java/com/google/cloud/dataflow/sdk/{io => util}/MovingFunctionTest.java (96%) 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 index a72f1698f9a5..52fec011b7ab 100644 --- 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 @@ -33,7 +33,9 @@ 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; diff --git a/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/io/BucketingFunction.java b/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/util/BucketingFunction.java similarity index 98% rename from sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/io/BucketingFunction.java rename to sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/util/BucketingFunction.java index fd660bd5afe0..9472cdef4fd4 100644 --- a/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/io/BucketingFunction.java +++ b/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/util/BucketingFunction.java @@ -16,7 +16,7 @@ * limitations under the License. */ -package com.google.cloud.dataflow.sdk.io; +package com.google.cloud.dataflow.sdk.util; import com.google.cloud.dataflow.sdk.transforms.Combine; import com.google.common.base.Preconditions; diff --git a/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/io/MovingFunction.java b/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/util/MovingFunction.java similarity index 99% rename from sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/io/MovingFunction.java rename to sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/util/MovingFunction.java index 922112428447..713bfa124aea 100644 --- a/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/io/MovingFunction.java +++ b/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/util/MovingFunction.java @@ -16,7 +16,7 @@ * limitations under the License. */ -package com.google.cloud.dataflow.sdk.io; +package com.google.cloud.dataflow.sdk.util; import com.google.cloud.dataflow.sdk.transforms.Combine; import com.google.common.base.Preconditions; diff --git a/sdks/java/core/src/test/java/com/google/cloud/dataflow/sdk/io/BucketingFunctionTest.java b/sdks/java/core/src/test/java/com/google/cloud/dataflow/sdk/util/BucketingFunctionTest.java similarity index 96% rename from sdks/java/core/src/test/java/com/google/cloud/dataflow/sdk/io/BucketingFunctionTest.java rename to sdks/java/core/src/test/java/com/google/cloud/dataflow/sdk/util/BucketingFunctionTest.java index 62ff64705e0d..4d8b26ced837 100644 --- a/sdks/java/core/src/test/java/com/google/cloud/dataflow/sdk/io/BucketingFunctionTest.java +++ b/sdks/java/core/src/test/java/com/google/cloud/dataflow/sdk/util/BucketingFunctionTest.java @@ -16,9 +16,10 @@ * limitations under the License. */ -package com.google.cloud.dataflow.sdk.io; +package com.google.cloud.dataflow.sdk.util; import com.google.cloud.dataflow.sdk.transforms.Combine; +import com.google.cloud.dataflow.sdk.util.BucketingFunction; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertFalse; import static org.junit.Assert.assertTrue; diff --git a/sdks/java/core/src/test/java/com/google/cloud/dataflow/sdk/io/MovingFunctionTest.java b/sdks/java/core/src/test/java/com/google/cloud/dataflow/sdk/util/MovingFunctionTest.java similarity index 96% rename from sdks/java/core/src/test/java/com/google/cloud/dataflow/sdk/io/MovingFunctionTest.java rename to sdks/java/core/src/test/java/com/google/cloud/dataflow/sdk/util/MovingFunctionTest.java index 7775c53d801e..3451c3c8d1cf 100644 --- a/sdks/java/core/src/test/java/com/google/cloud/dataflow/sdk/io/MovingFunctionTest.java +++ b/sdks/java/core/src/test/java/com/google/cloud/dataflow/sdk/util/MovingFunctionTest.java @@ -16,9 +16,10 @@ * limitations under the License. */ -package com.google.cloud.dataflow.sdk.io; +package com.google.cloud.dataflow.sdk.util; import com.google.cloud.dataflow.sdk.transforms.Combine; +import com.google.cloud.dataflow.sdk.util.MovingFunction; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertFalse; import static org.junit.Assert.assertTrue; From 96df83fc2024ddafee04a89669c5d8de42efb29c Mon Sep 17 00:00:00 2001 From: Mark Shields Date: Thu, 31 Mar 2016 15:36:12 -0700 Subject: [PATCH 12/14] Revert "Make util functions package only." This reverts commit 0befb344bd7451e4fc0e3caebb6f20a58a573cb1. --- .../dataflow/sdk/util/BucketingFunction.java | 20 +++++++++---------- .../dataflow/sdk/util/MovingFunction.java | 8 ++++---- 2 files changed, 14 insertions(+), 14 deletions(-) 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 index 9472cdef4fd4..56bec8db6fdb 100644 --- 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 @@ -27,28 +27,28 @@ * Keep track of the minimum/maximum/sum of a set of timestamped long values. * For efficiency, bucket values by their timestamp. */ -class BucketingFunction { +public class BucketingFunction { private static class Bucket { private int numSamples; private long combinedValue; - Bucket(BucketingFunction outer) { + public Bucket(BucketingFunction outer) { numSamples = 0; combinedValue = outer.function.identity(); } - void add(BucketingFunction outer, long value) { + public void add(BucketingFunction outer, long value) { combinedValue = outer.function.apply(combinedValue, value); numSamples++; } - boolean remove() { + public boolean remove() { numSamples--; Preconditions.checkState(numSamples >= 0); return numSamples == 0; } - long get() { + public long get() { return combinedValue; } } @@ -78,7 +78,7 @@ long get() { */ private final Map buckets; - BucketingFunction( + public BucketingFunction( long bucketWidthMs, int numSignificantBuckets, int numSignificantSamples, @@ -100,7 +100,7 @@ private long key(long timeMsSinceEpoch) { /** * Add one sample of {@code value} (to bucket) at {@code timeMsSinceEpoch}. */ - void add(long timeMsSinceEpoch, long value) { + public void add(long timeMsSinceEpoch, long value) { long key = key(timeMsSinceEpoch); Bucket bucket = buckets.get(key); if (bucket == null) { @@ -113,7 +113,7 @@ void add(long timeMsSinceEpoch, long value) { /** * Remove one sample (from bucket) at {@code timeMsSinceEpoch}. */ - void remove(long timeMsSinceEpoch) { + public void remove(long timeMsSinceEpoch) { long key = key(timeMsSinceEpoch); Bucket bucket = buckets.get(key); if (bucket == null) { @@ -127,7 +127,7 @@ void remove(long timeMsSinceEpoch) { /** * Return the (bucketized) combined value of all samples. */ - long get() { + public long get() { long result = function.identity(); for (Bucket bucket : buckets.values()) { result = function.apply(result, bucket.get()); @@ -139,7 +139,7 @@ long get() { * Is the current result 'significant'? Ie is it drawn from enough buckets * or from enough samples? */ - boolean isSignificant() { + public boolean isSignificant() { if (buckets.size() >= numSignificantBuckets) { return true; } 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 index 713bfa124aea..620f42ea47aa 100644 --- 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 @@ -73,7 +73,7 @@ class MovingFunction { */ private int currentIndex; - MovingFunction(long samplePeriodMs, long sampleUpdateMs, + public MovingFunction(long samplePeriodMs, long sampleUpdateMs, int numSignificantBuckets, int numSignificantSamples, Combine.BinaryCombineLongFn function) { this.samplePeriodMs = samplePeriodMs; @@ -115,7 +115,7 @@ private void flush(long nowMsSinceEpoch) { /** * Add {@code value} at {@code nowMsSinceEpoch}. */ - void add(long nowMsSinceEpoch, long value) { + public void add(long nowMsSinceEpoch, long value) { flush(nowMsSinceEpoch); buckets[currentIndex] = function.apply(buckets[currentIndex], value); numSamples[currentIndex]++; @@ -125,7 +125,7 @@ void add(long nowMsSinceEpoch, long value) { * Return the minimum/maximum/sum of all retained values within {@link #samplePeriodMs} * of {@code nowMsSinceEpoch}. */ - long get(long nowMsSinceEpoch) { + public long get(long nowMsSinceEpoch) { flush(nowMsSinceEpoch); long result = function.identity(); for (int i = 0; i < buckets.length; i++) { @@ -138,7 +138,7 @@ long get(long nowMsSinceEpoch) { * Is the current result 'significant'? Ie is it drawn from enough buckets * or from enough samples? */ - boolean isSignificant() { + public boolean isSignificant() { int totalSamples = 0; int activeBuckets = 0; for (int i = 0; i < buckets.length; i++) { From f68edac1de1d0666e30a1aa73e18c932887164f1 Mon Sep 17 00:00:00 2001 From: Mark Shields Date: Thu, 31 Mar 2016 15:38:35 -0700 Subject: [PATCH 13/14] really --- .../java/com/google/cloud/dataflow/sdk/util/MovingFunction.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) 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 index 620f42ea47aa..22827e62512a 100644 --- 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 @@ -27,7 +27,7 @@ * is over at most the last {@link #samplePeriodMs}, and is updated every * {@link #sampleUpdateMs}. */ -class MovingFunction { +public class MovingFunction { /** * How far back to retain samples, in ms. */ From 3f3057e87a0782233f84811ce10e4a2f63c63121 Mon Sep 17 00:00:00 2001 From: Mark Shields Date: Thu, 31 Mar 2016 15:47:16 -0700 Subject: [PATCH 14/14] Fix import --- .../google/cloud/dataflow/sdk/util/BucketingFunctionTest.java | 1 - .../com/google/cloud/dataflow/sdk/util/MovingFunctionTest.java | 1 - 2 files changed, 2 deletions(-) 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 index 4d8b26ced837..f5e52e059ec1 100644 --- 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 @@ -19,7 +19,6 @@ package com.google.cloud.dataflow.sdk.util; import com.google.cloud.dataflow.sdk.transforms.Combine; -import com.google.cloud.dataflow.sdk.util.BucketingFunction; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertFalse; import static org.junit.Assert.assertTrue; 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 index 3451c3c8d1cf..2ca8b7cc7c6d 100644 --- 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 @@ -19,7 +19,6 @@ package com.google.cloud.dataflow.sdk.util; import com.google.cloud.dataflow.sdk.transforms.Combine; -import com.google.cloud.dataflow.sdk.util.MovingFunction; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertFalse; import static org.junit.Assert.assertTrue;