|
| 1 | +/* |
| 2 | + * Copyright (C) 2016 Google Inc. |
| 3 | + * |
| 4 | + * Licensed under the Apache License, Version 2.0 (the "License"); you may not |
| 5 | + * use this file except in compliance with the License. You may obtain a copy of |
| 6 | + * the License at |
| 7 | + * |
| 8 | + * http://www.apache.org/licenses/LICENSE-2.0 |
| 9 | + * |
| 10 | + * Unless required by applicable law or agreed to in writing, software |
| 11 | + * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT |
| 12 | + * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the |
| 13 | + * License for the specific language governing permissions and limitations under |
| 14 | + * the License. |
| 15 | + */ |
| 16 | + |
| 17 | +package com.google.cloud.dataflow.sdk.io; |
| 18 | + |
| 19 | +import static com.google.common.base.Preconditions.checkArgument; |
| 20 | +import static com.google.common.base.Preconditions.checkNotNull; |
| 21 | + |
| 22 | +import com.google.cloud.dataflow.sdk.io.CountingSource.NowTimestampFn; |
| 23 | +import com.google.cloud.dataflow.sdk.io.Read.Unbounded; |
| 24 | +import com.google.cloud.dataflow.sdk.transforms.PTransform; |
| 25 | +import com.google.cloud.dataflow.sdk.transforms.SerializableFunction; |
| 26 | +import com.google.cloud.dataflow.sdk.values.PBegin; |
| 27 | +import com.google.cloud.dataflow.sdk.values.PCollection; |
| 28 | +import com.google.cloud.dataflow.sdk.values.PCollection.IsBounded; |
| 29 | +import com.google.common.base.Optional; |
| 30 | + |
| 31 | +import org.joda.time.Duration; |
| 32 | +import org.joda.time.Instant; |
| 33 | + |
| 34 | +/** |
| 35 | + * A {@link PTransform} that produces longs. When used to produce a |
| 36 | + * {@link IsBounded#BOUNDED bounded} {@link PCollection}, {@link CountingInput} starts at {@code 0} |
| 37 | + * and counts up to a specified maximum. When used to produce an |
| 38 | + * {@link IsBounded#UNBOUNDED unbounded} {@link PCollection}, it counts up to {@link Long#MAX_VALUE} |
| 39 | + * and then never produces more output. (In practice, this limit should never be reached.) |
| 40 | + * |
| 41 | + * <p>The bounded {@link CountingInput} is implemented based on {@link OffsetBasedSource} and |
| 42 | + * {@link OffsetBasedSource.OffsetBasedReader}, so it performs efficient initial splitting and it |
| 43 | + * supports dynamic work rebalancing. |
| 44 | + * |
| 45 | + * <p>To produce a bounded {@code PCollection<Long>}, use {@link CountingInput#upTo(long)}: |
| 46 | + * |
| 47 | + * <pre>{@code |
| 48 | + * Pipeline p = ... |
| 49 | + * PTransform<PBegin, PCollection<Long>> producer = CountingInput.upTo(1000); |
| 50 | + * PCollection<Long> bounded = p.apply(producer); |
| 51 | + * }</pre> |
| 52 | + * |
| 53 | + * <p>To produce an unbounded {@code PCollection<Long>}, use {@link CountingInput#unbounded()}, |
| 54 | + * calling {@link UnboundedCountingInput#withTimestampFn(SerializableFunction)} to provide values |
| 55 | + * with timestamps other than {@link Instant#now}. |
| 56 | + * |
| 57 | + * <pre>{@code |
| 58 | + * Pipeline p = ... |
| 59 | + * |
| 60 | + * // To create an unbounded producer that uses processing time as the element timestamp. |
| 61 | + * PCollection<Long> unbounded = p.apply(CountingInput.unbounded()); |
| 62 | + * // Or, to create an unbounded source that uses a provided function to set the element timestamp. |
| 63 | + * PCollection<Long> unboundedWithTimestamps = |
| 64 | + * p.apply(CountingInput.unbounded().withTimestampFn(someFn)); |
| 65 | + * }</pre> |
| 66 | + */ |
| 67 | +public class CountingInput { |
| 68 | + /** |
| 69 | + * Creates a {@link BoundedCountingInput} that will produce the specified number of elements, |
| 70 | + * from {@code 0} to {@code numElements - 1}. |
| 71 | + */ |
| 72 | + public static BoundedCountingInput upTo(long numElements) { |
| 73 | + checkArgument(numElements > 0, "numElements (%s) must be greater than 0", numElements); |
| 74 | + return new BoundedCountingInput(numElements); |
| 75 | + } |
| 76 | + |
| 77 | + /** |
| 78 | + * Creates an {@link UnboundedCountingInput} that will produce numbers starting from {@code 0} up |
| 79 | + * to {@link Long#MAX_VALUE}. |
| 80 | + * |
| 81 | + * <p>After {@link Long#MAX_VALUE}, the transform never produces more output. (In practice, this |
| 82 | + * limit should never be reached.) |
| 83 | + * |
| 84 | + * <p>Elements in the resulting {@link PCollection PCollection<Long>} will by default have |
| 85 | + * timestamps corresponding to processing time at element generation, provided by |
| 86 | + * {@link Instant#now}. Use the transform returned by |
| 87 | + * {@link UnboundedCountingInput#withTimestampFn(SerializableFunction)} to control the output |
| 88 | + * timestamps. |
| 89 | + */ |
| 90 | + public static UnboundedCountingInput unbounded() { |
| 91 | + return new UnboundedCountingInput( |
| 92 | + new NowTimestampFn(), Optional.<Long>absent(), Optional.<Duration>absent()); |
| 93 | + } |
| 94 | + |
| 95 | + /** |
| 96 | + * A {@link PTransform} that will produce a specified number of {@link Long Longs} starting from |
| 97 | + * 0. |
| 98 | + */ |
| 99 | + public static class BoundedCountingInput extends PTransform<PBegin, PCollection<Long>> { |
| 100 | + private final long numElements; |
| 101 | + |
| 102 | + private BoundedCountingInput(long numElements) { |
| 103 | + this.numElements = numElements; |
| 104 | + } |
| 105 | + |
| 106 | + @SuppressWarnings("deprecation") |
| 107 | + @Override |
| 108 | + public PCollection<Long> apply(PBegin begin) { |
| 109 | + return begin.apply(Read.from(CountingSource.upTo(numElements))); |
| 110 | + } |
| 111 | + } |
| 112 | + |
| 113 | + /** |
| 114 | + * A {@link PTransform} that will produce numbers starting from {@code 0} up to |
| 115 | + * {@link Long#MAX_VALUE}. |
| 116 | + * |
| 117 | + * <p>After {@link Long#MAX_VALUE}, the transform never produces more output. (In practice, this |
| 118 | + * limit should never be reached.) |
| 119 | + * |
| 120 | + * <p>Elements in the resulting {@link PCollection PCollection<Long>} will by default have |
| 121 | + * timestamps corresponding to processing time at element generation, provided by |
| 122 | + * {@link Instant#now}. Use the transform returned by |
| 123 | + * {@link UnboundedCountingInput#withTimestampFn(SerializableFunction)} to control the output |
| 124 | + * timestamps. |
| 125 | + */ |
| 126 | + public static class UnboundedCountingInput extends PTransform<PBegin, PCollection<Long>> { |
| 127 | + private final SerializableFunction<Long, Instant> timestampFn; |
| 128 | + private final Optional<Long> maxNumRecords; |
| 129 | + private final Optional<Duration> maxReadTime; |
| 130 | + |
| 131 | + private UnboundedCountingInput( |
| 132 | + SerializableFunction<Long, Instant> timestampFn, |
| 133 | + Optional<Long> maxNumRecords, |
| 134 | + Optional<Duration> maxReadTime) { |
| 135 | + this.timestampFn = timestampFn; |
| 136 | + this.maxNumRecords = maxNumRecords; |
| 137 | + this.maxReadTime = maxReadTime; |
| 138 | + } |
| 139 | + |
| 140 | + /** |
| 141 | + * Returns an {@link UnboundedCountingInput} like this one, but where output elements have the |
| 142 | + * timestamp specified by the timestampFn. |
| 143 | + * |
| 144 | + * <p>Note that the timestamps produced by {@code timestampFn} may not decrease. |
| 145 | + */ |
| 146 | + public UnboundedCountingInput withTimestampFn(SerializableFunction<Long, Instant> timestampFn) { |
| 147 | + return new UnboundedCountingInput(timestampFn, maxNumRecords, maxReadTime); |
| 148 | + } |
| 149 | + |
| 150 | + /** |
| 151 | + * Returns an {@link UnboundedCountingInput} like this one, but that will read at most the |
| 152 | + * specified number of elements. |
| 153 | + * |
| 154 | + * <p>A bounded amount of elements will be produced by the result transform, and the result |
| 155 | + * {@link PCollection} will be {@link IsBounded#BOUNDED bounded}. |
| 156 | + */ |
| 157 | + public UnboundedCountingInput withMaxNumRecords(long maxRecords) { |
| 158 | + checkArgument( |
| 159 | + maxRecords > 0, "MaxRecords must be a positive (nonzero) value. Got %s", maxRecords); |
| 160 | + return new UnboundedCountingInput(timestampFn, Optional.of(maxRecords), maxReadTime); |
| 161 | + } |
| 162 | + |
| 163 | + /** |
| 164 | + * Returns an {@link UnboundedCountingInput} like this one, but that will read for at most the |
| 165 | + * specified amount of time. |
| 166 | + * |
| 167 | + * <p>A bounded amount of elements will be produced by the result transform, and the result |
| 168 | + * {@link PCollection} will be {@link IsBounded#BOUNDED bounded}. |
| 169 | + */ |
| 170 | + public UnboundedCountingInput withMaxReadTime(Duration readTime) { |
| 171 | + checkNotNull(readTime, "ReadTime cannot be null"); |
| 172 | + return new UnboundedCountingInput(timestampFn, maxNumRecords, Optional.of(readTime)); |
| 173 | + } |
| 174 | + |
| 175 | + @SuppressWarnings("deprecation") |
| 176 | + @Override |
| 177 | + public PCollection<Long> apply(PBegin begin) { |
| 178 | + Unbounded<Long> read = Read.from(CountingSource.unboundedWithTimestampFn(timestampFn)); |
| 179 | + if (!maxNumRecords.isPresent() && !maxReadTime.isPresent()) { |
| 180 | + return begin.apply(read); |
| 181 | + } else if (maxNumRecords.isPresent() && !maxReadTime.isPresent()) { |
| 182 | + return begin.apply(read.withMaxNumRecords(maxNumRecords.get())); |
| 183 | + } else if (!maxNumRecords.isPresent() && maxReadTime.isPresent()) { |
| 184 | + return begin.apply(read.withMaxReadTime(maxReadTime.get())); |
| 185 | + } else { |
| 186 | + return begin.apply( |
| 187 | + read.withMaxReadTime(maxReadTime.get()).withMaxNumRecords(maxNumRecords.get())); |
| 188 | + } |
| 189 | + } |
| 190 | + } |
| 191 | +} |
0 commit comments