Skip to content

Commit

Permalink
apache#13 [euphoria-flink] Copy input instance in ReduceByKeyTranslat…
Browse files Browse the repository at this point in the history
…or because of Flink's object reuse
  • Loading branch information
vanekjar authored and David Moravek committed May 15, 2018
1 parent c9ab3c2 commit 9d0fe7c
Show file tree
Hide file tree
Showing 2 changed files with 67 additions and 10 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -185,30 +185,44 @@ private void doReduce(Iterable<BatchElement<Window, Pair>> values,
org.apache.flink.util.Collector<BatchElement<Window, Pair>> out) {

// Tuple2[Window, Key] => Reduced Value
Map<Tuple2, BatchElement<Window, Pair>> reducedValues = new HashMap<>();
Map<Tuple2, TimestampedElement> reducedValues = new HashMap<>();

for (BatchElement<Window, Pair> batchElement : values) {
Object key = batchElement.getElement().getFirst();
Window window = batchElement.getWindow();

Tuple2 kw = new Tuple2<>(window, key);
BatchElement<Window, Pair> val = reducedValues.get(kw);

// TimestampedElement holds only timestamp and reduced value.
// Key and window is stored separately in key part of the HashMap.
TimestampedElement val = reducedValues.get(kw);
if (val == null) {
reducedValues.put(kw, batchElement);

// It is necessary here to make a copy of the input element
// because of the reported bug in Apache Flink.
// See https://issues.apache.org/jira/browse/FLINK-6394
reducedValues.put(kw, new TimestampedElement(
batchElement.getTimestamp(),
batchElement.getElement().getSecond()));
} else {
Object reduced =
reducer.apply(Arrays.asList(val.getElement().getSecond(),
reducer.apply(Arrays.asList(val.getElement(),
batchElement.getElement().getSecond()));

reducedValues.put(kw, new BatchElement<>(
window,
Math.max(val.getTimestamp(), batchElement.getTimestamp()),
Pair.of(key, reduced)));
val.setElement(reduced);
val.setTimestamp(Math.max(val.getTimestamp(), batchElement.getTimestamp()));
}
}

for (Map.Entry<Tuple2, BatchElement<Window, Pair>> e : reducedValues.entrySet()) {
out.collect(e.getValue());
for (Map.Entry<Tuple2, TimestampedElement> e : reducedValues.entrySet()) {
Window window = (Window) e.getKey().f0;
Object key = e.getKey().f1;

out.collect(new BatchElement<>(
window,
e.getValue().getTimestamp(),
Pair.of(key, e.getValue().getElement())
));
}
}

Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,43 @@
/**
* Copyright 2016-2017 Seznam.cz, a.s.
*
* 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 cz.seznam.euphoria.flink.batch;

final class TimestampedElement {

private long timestamp;
private Object el;

public TimestampedElement(long timestamp, Object el) {
this.timestamp = timestamp;
this.el = el;
}

public long getTimestamp() {
return timestamp;
}

public void setTimestamp(long timestamp) {
this.timestamp = timestamp;
}

public Object getElement() {
return el;
}

public void setElement(Object el) {
this.el = el;
}
}

0 comments on commit 9d0fe7c

Please sign in to comment.