-
Notifications
You must be signed in to change notification settings - Fork 4.3k
Commit
This commit does not belong to any branch on this repository, and may belong to a fork outside of the repository.
Browse files
Browse the repository at this point in the history
- Loading branch information
Showing
9 changed files
with
447 additions
and
64 deletions.
There are no files selected for viewing
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
192 changes: 192 additions & 0 deletions
192
sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/Reify.java
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,192 @@ | ||
/* | ||
* Licensed to the Apache Software Foundation (ASF) under one | ||
* or more contributor license agreements. See the NOTICE file | ||
* distributed with this work for additional information | ||
* regarding copyright ownership. The ASF licenses this file | ||
* to you under the Apache License, Version 2.0 (the | ||
* "License"); you may not use this file except in compliance | ||
* with the License. You may obtain a copy of the License at | ||
* | ||
* http://www.apache.org/licenses/LICENSE-2.0 | ||
* | ||
* Unless required by applicable law or agreed to in writing, software | ||
* distributed under the License is distributed on an "AS IS" BASIS, | ||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. | ||
* See the License for the specific language governing permissions and | ||
* limitations under the License. | ||
*/ | ||
|
||
package org.apache.beam.sdk.transforms; | ||
|
||
import org.apache.beam.sdk.coders.KvCoder; | ||
import org.apache.beam.sdk.transforms.windowing.BoundedWindow; | ||
import org.apache.beam.sdk.values.KV; | ||
import org.apache.beam.sdk.values.PCollection; | ||
import org.apache.beam.sdk.values.TimestampedValue; | ||
import org.apache.beam.sdk.values.TimestampedValue.TimestampedValueCoder; | ||
import org.apache.beam.sdk.values.ValueInSingleWindow; | ||
import org.joda.time.Duration; | ||
|
||
/** {@link PTransform PTransforms} for reifying the timestamp, window and pane of values. */ | ||
public class Reify { | ||
/** Private implementation of {@link #windows()}. */ | ||
private static class Window<T> | ||
extends PTransform<PCollection<T>, PCollection<ValueInSingleWindow<T>>> { | ||
@Override | ||
public PCollection<ValueInSingleWindow<T>> expand(PCollection<T> input) { | ||
return input | ||
.apply( | ||
ParDo.of( | ||
new DoFn<T, ValueInSingleWindow<T>>() { | ||
@ProcessElement | ||
public void processElement(ProcessContext c, BoundedWindow window) { | ||
c.outputWithTimestamp( | ||
ValueInSingleWindow.of(c.element(), c.timestamp(), window, c.pane()), | ||
c.timestamp()); | ||
} | ||
})) | ||
.setCoder( | ||
ValueInSingleWindow.Coder.of( | ||
input.getCoder(), input.getWindowingStrategy().getWindowFn().windowCoder())); | ||
} | ||
} | ||
|
||
private static class Timestamp<T> | ||
extends PTransform<PCollection<T>, PCollection<TimestampedValue<T>>> { | ||
@Override | ||
public PCollection<TimestampedValue<T>> expand(PCollection<T> input) { | ||
return input | ||
.apply( | ||
ParDo.of( | ||
new DoFn<T, TimestampedValue<T>>() { | ||
@ProcessElement | ||
public void processElement(ProcessContext context) { | ||
context.output(TimestampedValue.of(context.element(), context.timestamp())); | ||
} | ||
})) | ||
.setCoder(TimestampedValueCoder.of(input.getCoder())); | ||
} | ||
} | ||
|
||
private static class WindowInValue<K, V> | ||
extends PTransform<PCollection<KV<K, V>>, PCollection<KV<K, ValueInSingleWindow<V>>>> { | ||
@Override | ||
public PCollection<KV<K, ValueInSingleWindow<V>>> expand(PCollection<KV<K, V>> input) { | ||
KvCoder<K, V> coder = (KvCoder<K, V>) input.getCoder(); | ||
return input | ||
.apply( | ||
ParDo.of( | ||
new DoFn<KV<K, V>, KV<K, ValueInSingleWindow<V>>>() { | ||
@ProcessElement | ||
public void processElement(ProcessContext c, BoundedWindow window) { | ||
c.output( | ||
KV.of( | ||
c.element().getKey(), | ||
ValueInSingleWindow.of( | ||
c.element().getValue(), c.timestamp(), window, c.pane()))); | ||
} | ||
})) | ||
.setCoder( | ||
KvCoder.of( | ||
coder.getKeyCoder(), | ||
ValueInSingleWindow.Coder.of( | ||
coder.getValueCoder(), | ||
input.getWindowingStrategy().getWindowFn().windowCoder()))); | ||
} | ||
} | ||
|
||
private static class TimestampInValue<K, V> | ||
extends PTransform<PCollection<KV<K, V>>, PCollection<KV<K, TimestampedValue<V>>>> { | ||
@Override | ||
public PCollection<KV<K, TimestampedValue<V>>> expand(PCollection<KV<K, V>> input) { | ||
KvCoder<K, V> coder = (KvCoder<K, V>) input.getCoder(); | ||
return input | ||
.apply( | ||
ParDo.of( | ||
new DoFn<KV<K, V>, KV<K, TimestampedValue<V>>>() { | ||
@ProcessElement | ||
public void processElement(ProcessContext context) { | ||
context.output( | ||
KV.of( | ||
context.element().getKey(), | ||
TimestampedValue.of( | ||
context.element().getValue(), context.timestamp()))); | ||
} | ||
})) | ||
.setCoder( | ||
KvCoder.of(coder.getKeyCoder(), TimestampedValueCoder.of(coder.getValueCoder()))); | ||
} | ||
} | ||
|
||
private static class ExtractTimestampsFromValues<K, V> | ||
extends PTransform<PCollection<KV<K, TimestampedValue<V>>>, PCollection<KV<K, V>>> { | ||
@Override | ||
public PCollection<KV<K, V>> expand(PCollection<KV<K, TimestampedValue<V>>> input) { | ||
KvCoder<K, TimestampedValue<V>> kvCoder = (KvCoder<K, TimestampedValue<V>>) input.getCoder(); | ||
TimestampedValueCoder<V> tvCoder = (TimestampedValueCoder<V>) kvCoder.getValueCoder(); | ||
return input | ||
.apply( | ||
ParDo.of( | ||
new DoFn<KV<K, TimestampedValue<V>>, KV<K, V>>() { | ||
@Override | ||
public Duration getAllowedTimestampSkew() { | ||
return Duration.millis(Long.MAX_VALUE); | ||
} | ||
|
||
@ProcessElement | ||
public void processElement(ProcessContext context) { | ||
KV<K, TimestampedValue<V>> kv = context.element(); | ||
context.outputWithTimestamp( | ||
KV.of(kv.getKey(), kv.getValue().getValue()), | ||
kv.getValue().getTimestamp()); | ||
} | ||
})) | ||
.setCoder(KvCoder.of(kvCoder.getKeyCoder(), tvCoder.getValueCoder())); | ||
} | ||
} | ||
|
||
private Reify() {} | ||
|
||
/** | ||
* Create a {@link PTransform} that will output all inputs wrapped in a {@link TimestampedValue}. | ||
*/ | ||
public static <T> PTransform<PCollection<T>, PCollection<TimestampedValue<T>>> timestamps() { | ||
return new Timestamp<>(); | ||
} | ||
|
||
/** | ||
* Create a {@link PTransform} that will output all input {@link KV KVs} with the timestamp inside | ||
* the value. | ||
*/ | ||
public static <K, V> | ||
PTransform<PCollection<KV<K, V>>, PCollection<KV<K, TimestampedValue<V>>>> | ||
timestampsInValue() { | ||
return new TimestampInValue<>(); | ||
} | ||
|
||
/** | ||
* Create a {@link PTransform} that will reify information from the processing context into | ||
* instances of {@link ValueInSingleWindow}. | ||
* | ||
* @param <T> element type | ||
*/ | ||
public static <T> PTransform<PCollection<T>, PCollection<ValueInSingleWindow<T>>> windows() { | ||
return new Window<>(); | ||
} | ||
|
||
/** | ||
* Create a {@link PTransform} that will output all input {@link KV KVs} with the window pane info | ||
* inside the value. | ||
*/ | ||
public static <K, V> | ||
PTransform<PCollection<KV<K, V>>, PCollection<KV<K, ValueInSingleWindow<V>>>> | ||
windowsInValue() { | ||
return new WindowInValue<>(); | ||
} | ||
|
||
public static <K, V> | ||
PTransform<PCollection<KV<K, TimestampedValue<V>>>, PCollection<KV<K, V>>> | ||
extractTimestampsFromValues() { | ||
return new ExtractTimestampsFromValues<>(); | ||
} | ||
} |
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Oops, something went wrong.