From 3832d7b7a0216e5cf6e5100bb56b5a703d4fb79e Mon Sep 17 00:00:00 2001 From: Aljoscha Krettek Date: Tue, 2 Dec 2014 17:34:46 +0100 Subject: [PATCH 1/4] [FLINK-1137] Enhance MutableObjectIterator with non-reuse next() This is in preparation for configurable object-reuse mode. We previously referred to this as mutable object vs. mutable object safe mode or some such thing. --- .../flink/streaming/io/CoReaderIterator.java | 9 +-- .../flink/streaming/util/MockContext.java | 11 ++++ .../flink/util/MutableObjectIterator.java | 20 +++++-- .../BroadcastVariableMaterialization.java | 8 +-- .../disk/ChannelReaderInputViewIterator.java | 14 +++++ .../runtime/io/disk/InputViewIterator.java | 9 +++ .../iterative/io/HashPartitionIterator.java | 33 +++++++++++ .../flink/runtime/operators/DataSinkTask.java | 5 +- .../operators/hash/CompactingHashTable.java | 21 +++++++ .../runtime/operators/hash/HashPartition.java | 23 +++++++- .../operators/hash/MutableHashTable.java | 50 ++++++++++++++++ .../AbstractBlockResettableIterator.java | 10 ++++ .../BlockResettableMutableObjectIterator.java | 36 ++++++++++- ...illingResettableMutableObjectIterator.java | 35 ++++++++++- .../sort/FixedLengthRecordSorter.java | 27 +++++++++ .../runtime/operators/sort/MergeIterator.java | 29 +++++++++ .../operators/sort/NormalizedKeySorter.java | 30 ++++++++++ .../operators/util/ReaderIterator.java | 33 +++++++++-- .../plugable/DeserializationDelegate.java | 39 +----------- .../NonReusingDeserializationDelegate.java | 57 ++++++++++++++++++ .../ReusingDeserializationDelegate.java | 59 +++++++++++++++++++ .../util/EmptyMutableObjectIterator.java | 11 ++++ .../util/KeyGroupedMutableObjectIterator.java | 37 ++++++++++++ .../util/RegularToMutableObjectIterator.java | 9 +++ .../operators/hash/HashTableITCase.java | 30 ++++++++++ .../sort/MassiveStringSortingITCase.java | 19 ++++++ .../operators/sort/MergeIteratorTest.java | 17 ++++++ .../operators/sort/MockRecordReader.java | 26 ++++++++ .../DelayingInfinitiveInputIterator.java | 10 +++- .../testutils/InfiniteInputIterator.java | 9 +++ .../MutableObjectIteratorWrapper.java | 13 ++++ .../testutils/RandomIntPairGenerator.java | 12 ++++ .../runtime/operators/testutils/TestData.java | 40 ++++++++++++- .../testutils/UniformIntPairGenerator.java | 33 +++++++++++ .../testutils/UniformRecordGenerator.java | 34 +++++++++++ .../testutils/UniformStringPairGenerator.java | 33 +++++++++++ .../operators/testutils/UnionIterator.java | 18 ++++++ .../util/KeyGroupedIteratorImmutableTest.java | 14 +++++ .../runtime/util/KeyGroupedIteratorTest.java | 15 +++++ .../misc/MassiveCaseClassSortingITCase.scala | 10 ++++ 40 files changed, 884 insertions(+), 64 deletions(-) create mode 100644 flink-runtime/src/main/java/org/apache/flink/runtime/plugable/NonReusingDeserializationDelegate.java create mode 100644 flink-runtime/src/main/java/org/apache/flink/runtime/plugable/ReusingDeserializationDelegate.java diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/io/CoReaderIterator.java b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/io/CoReaderIterator.java index e4110e7280b20..ed90c03d6c315 100755 --- a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/io/CoReaderIterator.java +++ b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/io/CoReaderIterator.java @@ -21,6 +21,7 @@ import org.apache.flink.api.common.typeutils.TypeSerializer; import org.apache.flink.runtime.plugable.DeserializationDelegate; +import org.apache.flink.runtime.plugable.ReusingDeserializationDelegate; /** * A CoReaderIterator wraps a {@link CoRecordReader} producing records of two @@ -31,15 +32,15 @@ public class CoReaderIterator { private final CoRecordReader, DeserializationDelegate> reader; // the // source - protected final DeserializationDelegate delegate1; - protected final DeserializationDelegate delegate2; + protected final ReusingDeserializationDelegate delegate1; + protected final ReusingDeserializationDelegate delegate2; public CoReaderIterator( CoRecordReader, DeserializationDelegate> reader, TypeSerializer serializer1, TypeSerializer serializer2) { this.reader = reader; - this.delegate1 = new DeserializationDelegate(serializer1); - this.delegate2 = new DeserializationDelegate(serializer2); + this.delegate1 = new ReusingDeserializationDelegate(serializer1); + this.delegate2 = new ReusingDeserializationDelegate(serializer2); } public int next(T1 target1, T2 target2) throws IOException { diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/util/MockContext.java b/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/util/MockContext.java index 87bedb2af1e3a..5537052d82f83 100644 --- a/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/util/MockContext.java +++ b/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/util/MockContext.java @@ -72,6 +72,17 @@ public StreamRecord next(StreamRecord reuse) throws IOException { } return reuse; } + + @Override + public StreamRecord next() throws IOException { + if (listIterator.hasNext()) { + StreamRecord result = new StreamRecord(); + result.setObject(listIterator.next()); + return result; + } else { + return null; + } + } } public List getOutputs() { diff --git a/flink-core/src/main/java/org/apache/flink/util/MutableObjectIterator.java b/flink-core/src/main/java/org/apache/flink/util/MutableObjectIterator.java index b7b41d40fffdd..ea5ed78b17d91 100644 --- a/flink-core/src/main/java/org/apache/flink/util/MutableObjectIterator.java +++ b/flink-core/src/main/java/org/apache/flink/util/MutableObjectIterator.java @@ -21,16 +21,18 @@ /** * A simple iterator interface. The key differences to the {@link java.util.Iterator} are that this - * iterator accepts an object into which it can place the content if the object is mutable, and that - * it consolidates the logic in a single next() function, rather than in two different - * functions such as hasNext() and next(). + * iterator also as a next() method that accepts an object into which it can + * place the content if the object is mutable, and that it consolidates the logic in a single + * next() function, rather than in two different functions such as + * hasNext() and next(). * * @param The element type of the collection iterated over. */ public interface MutableObjectIterator { /** - * Gets the next element from the collection. The contents of that next element is put into the given target object. + * Gets the next element from the collection. The contents of that next element is put into the + * given target object. * * @param reuse The target object into which to place next element if E is mutable. * @return The filled object or null if the iterator is exhausted @@ -39,4 +41,14 @@ public interface MutableObjectIterator { * serialization / deserialization logic */ public E next(E reuse) throws IOException; + + /** + * Gets the next element from the collection. The reader must create a new instance itself. + * + * @return The object or null if the iterator is exhausted + * + * @throws IOException Thrown, if a problem occurred in the underlying I/O layer or in the + * serialization / deserialization logic + */ + public E next() throws IOException; } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/broadcast/BroadcastVariableMaterialization.java b/flink-runtime/src/main/java/org/apache/flink/runtime/broadcast/BroadcastVariableMaterialization.java index f7aebb69fa45b..5b5f2f25c45d9 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/broadcast/BroadcastVariableMaterialization.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/broadcast/BroadcastVariableMaterialization.java @@ -30,7 +30,6 @@ import org.apache.flink.runtime.io.network.api.MutableReader; import org.apache.flink.runtime.operators.RegularPactTask; import org.apache.flink.runtime.operators.util.ReaderIterator; -import org.apache.flink.runtime.plugable.DeserializationDelegate; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -96,10 +95,11 @@ public void materializeVariable(MutableReader reader, TypeSerializerFactory> typedReader = (MutableReader>) reader; + final MutableReader typedReader = (MutableReader) reader; @SuppressWarnings("unchecked") final TypeSerializer serializer = ((TypeSerializerFactory) serializerFactory).getSerializer(); - + + @SuppressWarnings("unchecked") final ReaderIterator readerIterator = new ReaderIterator(typedReader, serializer); if (materializer) { @@ -111,7 +111,7 @@ public void materializeVariable(MutableReader reader, TypeSerializerFactory data = new ArrayList(); T element; - while ((element = readerIterator.next(serializer.createInstance())) != null) { + while ((element = readerIterator.next()) != null) { data.add(element); } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/io/disk/ChannelReaderInputViewIterator.java b/flink-runtime/src/main/java/org/apache/flink/runtime/io/disk/ChannelReaderInputViewIterator.java index f38aa2535f0eb..6007db913b636 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/io/disk/ChannelReaderInputViewIterator.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/io/disk/ChannelReaderInputViewIterator.java @@ -92,4 +92,18 @@ public E next(E reuse) throws IOException return null; } } + + @Override + public E next() throws IOException + { + try { + return this.accessors.deserialize(this.inView); + } catch (EOFException eofex) { + final List freeMem = this.inView.close(); + if (this.freeMemTarget != null) { + this.freeMemTarget.addAll(freeMem); + } + return null; + } + } } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/io/disk/InputViewIterator.java b/flink-runtime/src/main/java/org/apache/flink/runtime/io/disk/InputViewIterator.java index 74562c202b983..4f37549bffa26 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/io/disk/InputViewIterator.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/io/disk/InputViewIterator.java @@ -45,4 +45,13 @@ public E next(E reuse) throws IOException { return null; } } + + @Override + public E next() throws IOException { + try { + return this.serializer.deserialize(this.inputView); + } catch (EOFException e) { + return null; + } + } } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/iterative/io/HashPartitionIterator.java b/flink-runtime/src/main/java/org/apache/flink/runtime/iterative/io/HashPartitionIterator.java index 2daf5cf68eff0..209fb796fb84c 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/iterative/io/HashPartitionIterator.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/iterative/io/HashPartitionIterator.java @@ -65,6 +65,24 @@ public BT next(BT reuse) throws IOException { return reuse; } + @Override + public BT next() throws IOException { + if (currentPartition == null) { + if (!partitions.hasNext()) { + return null; + } + currentPartition = partitions.next(); + currentPartition.setReadPosition(0); + } + + try { + return serializer.deserialize(currentPartition); + } catch (EOFException e) { + return advanceAndRead(); + } + + } + /* jump to the next partition and continue reading from that */ private BT advanceAndRead(BT reuse) throws IOException { if (!partitions.hasNext()) { @@ -81,4 +99,19 @@ private BT advanceAndRead(BT reuse) throws IOException { return reuse; } + /* jump to the next partition and continue reading from that */ + private BT advanceAndRead() throws IOException { + if (!partitions.hasNext()) { + return null; + } + currentPartition = partitions.next(); + currentPartition.setReadPosition(0); + + try { + return serializer.deserialize(currentPartition); + } catch (EOFException e) { + return advanceAndRead(); + } + } + } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/DataSinkTask.java b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/DataSinkTask.java index 74c625f8bcb2d..610ab0661f8c3 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/DataSinkTask.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/DataSinkTask.java @@ -19,6 +19,7 @@ package org.apache.flink.runtime.operators; +import org.apache.flink.runtime.plugable.DeserializationDelegate; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import org.apache.flink.api.common.io.CleanupWhenUnsuccessful; @@ -38,7 +39,6 @@ import org.apache.flink.runtime.operators.util.CloseableInputProvider; import org.apache.flink.runtime.operators.util.ReaderIterator; import org.apache.flink.runtime.operators.util.TaskConfig; -import org.apache.flink.runtime.plugable.DeserializationDelegate; import org.apache.flink.util.MutableObjectIterator; /** @@ -334,9 +334,8 @@ private void initInputReaders() throws Exception { this.inputTypeSerializerFactory = this.config.getInputSerializer(0, getUserCodeClassLoader()); - MutableReader> reader = (MutableReader>) inputReader; @SuppressWarnings({ "rawtypes" }) - final MutableObjectIterator iter = new ReaderIterator(reader, this.inputTypeSerializerFactory.getSerializer()); + final MutableObjectIterator iter = new ReaderIterator(inputReader, this.inputTypeSerializerFactory.getSerializer()); this.reader = (MutableObjectIterator)iter; // final sanity check diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/hash/CompactingHashTable.java b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/hash/CompactingHashTable.java index 4f10e0b740a4f..4c39f28a7cbcd 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/hash/CompactingHashTable.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/hash/CompactingHashTable.java @@ -1238,6 +1238,27 @@ public T next(T reuse) throws IOException { } } + @Override + public T next() throws IOException { + // This is just a copy of the above, I wanted to keep the two separate, + // in case we change something later. Plus, it keeps the diff clean... :D + if(done || this.table.closed.get()) { + return null; + } else if(!cache.isEmpty()) { + return cache.remove(cache.size()-1); + } else { + while(!done && cache.isEmpty()) { + done = !fillCache(); + } + if(!done) { + return cache.remove(cache.size()-1); + } else { + return null; + } + } + } + + /** * utility function that inserts all entries from a bucket and its overflow buckets into the cache * diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/hash/HashPartition.java b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/hash/HashPartition.java index 08acd16544e7e..23a415ddd6085 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/hash/HashPartition.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/hash/HashPartition.java @@ -44,8 +44,8 @@ /** * - * @param The type of the build side records. - * @param The type of the probe side records. + * @tparam BT The type of the build side records. + * @tparam PT The type of the probe side records. */ public class HashPartition extends AbstractPagedInputView implements SeekableDataInputView { @@ -620,7 +620,24 @@ public final BT next(BT reuse) throws IOException return null; } } - + + public final BT next() throws IOException + { + final int pos = getCurrentPositionInSegment(); + final int buffer = HashPartition.this.currentBufferNum; + + this.currentPointer = (((long) buffer) << HashPartition.this.segmentSizeBits) + pos; + + try { + BT result = HashPartition.this.buildSideSerializer.deserialize(HashPartition.this); + this.currentHashCode = this.comparator.hash(result); + return result; + } catch (EOFException eofex) { + return null; + } + } + + protected final long getPointer() { return this.currentPointer; diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/hash/MutableHashTable.java b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/hash/MutableHashTable.java index 67f1ea25ea6ea..e69ef172717b2 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/hash/MutableHashTable.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/hash/MutableHashTable.java @@ -1386,6 +1386,56 @@ public BT next(BT reuse) { this.numInSegment = 0; } } + + public BT next() { + // loop over all segments that are involved in the bucket (original bucket plus overflow buckets) + while (true) { + + while (this.numInSegment < this.countInSegment) { + + final int thisCode = this.bucket.getInt(this.posInSegment); + this.posInSegment += HASH_CODE_LEN; + + // check if the hash code matches + if (thisCode == this.searchHashCode) { + // get the pointer to the pair + final long pointer = this.bucket.getLong(this.bucketInSegmentOffset + + BUCKET_POINTER_START_OFFSET + (this.numInSegment * POINTER_LEN)); + this.numInSegment++; + + // deserialize the key to check whether it is really equal, or whether we had only a hash collision + try { + this.partition.setReadPosition(pointer); + BT result = this.accessor.deserialize(this.partition); + if (this.comparator.equalToReference(result)) { + this.lastPointer = pointer; + return result; + } + } + catch (IOException ioex) { + throw new RuntimeException("Error deserializing key or value from the hashtable: " + + ioex.getMessage(), ioex); + } + } + else { + this.numInSegment++; + } + } + + // this segment is done. check if there is another chained bucket + final long forwardPointer = this.bucket.getLong(this.bucketInSegmentOffset + HEADER_FORWARD_OFFSET); + if (forwardPointer == BUCKET_FORWARD_POINTER_NOT_SET) { + return null; + } + + final int overflowSegNum = (int) (forwardPointer >>> 32); + this.bucket = this.overflowSegments[overflowSegNum]; + this.bucketInSegmentOffset = (int) (forwardPointer & 0xffffffff); + this.countInSegment = this.bucket.getShort(this.bucketInSegmentOffset + HEADER_COUNT_OFFSET); + this.posInSegment = this.bucketInSegmentOffset + BUCKET_HEADER_LENGTH; + this.numInSegment = 0; + } + } public void writeBack(BT value) throws IOException { final SeekableDataOutputView outView = this.partition.getWriteView(); diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/resettable/AbstractBlockResettableIterator.java b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/resettable/AbstractBlockResettableIterator.java index 421e0c7c1e626..730d19a8d2df9 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/resettable/AbstractBlockResettableIterator.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/resettable/AbstractBlockResettableIterator.java @@ -173,4 +173,14 @@ protected T getNextRecord(T reuse) throws IOException { return null; } } + + protected T getNextRecord() throws IOException { + if (this.numRecordsReturned < this.numRecordsInBuffer) { + this.numRecordsReturned++; + return this.serializer.deserialize(this.readView); + } else { + return null; + } + } + } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/resettable/BlockResettableMutableObjectIterator.java b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/resettable/BlockResettableMutableObjectIterator.java index 4fe7dbba11427..abce462ab4a08 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/resettable/BlockResettableMutableObjectIterator.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/resettable/BlockResettableMutableObjectIterator.java @@ -103,7 +103,41 @@ public T next(T target) throws IOException { } } } - + + @Override + public T next() throws IOException { + // check for the left over element + if (this.readPhase) { + return getNextRecord(); + } else { + // writing phase. check for leftover first + T result = null; + if (this.leftOverReturned) { + // get next record + if ((result = this.input.next()) != null) { + if (writeNextRecord(result)) { + return result; + } else { + // did not fit into memory, keep as leftover + this.leftOverRecord = this.serializer.copy(result); + this.leftOverReturned = false; + this.fullWriteBuffer = true; + return null; + } + } else { + this.noMoreBlocks = true; + return null; + } + } else if (this.fullWriteBuffer) { + return null; + } else { + this.leftOverReturned = true; + return this.leftOverRecord; + } + } + } + + public void reset() { // a reset always goes to the read phase diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/resettable/SpillingResettableMutableObjectIterator.java b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/resettable/SpillingResettableMutableObjectIterator.java index 1d0b5407e25f3..5467ae91ac7ff 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/resettable/SpillingResettableMutableObjectIterator.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/resettable/SpillingResettableMutableObjectIterator.java @@ -159,7 +159,40 @@ public T next(T reuse) throws IOException { } } } - + + @Override + public T next() throws IOException { + T result = null; + if (this.inView != null) { + // reading, any subsequent pass + if (this.currentElementNum < this.elementCount) { + try { + result = this.serializer.deserialize(this.inView); + } catch (IOException e) { + throw new RuntimeException("SpillingIterator: Error reading element from buffer.", e); + } + this.currentElementNum++; + return result; + } else { + return null; + } + } else { + // writing pass (first) + if ((result = this.input.next()) != null) { + try { + this.serializer.serialize(result, this.buffer); + } catch (IOException e) { + throw new RuntimeException("SpillingIterator: Error writing element to buffer.", e); + } + this.elementCount++; + return result; + } else { + return null; + } + } + } + + public void consumeAndCacheRemainingData() throws IOException { // check that we are in the first pass and that more input data is left if (this.inView == null) { diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/sort/FixedLengthRecordSorter.java b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/sort/FixedLengthRecordSorter.java index 46399e9195959..f05694b693e6e 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/sort/FixedLengthRecordSorter.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/sort/FixedLengthRecordSorter.java @@ -358,6 +358,33 @@ public T next(T reuse) { return null; } } + + @Override + public T next() { + if (this.currentTotal < this.numTotal) { + + if (this.currentInSegment >= this.numPerSegment) { + this.currentInSegment = 0; + this.currentSegmentIndex++; + this.in.set(sortBuffer.get(this.currentSegmentIndex), 0); + } + + this.currentTotal++; + this.currentInSegment++; + + try { + // This might blow up in our face, but we ignore the readWithNormalization/ + // writeWithNormalization methods for now. + return this.comp.readWithKeyDenormalization(null, this.in); + } + catch (IOException ioe) { + throw new RuntimeException(ioe); + } + } + else { + return null; + } + } }; } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/sort/MergeIterator.java b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/sort/MergeIterator.java index 7a5012de04d8f..f3dc50eb457cc 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/sort/MergeIterator.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/sort/MergeIterator.java @@ -93,6 +93,35 @@ public E next(E reuse) throws IOException } } + /** + * Gets the next smallest element, with respect to the definition of order implied by + * the {@link TypeSerializer} provided to this iterator. + * + * @return True, if the iterator had another element, false otherwise. + * + * @see org.apache.flink.util.MutableObjectIterator#next(java.lang.Object) + */ + @Override + public E next() throws IOException + { + if (this.heap.size() > 0) { + // get the smallest element + final HeadStream top = this.heap.peek(); + E result = this.serializer.copy(top.getHead()); + + // read an element + if (!top.nextHead()) { + this.heap.poll(); + } else { + this.heap.adjustTop(); + } + return result; + } + else { + return null; + } + } + // ============================================================================================ // Internal Classes that wrap the sorted input streams // ============================================================================================ diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/sort/NormalizedKeySorter.java b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/sort/NormalizedKeySorter.java index c69474a995121..c3827081304c0 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/sort/NormalizedKeySorter.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/sort/NormalizedKeySorter.java @@ -322,6 +322,11 @@ private final T getRecordFromBuffer(T reuse, long pointer) throws IOException { this.recordBuffer.setReadPosition(pointer); return this.serializer.deserialize(reuse, this.recordBuffer); } + + private final T getRecordFromBuffer(long pointer) throws IOException { + this.recordBuffer.setReadPosition(pointer); + return this.serializer.deserialize(this.recordBuffer); + } private final int compareRecords(long pointer1, long pointer2) { this.recordBuffer.setReadPosition(pointer1); @@ -431,6 +436,31 @@ public T next(T target) return null; } } + + @Override + public T next() + { + if (this.current < this.size) { + this.current++; + if (this.currentOffset > lastIndexEntryOffset) { + this.currentOffset = 0; + this.currentIndexSegment = sortIndex.get(++this.currentSegment); + } + + long pointer = this.currentIndexSegment.getLong(this.currentOffset); + this.currentOffset += indexEntrySize; + + try { + return getRecordFromBuffer(pointer); + } + catch (IOException ioe) { + throw new RuntimeException(ioe); + } + } + else { + return null; + } + } }; } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/util/ReaderIterator.java b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/util/ReaderIterator.java index 0d29d5e5454cf..606c50ce4511a 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/util/ReaderIterator.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/util/ReaderIterator.java @@ -24,6 +24,8 @@ import org.apache.flink.api.common.typeutils.TypeSerializer; import org.apache.flink.runtime.io.network.api.MutableReader; import org.apache.flink.runtime.plugable.DeserializationDelegate; +import org.apache.flink.runtime.plugable.NonReusingDeserializationDelegate; +import org.apache.flink.runtime.plugable.ReusingDeserializationDelegate; import org.apache.flink.util.MutableObjectIterator; @@ -32,9 +34,10 @@ */ public final class ReaderIterator implements MutableObjectIterator { - private final MutableReader> reader; // the source + private final MutableReader reader; // the source - private final DeserializationDelegate delegate; + private final ReusingDeserializationDelegate reusingDelegate; + private final NonReusingDeserializationDelegate nonReusingDelegate; /** * Creates a new iterator, wrapping the given reader. @@ -43,15 +46,33 @@ public final class ReaderIterator implements MutableObjectIterator { */ public ReaderIterator(MutableReader> reader, TypeSerializer serializer) { this.reader = reader; - this.delegate = new DeserializationDelegate(serializer); + this.reusingDelegate = new ReusingDeserializationDelegate(serializer); + this.nonReusingDelegate = new NonReusingDeserializationDelegate(serializer); } @Override + @SuppressWarnings("unchecked") public T next(T target) throws IOException { - this.delegate.setInstance(target); + this.reusingDelegate.setInstance(target); try { - if (this.reader.next(this.delegate)) { - return this.delegate.getInstance(); + if (this.reader.next(this.reusingDelegate)) { + return this.reusingDelegate.getInstance(); + } else { + return null; + } + + } + catch (InterruptedException e) { + throw new IOException("Reader interrupted.", e); + } + } + + @Override + @SuppressWarnings("unchecked") + public T next() throws IOException { + try { + if (this.reader.next(this.nonReusingDelegate)) { + return this.nonReusingDelegate.getInstance(); } else { return null; } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/plugable/DeserializationDelegate.java b/flink-runtime/src/main/java/org/apache/flink/runtime/plugable/DeserializationDelegate.java index fbe91eda92236..9ca5954dee21e 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/plugable/DeserializationDelegate.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/plugable/DeserializationDelegate.java @@ -15,45 +15,12 @@ * See the License for the specific language governing permissions and * limitations under the License. */ - - package org.apache.flink.runtime.plugable; -import java.io.IOException; - -import org.apache.flink.api.common.typeutils.TypeSerializer; import org.apache.flink.core.io.IOReadableWritable; -import org.apache.flink.core.memory.DataInputView; -import org.apache.flink.core.memory.DataOutputView; - - -public class DeserializationDelegate implements IOReadableWritable { - - private T instance; - - private final TypeSerializer serializer; - - - public DeserializationDelegate(TypeSerializer serializer) { - this.serializer = serializer; - } - - - public void setInstance(T instance) { - this.instance = instance; - } - - public T getInstance() { - return instance; - } - @Override - public void write(DataOutputView out) throws IOException { - throw new IllegalStateException("Serialization method called on DeserializationDelegate."); - } +public interface DeserializationDelegate extends IOReadableWritable { + void setInstance(T instance); - @Override - public void read(DataInputView in) throws IOException { - this.instance = this.serializer.deserialize(this.instance, in); - } + T getInstance(); } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/plugable/NonReusingDeserializationDelegate.java b/flink-runtime/src/main/java/org/apache/flink/runtime/plugable/NonReusingDeserializationDelegate.java new file mode 100644 index 0000000000000..859f354d07ede --- /dev/null +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/plugable/NonReusingDeserializationDelegate.java @@ -0,0 +1,57 @@ +/* + * 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.flink.runtime.plugable; + +import org.apache.flink.api.common.typeutils.TypeSerializer; +import org.apache.flink.core.memory.DataInputView; +import org.apache.flink.core.memory.DataOutputView; + +import java.io.IOException; + + +public class NonReusingDeserializationDelegate implements DeserializationDelegate { + + private T instance; + + private final TypeSerializer serializer; + + + public NonReusingDeserializationDelegate(TypeSerializer serializer) { + this.serializer = serializer; + } + + public void setInstance(T instance) { + this.instance = instance; + } + + public T getInstance() { + return instance; + } + + @Override + public void write(DataOutputView out) throws IOException { + throw new IllegalStateException("Serialization method called on DeserializationDelegate."); + } + + @Override + public void read(DataInputView in) throws IOException { + this.instance = this.serializer.deserialize(in); + } +} diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/plugable/ReusingDeserializationDelegate.java b/flink-runtime/src/main/java/org/apache/flink/runtime/plugable/ReusingDeserializationDelegate.java new file mode 100644 index 0000000000000..f3c254b4847b4 --- /dev/null +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/plugable/ReusingDeserializationDelegate.java @@ -0,0 +1,59 @@ +/* + * 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.flink.runtime.plugable; + +import java.io.IOException; + +import org.apache.flink.api.common.typeutils.TypeSerializer; +import org.apache.flink.core.memory.DataInputView; +import org.apache.flink.core.memory.DataOutputView; + + +public class ReusingDeserializationDelegate implements DeserializationDelegate { + + private T instance; + + private final TypeSerializer serializer; + + + public ReusingDeserializationDelegate(TypeSerializer serializer) { + this.serializer = serializer; + } + + @Override + public void setInstance(T instance) { + this.instance = instance; + } + + @Override + public T getInstance() { + return instance; + } + + @Override + public void write(DataOutputView out) throws IOException { + throw new IllegalStateException("Serialization method called on DeserializationDelegate."); + } + + @Override + public void read(DataInputView in) throws IOException { + this.instance = this.serializer.deserialize(this.instance, in); + } +} diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/util/EmptyMutableObjectIterator.java b/flink-runtime/src/main/java/org/apache/flink/runtime/util/EmptyMutableObjectIterator.java index 71fb30f21ef88..12ae5c1883c67 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/util/EmptyMutableObjectIterator.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/util/EmptyMutableObjectIterator.java @@ -54,4 +54,15 @@ public static MutableObjectIterator get() { public E next(E target) { return null; } + + /** + * Always returns null. + * + * @see MutableObjectIterator#next() + */ + @Override + public E next() { + return null; + } + } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/util/KeyGroupedMutableObjectIterator.java b/flink-runtime/src/main/java/org/apache/flink/runtime/util/KeyGroupedMutableObjectIterator.java index 88097cdb4c319..c139aca8a3282 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/util/KeyGroupedMutableObjectIterator.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/util/KeyGroupedMutableObjectIterator.java @@ -167,5 +167,42 @@ public E next(E target) ioex.getMessage(), ioex); } } + + @Override + public E next() + { + if (KeyGroupedMutableObjectIterator.this.next == null || KeyGroupedMutableObjectIterator.this.nextIsFresh) { + return null; + } + if (this.nextIsUnconsumed) { + return this.serializer.copy(KeyGroupedMutableObjectIterator.this.next); + } + + E result = null; + try { + if ((result = KeyGroupedMutableObjectIterator.this.iterator.next()) != null) { + // check whether the keys are equal + if (!this.comparator.equalToReference(result)) { + // moved to the next key, no more values here + KeyGroupedMutableObjectIterator.this.next = + this.serializer.copy(result); + KeyGroupedMutableObjectIterator.this.nextIsFresh = true; + return null; + } + // same key, next value is in "next" + return result; + } + else { + // backing iterator is consumed + KeyGroupedMutableObjectIterator.this.next = null; + return null; + } + } + catch (IOException ioex) { + throw new RuntimeException("An error occurred while reading the next record: " + + ioex.getMessage(), ioex); + } + } + } } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/util/RegularToMutableObjectIterator.java b/flink-runtime/src/main/java/org/apache/flink/runtime/util/RegularToMutableObjectIterator.java index f2fea80fb363d..8eb17c43726be 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/util/RegularToMutableObjectIterator.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/util/RegularToMutableObjectIterator.java @@ -47,4 +47,13 @@ public T next(T reuse) { return null; } } + + @Override + public T next() { + if (this.iterator.hasNext()) { + return this.serializer.copy(this.iterator.next()); + } else { + return null; + } + } } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/hash/HashTableITCase.java b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/hash/HashTableITCase.java index 877c5cccef3f3..27ece69d9880c 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/hash/HashTableITCase.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/hash/HashTableITCase.java @@ -1433,6 +1433,20 @@ public Record next(Record reuse) { return null; } } + + @Override + public Record next() { + if (this.numLeft > 0) { + this.numLeft--; + Record result = new Record(2); + result.setField(0, this.key); + result.setField(1, this.value); + return result; + } + else { + return null; + } + } } // ============================================================================================ @@ -1466,6 +1480,22 @@ public IntPair next(IntPair reuse) { return null; } } + + @Override + public IntPair next() { + if (this.numLeft > 0) { + this.numLeft--; + + IntPair result = new IntPair(); + result.setKey(this.key); + result.setValue(this.value); + return result; + } + else { + return null; + } + } + } // ============================================================================================ diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/sort/MassiveStringSortingITCase.java b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/sort/MassiveStringSortingITCase.java index 9dec847499a3b..8db9934d4cf53 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/sort/MassiveStringSortingITCase.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/sort/MassiveStringSortingITCase.java @@ -274,6 +274,11 @@ public StringReaderMutableObjectIterator(BufferedReader reader) { public String next(String reuse) throws IOException { return reader.readLine(); } + + @Override + public String next() throws IOException { + return reader.readLine(); + } } private static final class StringTupleReaderMutableObjectIterator implements MutableObjectIterator> { @@ -296,6 +301,20 @@ public Tuple2 next(Tuple2 reuse) throws IOEx reuse.f1 = parts; return reuse; } + + @Override + public Tuple2 next() throws IOException { + String line = reader.readLine(); + if (line == null) { + return null; + } + + String[] parts = line.split(" "); + Tuple2 result = new Tuple2(); + result.f0 = parts[0]; + result.f1 = parts; + return result; + } } // -------------------------------------------------------------------------------------------- diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/sort/MergeIteratorTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/sort/MergeIteratorTest.java index 7d681d8951a0e..c2214560e0aa9 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/sort/MergeIteratorTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/sort/MergeIteratorTest.java @@ -77,6 +77,23 @@ public Record next(Record reuse) return null; } } + + @Override + public Record next() + { + if (current < keys.length) { + key.setKey(keys[current]); + value.setValue(values[current]); + current++; + Record result = new Record(2); + result.setField(0, key); + result.setField(1, value); + return result; + } + else { + return null; + } + } }; } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/sort/MockRecordReader.java b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/sort/MockRecordReader.java index 738e7fc4b8ac4..e5b8ed092fb45 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/sort/MockRecordReader.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/sort/MockRecordReader.java @@ -65,6 +65,32 @@ public Record next(Record reuse) { } } + @Override + public Record next() { + Record r = null; + while (r == null) { + try { + r = queue.take(); + } catch (InterruptedException iex) { + throw new RuntimeException("Reader was interrupted."); + } + } + + if (r == SENTINEL) { + // put the sentinel back, to ensure that repeated calls do not block + try { + queue.put(r); + } catch (InterruptedException e) { + throw new RuntimeException("Reader was interrupted."); + } + return null; + } else { + Record result = new Record(r.getNumFields()); + r.copyTo(result); + return result; + } + } + public void emit(Record element) throws InterruptedException { queue.put(element.createCopy()); } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/testutils/DelayingInfinitiveInputIterator.java b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/testutils/DelayingInfinitiveInputIterator.java index 7e61a1819de81..f21d53f80919a 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/testutils/DelayingInfinitiveInputIterator.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/testutils/DelayingInfinitiveInputIterator.java @@ -37,5 +37,13 @@ public Record next(Record reuse) { catch (InterruptedException e) { } return super.next(reuse); } - + + @Override + public Record next() { + try { + Thread.sleep(delay); + } + catch (InterruptedException e) { } + return super.next(); + } } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/testutils/InfiniteInputIterator.java b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/testutils/InfiniteInputIterator.java index 2ae103b9ccf78..5c10f5e7b9b2c 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/testutils/InfiniteInputIterator.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/testutils/InfiniteInputIterator.java @@ -37,4 +37,13 @@ public Record next(Record reuse) { reuse.setField(1, val2); return reuse; } + + @Override + public Record next() { + Record result = new Record(2); + result.setField(0, val1); + result.setField(1, val2); + return result; + } + } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/testutils/MutableObjectIteratorWrapper.java b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/testutils/MutableObjectIteratorWrapper.java index 8a01f1214833d..88f16fc9aac6f 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/testutils/MutableObjectIteratorWrapper.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/testutils/MutableObjectIteratorWrapper.java @@ -50,4 +50,17 @@ public Record next(Record reuse) throws IOException { } } + @Override + public Record next() throws IOException { + if (this.source.hasNext()) { + Record result = new Record(); + this.source.next().copyTo(result); + return result; + } + else { + return null; + } + } + + } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/testutils/RandomIntPairGenerator.java b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/testutils/RandomIntPairGenerator.java index 57b53518b3c0f..48a512ce55141 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/testutils/RandomIntPairGenerator.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/testutils/RandomIntPairGenerator.java @@ -56,6 +56,18 @@ public IntPair next(IntPair reuse) { return null; } } + + @Override + public IntPair next() { + if (this.count++ < this.numRecords) { + IntPair result = new IntPair(); + result.setKey(this.rnd.nextInt()); + result.setValue(this.rnd.nextInt()); + return result; + } else { + return null; + } + } public void reset() { this.rnd = new Random(this.seed); diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/testutils/TestData.java b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/testutils/TestData.java index c28e542b078ae..5fe1303a89ca8 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/testutils/TestData.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/testutils/TestData.java @@ -181,6 +181,17 @@ public Record next(Record reuse) { return reuse; } + public Record next() { + this.key.setKey(keyMode == KeyMode.SORTED ? ++counter : Math.abs(random.nextInt() % keyMax) + 1); + if (this.valueMode != ValueMode.CONSTANT) { + this.value.setValue(randomString()); + } + Record result = new Record(2); + result.setField(0, this.key); + result.setField(1, this.value); + return result; + } + public boolean next(org.apache.flink.types.Value[] target) { this.key.setKey(keyMode == KeyMode.SORTED ? ++counter : Math.abs(random.nextInt() % keyMax) + 1); // TODO change this to something proper @@ -264,6 +275,17 @@ public Record next(Record target) { return null; } } + + @Override + public Record next() { + if (counter < numberOfRecords) { + counter++; + return generator.next(); + } + else { + return null; + } + } public void reset() { this.counter = 0; @@ -306,7 +328,23 @@ public Record next(Record reuse) { return null; } } - + + @Override + public Record next() { + if (pos < this.numPairs) { + this.value.setValue(this.valueValue + ' ' + pos); + Record result = new Record(2); + result.setField(0, this.key); + result.setField(1, this.value); + pos++; + return result; + } + else { + return null; + } + } + + public void reset() { this.pos = 0; } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/testutils/UniformIntPairGenerator.java b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/testutils/UniformIntPairGenerator.java index 5d820b952a5c3..a7e2a7c0cb7b5 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/testutils/UniformIntPairGenerator.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/testutils/UniformIntPairGenerator.java @@ -67,4 +67,37 @@ public IntPair next(IntPair target) { return target; } + + @Override + public IntPair next() { + IntPair result = new IntPair(); + if(!repeatKey) { + if(valCnt >= numVals) { + return null; + } + + result.setKey(keyCnt++); + result.setValue(valCnt); + + if(keyCnt == numKeys) { + keyCnt = 0; + valCnt++; + } + } else { + if(keyCnt >= numKeys) { + return null; + } + + result.setKey(keyCnt); + result.setValue(valCnt++); + + if(valCnt == numVals) { + valCnt = 0; + keyCnt++; + } + } + + return result; + } + } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/testutils/UniformRecordGenerator.java b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/testutils/UniformRecordGenerator.java index 8e35053054634..b628f059e14ae 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/testutils/UniformRecordGenerator.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/testutils/UniformRecordGenerator.java @@ -81,4 +81,38 @@ public Record next(Record reuse) { reuse.updateBinaryRepresenation(); return reuse; } + + @Override + public Record next() { + if(!repeatKey) { + if(valCnt >= numVals+startVal) { + return null; + } + + key.setValue(keyCnt++); + value.setValue(valCnt); + + if(keyCnt == numKeys+startKey) { + keyCnt = startKey; + valCnt++; + } + } else { + if(keyCnt >= numKeys+startKey) { + return null; + } + key.setValue(keyCnt); + value.setValue(valCnt++); + + if(valCnt == numVals+startVal) { + valCnt = startVal; + keyCnt++; + } + } + + Record result = new Record(2); + result.setField(0, this.key); + result.setField(1, this.value); + result.updateBinaryRepresenation(); + return result; + } } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/testutils/UniformStringPairGenerator.java b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/testutils/UniformStringPairGenerator.java index ef697a8de803b..45a44facb4440 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/testutils/UniformStringPairGenerator.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/testutils/UniformStringPairGenerator.java @@ -69,4 +69,37 @@ public StringPair next(StringPair target) throws IOException { return target; } + @Override + public StringPair next() throws IOException { + StringPair result = new StringPair(); + if(!repeatKey) { + if(valCnt >= numVals) { + return null; + } + + result.setKey(Integer.toString(keyCnt++)); + result.setValue(Integer.toBinaryString(valCnt)); + + if(keyCnt == numKeys) { + keyCnt = 0; + valCnt++; + } + } else { + if(keyCnt >= numKeys) { + return null; + } + + result.setKey(Integer.toString(keyCnt)); + result.setValue(Integer.toBinaryString(valCnt++)); + + if(valCnt == numVals) { + valCnt = 0; + keyCnt++; + } + } + + return result; + } + + } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/testutils/UnionIterator.java b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/testutils/UnionIterator.java index 824e96bde1e52..3a76ebdd6c279 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/testutils/UnionIterator.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/testutils/UnionIterator.java @@ -57,4 +57,22 @@ public E next(E target) throws IOException } } } + + @Override + public E next() throws IOException + { + E targetStaging = this.currentSource.next(); + if (targetStaging != null) { + return targetStaging; + } else { + if (this.nextSources.size() > 0) { + this.currentSource = this.nextSources.remove(0); + return next(); + } + else { + return null; + } + } + } + } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/util/KeyGroupedIteratorImmutableTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/util/KeyGroupedIteratorImmutableTest.java index 54a1492573717..3d1a80b55980a 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/util/KeyGroupedIteratorImmutableTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/util/KeyGroupedIteratorImmutableTest.java @@ -78,6 +78,20 @@ public Record next(Record reuse) throws IOException { return null; } } + + @Override + public Record next() throws IOException { + if (it.hasNext()) { + IntStringPair pair = it.next(); + Record result = new Record(2); + result.setField(0, pair.getInteger()); + result.setField(1, pair.getString()); + return result; + } + else { + return null; + } + } }; final RecordSerializer serializer = RecordSerializer.get(); diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/util/KeyGroupedIteratorTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/util/KeyGroupedIteratorTest.java index 85ca9a98813e7..39ff077e8e198 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/util/KeyGroupedIteratorTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/util/KeyGroupedIteratorTest.java @@ -79,6 +79,21 @@ public Record next(Record reuse) throws IOException { return null; } } + + @Override + public Record next() throws IOException { + if (it.hasNext()) { + IntStringPair pair = it.next(); + Record result = new Record(2); + result.setField(0, pair.getInteger()); + result.setField(1, pair.getString()); + return result; + } + else { + return null; + } + } + }; final RecordSerializer serializer = RecordSerializer.get(); diff --git a/flink-tests/src/test/scala/org/apache/flink/api/scala/misc/MassiveCaseClassSortingITCase.scala b/flink-tests/src/test/scala/org/apache/flink/api/scala/misc/MassiveCaseClassSortingITCase.scala index a34c7d8e0d598..7c6bcafcb73aa 100644 --- a/flink-tests/src/test/scala/org/apache/flink/api/scala/misc/MassiveCaseClassSortingITCase.scala +++ b/flink-tests/src/test/scala/org/apache/flink/api/scala/misc/MassiveCaseClassSortingITCase.scala @@ -216,6 +216,16 @@ class StringTupleReader(val reader: BufferedReader) extends MutableObjectIterato val parts = line.split(" ") StringTuple(parts(0), parts(1), parts) } + + override def next(): StringTuple = { + val line = reader.readLine() + if (line == null) { + return null + } + val parts = line.split(" ") + StringTuple(parts(0), parts(1), parts) + } + } class DummyInvokable extends AbstractInvokable { From b7b32a050e4b77012b9b70bb98b2a22d293dfbad Mon Sep 17 00:00:00 2001 From: Aljoscha Krettek Date: Mon, 15 Dec 2014 18:40:11 +0100 Subject: [PATCH 2/4] [FLINK-1285] Make execution mode configurable --- .../NepheleJobGraphGenerator.java | 14 +- .../flink/api/common/ExecutionConfig.java | 41 +- .../org/apache/flink/api/common/Plan.java | 42 +- .../flink/api/java/CollectionEnvironment.java | 7 +- .../flink/api/java/ExecutionEnvironment.java | 1 + .../typeutils/runtime/TupleSerializer.java | 1 + .../AbstractCachedBuildSideMatchDriver.java | 141 ++-- .../operators/AllGroupReduceDriver.java | 56 +- .../runtime/operators/AllReduceDriver.java | 50 +- .../runtime/operators/CoGroupDriver.java | 34 +- .../CoGroupWithSolutionSetFirstDriver.java | 121 ++- .../CoGroupWithSolutionSetSecondDriver.java | 122 ++- .../runtime/operators/CollectorMapDriver.java | 34 +- .../flink/runtime/operators/CrossDriver.java | 211 +++-- .../flink/runtime/operators/DataSinkTask.java | 44 +- .../runtime/operators/DataSourceTask.java | 88 +- .../runtime/operators/FlatMapDriver.java | 35 +- .../operators/GroupReduceCombineDriver.java | 49 +- .../runtime/operators/GroupReduceDriver.java | 26 +- .../JoinWithSolutionSetFirstDriver.java | 93 ++- .../JoinWithSolutionSetSecondDriver.java | 90 +- .../flink/runtime/operators/MapDriver.java | 6 + .../runtime/operators/MapPartitionDriver.java | 28 +- .../flink/runtime/operators/MatchDriver.java | 68 +- .../flink/runtime/operators/NoOpDriver.java | 38 +- .../runtime/operators/PactTaskContext.java | 5 +- .../operators/ReduceCombineDriver.java | 85 +- .../flink/runtime/operators/ReduceDriver.java | 80 +- .../runtime/operators/RegularPactTask.java | 26 +- .../operators/chaining/ChainedDriver.java | 10 +- .../SynchronousChainedCombineDriver.java | 52 +- .../operators/hash/CompactingHashTable.java | 83 +- .../operators/hash/HashMatchIteratorBase.java | 51 ++ .../operators/hash/InMemoryPartition.java | 5 + ...NonReusingBuildFirstHashMatchIterator.java | 152 ++++ ...BuildFirstReOpenableHashMatchIterator.java | 80 ++ ...onReusingBuildSecondHashMatchIterator.java | 150 ++++ ...uildSecondReOpenableHashMatchIterator.java | 81 ++ ...> ReusingBuildFirstHashMatchIterator.java} | 58 +- ...uildFirstReOpenableHashMatchIterator.java} | 10 +- ... ReusingBuildSecondHashMatchIterator.java} | 46 +- ...ildSecondReOpenableHashMatchIterator.java} | 10 +- .../sort/CombiningUnilateralSortMerger.java | 4 +- .../sort/FixedLengthRecordSorter.java | 4 +- .../operators/sort/MergeMatchIterator.java | 14 +- ...> NonReusingSortMergeCoGroupIterator.java} | 48 +- .../sort/ReusingSortMergeCoGroupIterator.java | 158 ++++ ...java => NonReusingKeyGroupedIterator.java} | 16 +- ...eusingMutableToRegularIteratorWrapper.java | 96 +++ ...or.java => ReusingKeyGroupedIterator.java} | 37 +- ...usingMutableToRegularIteratorWrapper.java} | 5 +- .../operators/CachedMatchTaskTest.java | 5 +- .../operators/CoGroupTaskExternalITCase.java | 5 +- .../runtime/operators/CoGroupTaskTest.java | 5 +- .../operators/CombineTaskExternalITCase.java | 5 +- .../runtime/operators/CombineTaskTest.java | 5 +- .../operators/CrossTaskExternalITCase.java | 5 +- .../runtime/operators/CrossTaskTest.java | 5 +- .../flink/runtime/operators/MapTaskTest.java | 5 +- .../operators/MatchTaskExternalITCase.java | 5 +- .../runtime/operators/MatchTaskTest.java | 5 +- .../operators/ReduceTaskExternalITCase.java | 5 +- .../runtime/operators/ReduceTaskTest.java | 5 +- .../operators/drivers/TestTaskContext.java | 8 + .../NonReusingHashMatchIteratorITCase.java | 778 ++++++++++++++++++ .../NonReusingReOpenableHashTableITCase.java | 533 ++++++++++++ ...va => ReusingHashMatchIteratorITCase.java} | 26 +- ... => ReusingReOpenableHashTableITCase.java} | 14 +- .../CombiningUnilateralSortMergerITCase.java | 4 +- .../sort/MassiveStringValueSortingITCase.java | 10 + ...ReusingSortMergeCoGroupIteratorITCase.java | 227 +++++ ...eusingSortMergeCoGroupIteratorITCase.java} | 4 +- .../operators/testutils/DriverTestBase.java | 73 +- .../operators/testutils/MockEnvironment.java | 10 +- .../util/HashVsSortMiniBenchmark.java | 12 +- ... => NonReusingKeyGroupedIteratorTest.java} | 6 +- ...ava => ReusingKeyGroupedIteratorTest.java} | 7 +- .../flink/test/util/JavaProgramTestBase.java | 51 +- .../CoGroupConnectedComponentsITCase.java | 4 +- .../DependencyConnectedComponentsITCase.java | 3 + .../test/javaApiOperators/ReduceITCase.java | 37 - .../test/operators/ObjectReuseITCase.java | 247 ++++++ 82 files changed, 4168 insertions(+), 752 deletions(-) create mode 100644 flink-runtime/src/main/java/org/apache/flink/runtime/operators/hash/HashMatchIteratorBase.java create mode 100644 flink-runtime/src/main/java/org/apache/flink/runtime/operators/hash/NonReusingBuildFirstHashMatchIterator.java create mode 100644 flink-runtime/src/main/java/org/apache/flink/runtime/operators/hash/NonReusingBuildFirstReOpenableHashMatchIterator.java create mode 100644 flink-runtime/src/main/java/org/apache/flink/runtime/operators/hash/NonReusingBuildSecondHashMatchIterator.java create mode 100644 flink-runtime/src/main/java/org/apache/flink/runtime/operators/hash/NonReusingBuildSecondReOpenableHashMatchIterator.java rename flink-runtime/src/main/java/org/apache/flink/runtime/operators/hash/{BuildFirstHashMatchIterator.java => ReusingBuildFirstHashMatchIterator.java} (69%) rename flink-runtime/src/main/java/org/apache/flink/runtime/operators/hash/{BuildFirstReOpenableHashMatchIterator.java => ReusingBuildFirstReOpenableHashMatchIterator.java} (90%) rename flink-runtime/src/main/java/org/apache/flink/runtime/operators/hash/{BuildSecondHashMatchIterator.java => ReusingBuildSecondHashMatchIterator.java} (72%) rename flink-runtime/src/main/java/org/apache/flink/runtime/operators/hash/{BuildSecondReOpenableHashMatchIterator.java => ReusingBuildSecondReOpenableHashMatchIterator.java} (90%) rename flink-runtime/src/main/java/org/apache/flink/runtime/operators/sort/{SortMergeCoGroupIterator.java => NonReusingSortMergeCoGroupIterator.java} (86%) create mode 100644 flink-runtime/src/main/java/org/apache/flink/runtime/operators/sort/ReusingSortMergeCoGroupIterator.java rename flink-runtime/src/main/java/org/apache/flink/runtime/util/{KeyGroupedIteratorImmutable.java => NonReusingKeyGroupedIterator.java} (92%) create mode 100644 flink-runtime/src/main/java/org/apache/flink/runtime/util/NonReusingMutableToRegularIteratorWrapper.java rename flink-runtime/src/main/java/org/apache/flink/runtime/util/{KeyGroupedIterator.java => ReusingKeyGroupedIterator.java} (86%) rename flink-runtime/src/main/java/org/apache/flink/runtime/util/{MutableToRegularIteratorWrapper.java => ReusingMutableToRegularIteratorWrapper.java} (92%) create mode 100644 flink-runtime/src/test/java/org/apache/flink/runtime/operators/hash/NonReusingHashMatchIteratorITCase.java create mode 100644 flink-runtime/src/test/java/org/apache/flink/runtime/operators/hash/NonReusingReOpenableHashTableITCase.java rename flink-runtime/src/test/java/org/apache/flink/runtime/operators/hash/{HashMatchIteratorITCase.java => ReusingHashMatchIteratorITCase.java} (96%) rename flink-runtime/src/test/java/org/apache/flink/runtime/operators/hash/{ReOpenableHashTableITCase.java => ReusingReOpenableHashTableITCase.java} (97%) create mode 100644 flink-runtime/src/test/java/org/apache/flink/runtime/operators/sort/NonReusingSortMergeCoGroupIteratorITCase.java rename flink-runtime/src/test/java/org/apache/flink/runtime/operators/sort/{SortMergeCoGroupIteratorITCase.java => ReusingSortMergeCoGroupIteratorITCase.java} (97%) rename flink-runtime/src/test/java/org/apache/flink/runtime/util/{KeyGroupedIteratorImmutableTest.java => NonReusingKeyGroupedIteratorTest.java} (98%) rename flink-runtime/src/test/java/org/apache/flink/runtime/util/{KeyGroupedIteratorTest.java => ReusingKeyGroupedIteratorTest.java} (98%) create mode 100644 flink-tests/src/test/java/org/apache/flink/test/operators/ObjectReuseITCase.java diff --git a/flink-compiler/src/main/java/org/apache/flink/compiler/plantranslate/NepheleJobGraphGenerator.java b/flink-compiler/src/main/java/org/apache/flink/compiler/plantranslate/NepheleJobGraphGenerator.java index 9c2efb3d8d4e0..260b30cf1aecc 100644 --- a/flink-compiler/src/main/java/org/apache/flink/compiler/plantranslate/NepheleJobGraphGenerator.java +++ b/flink-compiler/src/main/java/org/apache/flink/compiler/plantranslate/NepheleJobGraphGenerator.java @@ -18,6 +18,7 @@ package org.apache.flink.compiler.plantranslate; +import java.io.IOException; import java.util.ArrayList; import java.util.Collection; import java.util.Collections; @@ -27,6 +28,7 @@ import java.util.Map; import java.util.Map.Entry; +import org.apache.flink.api.common.ExecutionConfig; import org.apache.flink.api.common.aggregators.AggregatorRegistry; import org.apache.flink.api.common.aggregators.AggregatorWithName; import org.apache.flink.api.common.aggregators.ConvergenceCriterion; @@ -82,6 +84,7 @@ import org.apache.flink.runtime.operators.shipping.ShipStrategyType; import org.apache.flink.runtime.operators.util.LocalStrategy; import org.apache.flink.runtime.operators.util.TaskConfig; +import org.apache.flink.util.InstantiationUtil; import org.apache.flink.util.Visitor; /** @@ -209,7 +212,16 @@ public JobGraph compileJobGraph(OptimizedPlan program) { for (Entry e : program.getOriginalPactPlan().getCachedFiles()) { DistributedCache.writeFileInfoToConfig(e.getKey(), e.getValue(), graph.getJobConfiguration()); } - + + try { + InstantiationUtil.writeObjectToConfig( + program.getOriginalPactPlan().getExecutionConfig(), + graph.getJobConfiguration(), + ExecutionConfig.CONFIG_KEY); + } catch (IOException e) { + throw new RuntimeException("Config object could not be written to Job Configuration: " + e); + } + // release all references again this.vertices = null; this.chainedTasks = null; diff --git a/flink-core/src/main/java/org/apache/flink/api/common/ExecutionConfig.java b/flink-core/src/main/java/org/apache/flink/api/common/ExecutionConfig.java index 03d5e3a13fcc2..17e683f0b40f3 100644 --- a/flink-core/src/main/java/org/apache/flink/api/common/ExecutionConfig.java +++ b/flink-core/src/main/java/org/apache/flink/api/common/ExecutionConfig.java @@ -37,7 +37,7 @@ public class ExecutionConfig implements Serializable { // For future use... // private boolean forceGenericSerializer = false; -// private boolean objectReuse = false; + private boolean objectReuse = false; /** * Enables the ClosureCleaner. This analyzes user code functions and sets fields to null @@ -143,17 +143,30 @@ public ExecutionConfig setNumberOfExecutionRetries(int numberOfExecutionRetries) // return forceGenericSerializer; // } // -// public ExecutionConfig enableObjectReuse() { -// objectReuse = true; -// return this; -// } -// -// public ExecutionConfig disableObjectReuse() { -// objectReuse = false; -// return this; -// } -// -// public boolean isObjectReuseEnabled() { -// return objectReuse; -// } + + /** + * Enables reusing objects that Flink internally uses for deserialization and passing + * data to user-code functions. Keep in mind that this can lead to bugs when the + * user-code function of an operation is not aware of this behaviour. + */ + public ExecutionConfig enableObjectReuse() { + objectReuse = true; + return this; + } + + /** + * Disables reusing objects that Flink internally uses for deserialization and passing + * data to user-code functions. @see #enableObjectReuse() + */ + public ExecutionConfig disableObjectReuse() { + objectReuse = false; + return this; + } + + /** + * Returns whether object reuse has been enabled or disabled. @see #enableObjectReuse() + */ + public boolean isObjectReuseEnabled() { + return objectReuse; + } } diff --git a/flink-core/src/main/java/org/apache/flink/api/common/Plan.java b/flink-core/src/main/java/org/apache/flink/api/common/Plan.java index f299ef4c4df64..4a975d25c24ac 100644 --- a/flink-core/src/main/java/org/apache/flink/api/common/Plan.java +++ b/flink-core/src/main/java/org/apache/flink/api/common/Plan.java @@ -69,14 +69,14 @@ public class Plan implements Visitable> { protected int defaultParallelism = DEFAULT_PARALELLISM; /** - * The number of times failed tasks are re-executed. + * Hash map for files in the distributed cache: registered name to cache entry. */ - protected int numberOfExecutionRetries; + protected HashMap cacheFile = new HashMap(); /** - * Hash map for files in the distributed cache: registered name to cache entry. + * Config object for runtime execution parameters. */ - protected HashMap cacheFile = new HashMap(); + protected ExecutionConfig executionConfig = new ExecutionConfig(); // ------------------------------------------------------------------------ @@ -263,20 +263,6 @@ public void setDefaultParallelism(int defaultParallelism) { this.defaultParallelism = defaultParallelism; } - /** - * Sets the number of times that failed tasks are re-executed. A value of zero - * effectively disables fault tolerance. A value of {@code -1} indicates that the system - * default value (as defined in the configuration) should be used. - * - * @param numberOfExecutionRetries The number of times the system will try to re-execute failed tasks. - */ - public void setNumberOfExecutionRetries(int numberOfExecutionRetries) { - if (numberOfExecutionRetries < -1) { - throw new IllegalArgumentException("The number of execution retries must be non-negative, or -1 (use system default)"); - } - this.numberOfExecutionRetries = numberOfExecutionRetries; - } - /** * Gets the number of times the system will try to re-execute failed tasks. A value * of {@code -1} indicates that the system default value (as defined in the configuration) @@ -285,7 +271,7 @@ public void setNumberOfExecutionRetries(int numberOfExecutionRetries) { * @return The number of times the system will try to re-execute failed tasks. */ public int getNumberOfExecutionRetries() { - return numberOfExecutionRetries; + return executionConfig.getNumberOfExecutionRetries(); } /** @@ -297,7 +283,23 @@ public int getNumberOfExecutionRetries() { public String getPostPassClassName() { return "org.apache.flink.compiler.postpass.RecordModelPostPass"; } - + + /** + * Sets the runtime config object. + * @return + */ + public ExecutionConfig getExecutionConfig() { + return executionConfig; + } + + /** + * Gets the runtime config object. + * @param executionConfig + */ + public void setExecutionConfig(ExecutionConfig executionConfig) { + this.executionConfig = executionConfig; + } + // ------------------------------------------------------------------------ /** diff --git a/flink-java/src/main/java/org/apache/flink/api/java/CollectionEnvironment.java b/flink-java/src/main/java/org/apache/flink/api/java/CollectionEnvironment.java index 3e9ff663eab93..2d5749024d170 100644 --- a/flink-java/src/main/java/org/apache/flink/api/java/CollectionEnvironment.java +++ b/flink-java/src/main/java/org/apache/flink/api/java/CollectionEnvironment.java @@ -24,13 +24,12 @@ public class CollectionEnvironment extends ExecutionEnvironment { - private boolean mutableObjectSafeMode = true; - @Override public JobExecutionResult execute(String jobName) throws Exception { Plan p = createProgramPlan(jobName); - - CollectionExecutor exec = new CollectionExecutor(mutableObjectSafeMode); + + // We need to reverse here. Object-Reuse enabled, means safe mode is disabled. + CollectionExecutor exec = new CollectionExecutor(!getConfig().isObjectReuseEnabled()); return exec.execute(p); } diff --git a/flink-java/src/main/java/org/apache/flink/api/java/ExecutionEnvironment.java b/flink-java/src/main/java/org/apache/flink/api/java/ExecutionEnvironment.java index c19e9aa7c9a1d..f7057ef3905d6 100644 --- a/flink-java/src/main/java/org/apache/flink/api/java/ExecutionEnvironment.java +++ b/flink-java/src/main/java/org/apache/flink/api/java/ExecutionEnvironment.java @@ -749,6 +749,7 @@ public JavaPlan createProgramPlan(String jobName, boolean clearSinks) { if (getDegreeOfParallelism() > 0) { plan.setDefaultParallelism(getDegreeOfParallelism()); } + plan.setExecutionConfig(getConfig()); try { registerCachedFilesWithPlan(plan); diff --git a/flink-java/src/main/java/org/apache/flink/api/java/typeutils/runtime/TupleSerializer.java b/flink-java/src/main/java/org/apache/flink/api/java/typeutils/runtime/TupleSerializer.java index ae429a773fb8b..9564c0101b4f4 100644 --- a/flink-java/src/main/java/org/apache/flink/api/java/typeutils/runtime/TupleSerializer.java +++ b/flink-java/src/main/java/org/apache/flink/api/java/typeutils/runtime/TupleSerializer.java @@ -53,6 +53,7 @@ public T createInstance() { @Override public T createInstance(Object[] fields) { + try { T t = tupleClass.newInstance(); diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/AbstractCachedBuildSideMatchDriver.java b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/AbstractCachedBuildSideMatchDriver.java index 286d8308c2544..f3b2dfd1a2420 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/AbstractCachedBuildSideMatchDriver.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/AbstractCachedBuildSideMatchDriver.java @@ -19,12 +19,15 @@ package org.apache.flink.runtime.operators; +import org.apache.flink.api.common.ExecutionConfig; import org.apache.flink.api.common.functions.FlatJoinFunction; import org.apache.flink.api.common.typeutils.TypeComparator; import org.apache.flink.api.common.typeutils.TypePairComparatorFactory; import org.apache.flink.api.common.typeutils.TypeSerializer; -import org.apache.flink.runtime.operators.hash.BuildFirstReOpenableHashMatchIterator; -import org.apache.flink.runtime.operators.hash.BuildSecondReOpenableHashMatchIterator; +import org.apache.flink.runtime.operators.hash.NonReusingBuildFirstReOpenableHashMatchIterator; +import org.apache.flink.runtime.operators.hash.NonReusingBuildSecondReOpenableHashMatchIterator; +import org.apache.flink.runtime.operators.hash.ReusingBuildFirstReOpenableHashMatchIterator; +import org.apache.flink.runtime.operators.hash.ReusingBuildSecondReOpenableHashMatchIterator; import org.apache.flink.runtime.operators.util.JoinTaskIterator; import org.apache.flink.runtime.operators.util.TaskConfig; import org.apache.flink.util.Collector; @@ -37,7 +40,10 @@ public abstract class AbstractCachedBuildSideMatchDriver extends M private final int buildSideIndex; private final int probeSideIndex; - + + private boolean objectReuseEnabled = false; + + protected AbstractCachedBuildSideMatchDriver(int buildSideIndex, int probeSideIndex) { this.buildSideIndex = buildSideIndex; this.probeSideIndex = probeSideIndex; @@ -69,34 +75,67 @@ public void initialize() throws Exception { double availableMemory = config.getRelativeMemoryDriver(); - if (buildSideIndex == 0 && probeSideIndex == 1) { - - matchIterator = - new BuildFirstReOpenableHashMatchIterator(input1, input2, - serializer1, comparator1, - serializer2, comparator2, - pairComparatorFactory.createComparator21(comparator1, comparator2), - this.taskContext.getMemoryManager(), - this.taskContext.getIOManager(), - this.taskContext.getOwningNepheleTask(), - availableMemory - ); - - } else if (buildSideIndex == 1 && probeSideIndex == 0) { - - matchIterator = - new BuildSecondReOpenableHashMatchIterator(input1, input2, - serializer1, comparator1, - serializer2, comparator2, - pairComparatorFactory.createComparator12(comparator1, comparator2), - this.taskContext.getMemoryManager(), - this.taskContext.getIOManager(), - this.taskContext.getOwningNepheleTask(), - availableMemory - ); - + ExecutionConfig executionConfig = taskContext.getExecutionConfig(); + objectReuseEnabled = executionConfig.isObjectReuseEnabled(); + + if (objectReuseEnabled) { + if (buildSideIndex == 0 && probeSideIndex == 1) { + + matchIterator = new ReusingBuildFirstReOpenableHashMatchIterator( + input1, input2, + serializer1, comparator1, + serializer2, comparator2, + pairComparatorFactory.createComparator21(comparator1, comparator2), + this.taskContext.getMemoryManager(), + this.taskContext.getIOManager(), + this.taskContext.getOwningNepheleTask(), + availableMemory); + + + } else if (buildSideIndex == 1 && probeSideIndex == 0) { + + matchIterator = new ReusingBuildSecondReOpenableHashMatchIterator( + input1, input2, + serializer1, comparator1, + serializer2, comparator2, + pairComparatorFactory.createComparator12(comparator1, comparator2), + this.taskContext.getMemoryManager(), + this.taskContext.getIOManager(), + this.taskContext.getOwningNepheleTask(), + availableMemory); + + } else { + throw new Exception("Error: Inconsistent setup for repeatable hash join driver."); + } } else { - throw new Exception("Error: Inconcistent setup for repeatable hash join driver."); + if (buildSideIndex == 0 && probeSideIndex == 1) { + + matchIterator = new NonReusingBuildFirstReOpenableHashMatchIterator( + input1, input2, + serializer1, comparator1, + serializer2, comparator2, + pairComparatorFactory.createComparator21(comparator1, comparator2), + this.taskContext.getMemoryManager(), + this.taskContext.getIOManager(), + this.taskContext.getOwningNepheleTask(), + availableMemory); + + + } else if (buildSideIndex == 1 && probeSideIndex == 0) { + + matchIterator = new NonReusingBuildSecondReOpenableHashMatchIterator( + input1, input2, + serializer1, comparator1, + serializer2, comparator2, + pairComparatorFactory.createComparator12(comparator1, comparator2), + this.taskContext.getMemoryManager(), + this.taskContext.getIOManager(), + this.taskContext.getOwningNepheleTask(), + availableMemory); + + } else { + throw new Exception("Error: Inconsistent setup for repeatable hash join driver."); + } } this.matchIterator.open(); @@ -113,21 +152,8 @@ public void run() throws Exception { final FlatJoinFunction matchStub = this.taskContext.getStub(); final Collector collector = this.taskContext.getOutputCollector(); - if (buildSideIndex == 0) { - - final BuildFirstReOpenableHashMatchIterator matchIterator = (BuildFirstReOpenableHashMatchIterator) this.matchIterator; - - while (this.running && matchIterator != null && matchIterator.callWithNextKey(matchStub, collector)); - - } else if (buildSideIndex == 1) { + while (this.running && matchIterator != null && matchIterator.callWithNextKey(matchStub, collector)); - final BuildSecondReOpenableHashMatchIterator matchIterator = (BuildSecondReOpenableHashMatchIterator) this.matchIterator; - - while (this.running && matchIterator != null && matchIterator.callWithNextKey(matchStub, collector)); - - } else { - throw new Exception(); - } } @Override @@ -138,14 +164,25 @@ public void reset() throws Exception { MutableObjectIterator input1 = this.taskContext.getInput(0); MutableObjectIterator input2 = this.taskContext.getInput(1); - - if (buildSideIndex == 0 && probeSideIndex == 1) { - final BuildFirstReOpenableHashMatchIterator matchIterator = (BuildFirstReOpenableHashMatchIterator) this.matchIterator; - matchIterator.reopenProbe(input2); - } - else { - final BuildSecondReOpenableHashMatchIterator matchIterator = (BuildSecondReOpenableHashMatchIterator) this.matchIterator; - matchIterator.reopenProbe(input1); + + if (objectReuseEnabled) { + if (buildSideIndex == 0 && probeSideIndex == 1) { + final ReusingBuildFirstReOpenableHashMatchIterator matchIterator = (ReusingBuildFirstReOpenableHashMatchIterator) this.matchIterator; + + matchIterator.reopenProbe(input2); + } else { + final ReusingBuildSecondReOpenableHashMatchIterator matchIterator = (ReusingBuildSecondReOpenableHashMatchIterator) this.matchIterator; + matchIterator.reopenProbe(input1); + } + } else { + if (buildSideIndex == 0 && probeSideIndex == 1) { + final NonReusingBuildFirstReOpenableHashMatchIterator matchIterator = (NonReusingBuildFirstReOpenableHashMatchIterator) this.matchIterator; + + matchIterator.reopenProbe(input2); + } else { + final NonReusingBuildSecondReOpenableHashMatchIterator matchIterator = (NonReusingBuildSecondReOpenableHashMatchIterator) this.matchIterator; + matchIterator.reopenProbe(input1); + } } } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/AllGroupReduceDriver.java b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/AllGroupReduceDriver.java index 5e68babb63a65..854dbd572a612 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/AllGroupReduceDriver.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/AllGroupReduceDriver.java @@ -19,13 +19,15 @@ package org.apache.flink.runtime.operators; +import org.apache.flink.api.common.ExecutionConfig; +import org.apache.flink.runtime.util.NonReusingMutableToRegularIteratorWrapper; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import org.apache.flink.api.common.functions.FlatCombineFunction; import org.apache.flink.api.common.functions.GroupReduceFunction; import org.apache.flink.api.common.typeutils.TypeSerializer; import org.apache.flink.runtime.operators.util.TaskConfig; -import org.apache.flink.runtime.util.MutableToRegularIteratorWrapper; +import org.apache.flink.runtime.util.ReusingMutableToRegularIteratorWrapper; import org.apache.flink.util.Collector; import org.apache.flink.util.MutableObjectIterator; @@ -51,6 +53,8 @@ public class AllGroupReduceDriver implements PactDrivergetInputSerializer(0).getSerializer(); this.input = this.taskContext.getInput(0); + + ExecutionConfig executionConfig = taskContext.getExecutionConfig(); + this.objectReuseEnabled = executionConfig.isObjectReuseEnabled(); + + if (LOG.isDebugEnabled()) { + LOG.debug("AllGroupReduceDriver object reuse: " + (this.objectReuseEnabled ? "ENABLED" : "DISABLED") + "."); + } } @Override @@ -100,21 +111,36 @@ public void run() throws Exception { LOG.debug(this.taskContext.formatLogString("AllGroupReduce preprocessing done. Running Reducer code.")); } - final MutableToRegularIteratorWrapper inIter = new MutableToRegularIteratorWrapper(this.input, this.serializer); - - // single UDF call with the single group - if (inIter.hasNext()) { - if (strategy == DriverStrategy.ALL_GROUP_REDUCE) { - final GroupReduceFunction reducer = this.taskContext.getStub(); - final Collector output = this.taskContext.getOutputCollector(); - reducer.reduce(inIter, output); + if (objectReuseEnabled) { + final ReusingMutableToRegularIteratorWrapper inIter = new ReusingMutableToRegularIteratorWrapper(this.input, this.serializer); + + // single UDF call with the single group + if (inIter.hasNext()) { + if (strategy == DriverStrategy.ALL_GROUP_REDUCE) { + final GroupReduceFunction reducer = this.taskContext.getStub(); + final Collector output = this.taskContext.getOutputCollector(); + reducer.reduce(inIter, output); + } else { + @SuppressWarnings("unchecked") final FlatCombineFunction combiner = (FlatCombineFunction) this.taskContext.getStub(); + @SuppressWarnings("unchecked") final Collector output = (Collector) this.taskContext.getOutputCollector(); + combiner.combine(inIter, output); + } } - else { - @SuppressWarnings("unchecked") - final FlatCombineFunction combiner = (FlatCombineFunction) this.taskContext.getStub(); - @SuppressWarnings("unchecked") - final Collector output = (Collector) this.taskContext.getOutputCollector(); - combiner.combine(inIter, output); + + } else { + final NonReusingMutableToRegularIteratorWrapper inIter = new NonReusingMutableToRegularIteratorWrapper(this.input, this.serializer); + + // single UDF call with the single group + if (inIter.hasNext()) { + if (strategy == DriverStrategy.ALL_GROUP_REDUCE) { + final GroupReduceFunction reducer = this.taskContext.getStub(); + final Collector output = this.taskContext.getOutputCollector(); + reducer.reduce(inIter, output); + } else { + @SuppressWarnings("unchecked") final FlatCombineFunction combiner = (FlatCombineFunction) this.taskContext.getStub(); + @SuppressWarnings("unchecked") final Collector output = (Collector) this.taskContext.getOutputCollector(); + combiner.combine(inIter, output); + } } } } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/AllReduceDriver.java b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/AllReduceDriver.java index e26f4ebcf69d3..dff2dbdaddd81 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/AllReduceDriver.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/AllReduceDriver.java @@ -19,6 +19,7 @@ package org.apache.flink.runtime.operators; +import org.apache.flink.api.common.ExecutionConfig; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import org.apache.flink.api.common.functions.ReduceFunction; @@ -49,6 +50,8 @@ public class AllReduceDriver implements PactDriver, T> { private boolean running; + private boolean objectReuseEnabled = false; + // ------------------------------------------------------------------------ @Override @@ -86,6 +89,13 @@ public void prepare() throws Exception { TypeSerializerFactory serializerFactory = this.taskContext.getInputSerializer(0); this.serializer = serializerFactory.getSerializer(); this.input = this.taskContext.getInput(0); + + ExecutionConfig executionConfig = taskContext.getExecutionConfig(); + this.objectReuseEnabled = executionConfig.isObjectReuseEnabled(); + + if (LOG.isDebugEnabled()) { + LOG.debug("AllReduceDriver object reuse: " + (this.objectReuseEnabled ? "ENABLED" : "DISABLED") + "."); + } } @Override @@ -97,19 +107,35 @@ public void run() throws Exception { final ReduceFunction stub = this.taskContext.getStub(); final MutableObjectIterator input = this.input; final TypeSerializer serializer = this.serializer; - - T val1 = serializer.createInstance(); - - if ((val1 = input.next(val1)) == null) { - return; - } - - T val2; - while (running && (val2 = input.next(serializer.createInstance())) != null) { - val1 = stub.reduce(val1, val2); + + + if (objectReuseEnabled) { + T val1 = serializer.createInstance(); + + if ((val1 = input.next(val1)) == null) { + return; + } + + T val2 = serializer.createInstance(); + while (running && (val2 = input.next(val2)) != null) { + val1 = stub.reduce(val1, val2); + } + + this.taskContext.getOutputCollector().collect(val1); + } else { + T val1 = serializer.createInstance(); + + if ((val1 = input.next(val1)) == null) { + return; + } + + T val2; + while (running && (val2 = input.next(serializer.createInstance())) != null) { + val1 = stub.reduce(val1, val2); + } + + this.taskContext.getOutputCollector().collect(val1); } - - this.taskContext.getOutputCollector().collect(val1); } @Override diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/CoGroupDriver.java b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/CoGroupDriver.java index 55498fb399f00..6ace918f2702d 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/CoGroupDriver.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/CoGroupDriver.java @@ -19,13 +19,15 @@ package org.apache.flink.runtime.operators; +import org.apache.flink.api.common.ExecutionConfig; +import org.apache.flink.runtime.operators.sort.NonReusingSortMergeCoGroupIterator; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import org.apache.flink.api.common.functions.CoGroupFunction; import org.apache.flink.api.common.typeutils.TypeComparator; import org.apache.flink.api.common.typeutils.TypePairComparatorFactory; import org.apache.flink.api.common.typeutils.TypeSerializer; -import org.apache.flink.runtime.operators.sort.SortMergeCoGroupIterator; +import org.apache.flink.runtime.operators.sort.ReusingSortMergeCoGroupIterator; import org.apache.flink.runtime.operators.util.CoGroupTaskIterator; import org.apache.flink.runtime.operators.util.TaskConfig; import org.apache.flink.util.Collector; @@ -39,7 +41,7 @@ * The CoGroupTask group all pairs that share the same key from both inputs. Each for each key, the sets of values that * were pair with that key of both inputs are handed to the coGroup() method of the CoGroupFunction. * - * @see org.apache.flink.api.java.record.functions.CoGroupFunction + * @see org.apache.flink.api.common.functions.CoGroupFunction */ public class CoGroupDriver implements PactDriver, OT> { @@ -52,6 +54,8 @@ public class CoGroupDriver implements PactDriver(in1, in2, - serializer1, groupComparator1, serializer2, groupComparator2, - pairComparatorFactory.createComparator12(groupComparator1, groupComparator2)); + ExecutionConfig executionConfig = taskContext.getExecutionConfig(); + this.objectReuseEnabled = executionConfig.isObjectReuseEnabled(); + + if (LOG.isDebugEnabled()) { + LOG.debug("CoGroupDriver object reuse: " + (this.objectReuseEnabled ? "ENABLED" : "DISABLED") + "."); + } + + if (objectReuseEnabled) { + // create CoGropuTaskIterator according to provided local strategy. + this.coGroupIterator = new ReusingSortMergeCoGroupIterator( + in1, in2, + serializer1, groupComparator1, + serializer2, groupComparator2, + pairComparatorFactory.createComparator12(groupComparator1, groupComparator2)); + } else { + // create CoGropuTaskIterator according to provided local strategy. + this.coGroupIterator = new NonReusingSortMergeCoGroupIterator( + in1, in2, + serializer1, groupComparator1, + serializer2, groupComparator2, + pairComparatorFactory.createComparator12(groupComparator1, groupComparator2)); + } // open CoGroupTaskIterator - this triggers the sorting and blocks until the iterator is ready this.coGroupIterator.open(); diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/CoGroupWithSolutionSetFirstDriver.java b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/CoGroupWithSolutionSetFirstDriver.java index 5e0ca6dd0f508..a3c69a346c504 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/CoGroupWithSolutionSetFirstDriver.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/CoGroupWithSolutionSetFirstDriver.java @@ -20,6 +20,7 @@ import java.util.Collections; +import org.apache.flink.api.common.ExecutionConfig; import org.apache.flink.api.common.functions.CoGroupFunction; import org.apache.flink.api.common.operators.util.JoinHashMap; import org.apache.flink.api.common.typeutils.TypeComparator; @@ -31,7 +32,8 @@ import org.apache.flink.runtime.iterative.task.AbstractIterativePactTask; import org.apache.flink.runtime.operators.hash.CompactingHashTable; import org.apache.flink.runtime.operators.util.TaskConfig; -import org.apache.flink.runtime.util.KeyGroupedIterator; +import org.apache.flink.runtime.util.NonReusingKeyGroupedIterator; +import org.apache.flink.runtime.util.ReusingKeyGroupedIterator; import org.apache.flink.runtime.util.SingleElementIterator; import org.apache.flink.util.Collector; @@ -46,13 +48,18 @@ public class CoGroupWithSolutionSetFirstDriver implements Resettab private TypeSerializer probeSideSerializer; private TypeComparator probeSideComparator; - + + private TypeSerializer solutionSetSerializer; + + private TypePairComparator pairComparator; private IT1 solutionSideRecord; protected volatile boolean running; + private boolean objectReuseEnabled = false; + // -------------------------------------------------------------------------------------------- @Override @@ -96,7 +103,6 @@ public boolean isInputResettable(int inputNum) { @SuppressWarnings("unchecked") public void initialize() { - final TypeSerializer solutionSetSerializer; final TypeComparator solutionSetComparator; // grab a handle to the hash table from the iteration broker @@ -130,7 +136,12 @@ else if (table instanceof JoinHashMap) { this.probeSideSerializer = taskContext.getInputSerializer(0).getSerializer(); this.probeSideComparator = probeSideComparatorFactory.createComparator(); - solutionSideRecord = solutionSetSerializer.createInstance(); + ExecutionConfig executionConfig = taskContext.getExecutionConfig(); + objectReuseEnabled = executionConfig.isObjectReuseEnabled(); + + if (objectReuseEnabled) { + solutionSideRecord = solutionSetSerializer.createInstance(); + } TypePairComparatorFactory factory = taskContext.getTaskConfig().getPairComparatorFactory(taskContext.getUserCodeClassLoader()); pairComparator = factory.createComparator21(solutionSetComparator, this.probeSideComparator); @@ -149,46 +160,84 @@ public void run() throws Exception { final CoGroupFunction coGroupStub = taskContext.getStub(); final Collector collector = taskContext.getOutputCollector(); - final KeyGroupedIterator probeSideInput = new KeyGroupedIterator(taskContext.getInput(0), probeSideSerializer, probeSideComparator); final SingleElementIterator siIter = new SingleElementIterator(); final Iterable emptySolutionSide = Collections.emptySet(); - - if (this.hashTable != null) { - final CompactingHashTable join = hashTable; - final CompactingHashTable.HashTableProber prober = join.getProber(this.probeSideComparator, this.pairComparator); - - IT1 buildSideRecord = solutionSideRecord; - - while (this.running && probeSideInput.nextKey()) { - IT2 current = probeSideInput.getCurrent(); - - buildSideRecord = prober.getMatchFor(current, buildSideRecord); - if (buildSideRecord != null) { - siIter.set(buildSideRecord); - coGroupStub.coGroup(siIter, probeSideInput.getValues(), collector); + + if (objectReuseEnabled) { + final ReusingKeyGroupedIterator probeSideInput = new ReusingKeyGroupedIterator(taskContext.getInput(0), probeSideSerializer, probeSideComparator); + if (this.hashTable != null) { + final CompactingHashTable join = hashTable; + final CompactingHashTable.HashTableProber prober = join.getProber(this.probeSideComparator, this.pairComparator); + + + IT1 buildSideRecord = solutionSideRecord; + + while (this.running && probeSideInput.nextKey()) { + IT2 current = probeSideInput.getCurrent(); + + buildSideRecord = prober.getMatchFor(current, buildSideRecord); + if (buildSideRecord != null) { + siIter.set(buildSideRecord); + coGroupStub.coGroup(siIter, probeSideInput.getValues(), collector); + } else { + coGroupStub.coGroup(emptySolutionSide, probeSideInput.getValues(), collector); + } } - else { - coGroupStub.coGroup(emptySolutionSide, probeSideInput.getValues(), collector); + } else { + final JoinHashMap join = this.objectMap; + final JoinHashMap.Prober prober = join.createProber(this.probeSideComparator, this.pairComparator); + final TypeSerializer serializer = join.getBuildSerializer(); + + while (this.running && probeSideInput.nextKey()) { + IT2 current = probeSideInput.getCurrent(); + + IT1 buildSideRecord = prober.lookupMatch(current); + if (buildSideRecord != null) { + siIter.set(serializer.copy(buildSideRecord)); + coGroupStub.coGroup(siIter, probeSideInput.getValues(), collector); + } else { + coGroupStub.coGroup(emptySolutionSide, probeSideInput.getValues(), collector); + } } } - } - else { - final JoinHashMap join = this.objectMap; - final JoinHashMap.Prober prober = join.createProber(this.probeSideComparator, this.pairComparator); - final TypeSerializer serializer = join.getBuildSerializer(); - - while (this.running && probeSideInput.nextKey()) { - IT2 current = probeSideInput.getCurrent(); - - IT1 buildSideRecord = prober.lookupMatch(current); - if (buildSideRecord != null) { - siIter.set(serializer.copy(buildSideRecord)); - coGroupStub.coGroup(siIter, probeSideInput.getValues(), collector); + } else { + final NonReusingKeyGroupedIterator probeSideInput = new NonReusingKeyGroupedIterator(taskContext.getInput(0), probeSideSerializer, probeSideComparator); + if (this.hashTable != null) { + final CompactingHashTable join = hashTable; + final CompactingHashTable.HashTableProber prober = join.getProber(this + .probeSideComparator, this.pairComparator); + + IT1 buildSideRecord; + + while (this.running && probeSideInput.nextKey()) { + IT2 current = probeSideInput.getCurrent(); + + buildSideRecord = prober.getMatchFor(current); + if (buildSideRecord != null) { + siIter.set(solutionSetSerializer.copy(buildSideRecord)); + coGroupStub.coGroup(siIter, probeSideInput.getValues(), collector); + } else { + coGroupStub.coGroup(emptySolutionSide, probeSideInput.getValues(), collector); + } } - else { - coGroupStub.coGroup(emptySolutionSide, probeSideInput.getValues(), collector); + } else { + final JoinHashMap join = this.objectMap; + final JoinHashMap.Prober prober = join.createProber(this.probeSideComparator, this.pairComparator); + final TypeSerializer serializer = join.getBuildSerializer(); + + while (this.running && probeSideInput.nextKey()) { + IT2 current = probeSideInput.getCurrent(); + + IT1 buildSideRecord = prober.lookupMatch(current); + if (buildSideRecord != null) { + siIter.set(serializer.copy(buildSideRecord)); + coGroupStub.coGroup(siIter, probeSideInput.getValues(), collector); + } else { + coGroupStub.coGroup(emptySolutionSide, probeSideInput.getValues(), collector); + } } } + } } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/CoGroupWithSolutionSetSecondDriver.java b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/CoGroupWithSolutionSetSecondDriver.java index fb88505ef7b27..17fc47198dc6a 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/CoGroupWithSolutionSetSecondDriver.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/CoGroupWithSolutionSetSecondDriver.java @@ -18,6 +18,7 @@ package org.apache.flink.runtime.operators; +import org.apache.flink.api.common.ExecutionConfig; import org.apache.flink.api.common.functions.CoGroupFunction; import org.apache.flink.api.common.operators.util.JoinHashMap; import org.apache.flink.api.common.typeutils.TypeComparator; @@ -30,7 +31,8 @@ import org.apache.flink.runtime.operators.hash.CompactingHashTable; import org.apache.flink.runtime.operators.util.TaskConfig; import org.apache.flink.runtime.util.EmptyIterator; -import org.apache.flink.runtime.util.KeyGroupedIterator; +import org.apache.flink.runtime.util.NonReusingKeyGroupedIterator; +import org.apache.flink.runtime.util.ReusingKeyGroupedIterator; import org.apache.flink.runtime.util.SingleElementIterator; import org.apache.flink.util.Collector; @@ -45,13 +47,17 @@ public class CoGroupWithSolutionSetSecondDriver implements Resetta private TypeSerializer probeSideSerializer; private TypeComparator probeSideComparator; - + + private TypeSerializer solutionSetSerializer; + private TypePairComparator pairComparator; private IT2 solutionSideRecord; protected volatile boolean running; + private boolean objectReuseEnabled = false; + // -------------------------------------------------------------------------------------------- @Override @@ -95,7 +101,6 @@ public boolean isInputResettable(int inputNum) { @SuppressWarnings("unchecked") public void initialize() throws Exception { - final TypeSerializer solutionSetSerializer; final TypeComparator solutionSetComparator; // grab a handle to the hash table from the iteration broker @@ -129,8 +134,13 @@ else if (table instanceof JoinHashMap) { this.probeSideSerializer = taskContext.getInputSerializer(0).getSerializer(); this.probeSideComparator = probeSideComparatorFactory.createComparator(); - - solutionSideRecord = solutionSetSerializer.createInstance(); + + ExecutionConfig executionConfig = taskContext.getExecutionConfig(); + objectReuseEnabled = executionConfig.isObjectReuseEnabled(); + + if (objectReuseEnabled) { + solutionSideRecord = solutionSetSerializer.createInstance(); + }; TypePairComparatorFactory factory = taskContext.getTaskConfig().getPairComparatorFactory(taskContext.getUserCodeClassLoader()); pairComparator = factory.createComparator12(this.probeSideComparator, solutionSetComparator); @@ -149,46 +159,84 @@ public void run() throws Exception { final CoGroupFunction coGroupStub = taskContext.getStub(); final Collector collector = taskContext.getOutputCollector(); - final KeyGroupedIterator probeSideInput = new KeyGroupedIterator(taskContext.getInput(0), probeSideSerializer, probeSideComparator); final SingleElementIterator siIter = new SingleElementIterator(); final Iterable emptySolutionSide = EmptyIterator.get(); - - if (this.hashTable != null) { - final CompactingHashTable join = hashTable; - final CompactingHashTable.HashTableProber prober = join.getProber(this.probeSideComparator, this.pairComparator); - - IT2 buildSideRecord = solutionSideRecord; - - while (this.running && probeSideInput.nextKey()) { - IT1 current = probeSideInput.getCurrent(); - - buildSideRecord = prober.getMatchFor(current, buildSideRecord); - if (buildSideRecord != null) { - siIter.set(buildSideRecord); - coGroupStub.coGroup(probeSideInput.getValues(), siIter, collector); + + if (objectReuseEnabled) { + final ReusingKeyGroupedIterator probeSideInput = new ReusingKeyGroupedIterator(taskContext.getInput(0), probeSideSerializer, probeSideComparator); + + if (this.hashTable != null) { + final CompactingHashTable join = hashTable; + final CompactingHashTable.HashTableProber prober = join.getProber(this.probeSideComparator, this.pairComparator); + + IT2 buildSideRecord = solutionSideRecord; + + while (this.running && probeSideInput.nextKey()) { + IT1 current = probeSideInput.getCurrent(); + + buildSideRecord = prober.getMatchFor(current, buildSideRecord); + if (buildSideRecord != null) { + siIter.set(buildSideRecord); + coGroupStub.coGroup(probeSideInput.getValues(), siIter, collector); + } else { + coGroupStub.coGroup(probeSideInput.getValues(), emptySolutionSide, collector); + } } - else { - coGroupStub.coGroup(probeSideInput.getValues(), emptySolutionSide, collector); + } else { + final JoinHashMap join = this.objectMap; + final JoinHashMap.Prober prober = join.createProber(this.probeSideComparator, this.pairComparator); + final TypeSerializer serializer = join.getBuildSerializer(); + + while (this.running && probeSideInput.nextKey()) { + IT1 current = probeSideInput.getCurrent(); + + IT2 buildSideRecord = prober.lookupMatch(current); + if (buildSideRecord != null) { + siIter.set(serializer.copy(buildSideRecord)); + coGroupStub.coGroup(probeSideInput.getValues(), siIter, collector); + } else { + coGroupStub.coGroup(probeSideInput.getValues(), emptySolutionSide, collector); + } } } - } - else { - final JoinHashMap join = this.objectMap; - final JoinHashMap.Prober prober = join.createProber(this.probeSideComparator, this.pairComparator); - final TypeSerializer serializer = join.getBuildSerializer(); - - while (this.running && probeSideInput.nextKey()) { - IT1 current = probeSideInput.getCurrent(); - - IT2 buildSideRecord = prober.lookupMatch(current); - if (buildSideRecord != null) { - siIter.set(serializer.copy(buildSideRecord)); - coGroupStub.coGroup(probeSideInput.getValues(), siIter, collector); + } else { + final NonReusingKeyGroupedIterator probeSideInput = new NonReusingKeyGroupedIterator(taskContext.getInput(0), probeSideSerializer, probeSideComparator); + + if (this.hashTable != null) { + final CompactingHashTable join = hashTable; + final CompactingHashTable.HashTableProber prober = join.getProber(this.probeSideComparator, this.pairComparator); + + IT2 buildSideRecord; + + while (this.running && probeSideInput.nextKey()) { + IT1 current = probeSideInput.getCurrent(); + + buildSideRecord = prober.getMatchFor(current); + if (buildSideRecord != null) { + siIter.set(solutionSetSerializer.copy(buildSideRecord)); + coGroupStub.coGroup(probeSideInput.getValues(), siIter, collector); + } else { + coGroupStub.coGroup(probeSideInput.getValues(), emptySolutionSide, collector); + } } - else { - coGroupStub.coGroup(probeSideInput.getValues(), emptySolutionSide, collector); + } else { + final JoinHashMap join = this.objectMap; + final JoinHashMap.Prober prober = join.createProber(this.probeSideComparator, this.pairComparator); + final TypeSerializer serializer = join.getBuildSerializer(); + + while (this.running && probeSideInput.nextKey()) { + IT1 current = probeSideInput.getCurrent(); + + IT2 buildSideRecord = prober.lookupMatch(current); + if (buildSideRecord != null) { + siIter.set(serializer.copy(buildSideRecord)); + coGroupStub.coGroup(probeSideInput.getValues(), siIter, collector); + } else { + coGroupStub.coGroup(probeSideInput.getValues(), emptySolutionSide, collector); + } } } + } } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/CollectorMapDriver.java b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/CollectorMapDriver.java index 385886496cd3f..766a9d9134e36 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/CollectorMapDriver.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/CollectorMapDriver.java @@ -19,9 +19,12 @@ package org.apache.flink.runtime.operators; +import org.apache.flink.api.common.ExecutionConfig; import org.apache.flink.api.common.functions.GenericCollectorMap; import org.apache.flink.util.Collector; import org.apache.flink.util.MutableObjectIterator; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; /** * Map task which is executed by a Nephele task manager. The task has a single @@ -38,12 +41,16 @@ */ @SuppressWarnings("deprecation") public class CollectorMapDriver implements PactDriver, OT> { - + + private static final Logger LOG = LoggerFactory.getLogger(CollectorMapDriver.class); + + private PactTaskContext, OT> taskContext; private volatile boolean running; - - + + private boolean objectReuseEnabled = false; + @Override public void setup(PactTaskContext, OT> context) { this.taskContext = context; @@ -69,7 +76,12 @@ public int getNumberOfDriverComparators() { @Override public void prepare() { - // nothing, since a mapper does not need any preparation + ExecutionConfig executionConfig = taskContext.getExecutionConfig(); + this.objectReuseEnabled = executionConfig.isObjectReuseEnabled(); + + if (LOG.isDebugEnabled()) { + LOG.debug("CollectorMapDriver object reuse: " + (this.objectReuseEnabled ? "ENABLED" : "DISABLED") + "."); + } } @Override @@ -79,10 +91,18 @@ public void run() throws Exception { final GenericCollectorMap stub = this.taskContext.getStub(); final Collector output = this.taskContext.getOutputCollector(); - IT record = this.taskContext.getInputSerializer(0).getSerializer().createInstance(); + if (objectReuseEnabled) { + IT record = this.taskContext.getInputSerializer(0).getSerializer().createInstance(); + - while (this.running && ((record = input.next(record)) != null)) { - stub.map(record, output); + while (this.running && ((record = input.next(record)) != null)) { + stub.map(record, output); + } + } else { + IT record; + while (this.running && ((record = input.next()) != null)) { + stub.map(record, output); + } } } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/CrossDriver.java b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/CrossDriver.java index 4e6745a17ea83..197c08d11b2c7 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/CrossDriver.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/CrossDriver.java @@ -19,6 +19,7 @@ package org.apache.flink.runtime.operators; +import org.apache.flink.api.common.ExecutionConfig; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import org.apache.flink.api.common.functions.CrossFunction; @@ -62,7 +63,9 @@ public class CrossDriver implements PactDriver crosser = this.taskContext.getStub(); final Collector collector = this.taskContext.getOutputCollector(); - - // for all blocks - do { - // for all values from the spilling side - while (this.running && ((val2 = spillVals.next(val2Reuse)) != null)) { - // for all values in the block - while ((val1 = blockVals.next(val1Reuse)) != null) { - val2Copy = serializer2.copy(val2, val2Copy); - collector.collect(crosser.cross(val1,val2Copy)); - //crosser.cross(val1, val2Copy, collector); + + + if (objectReuseEnabled) { + final T1 val1Reuse = serializer1.createInstance(); + final T2 val2Reuse = serializer2.createInstance(); + T1 val1; + T2 val2; + + // for all blocks + do { + // for all values from the spilling side + while (this.running && ((val2 = spillVals.next(val2Reuse)) != null)) { + // for all values in the block + while ((val1 = blockVals.next(val1Reuse)) != null) { + collector.collect(crosser.cross(val1, val2)); + } + blockVals.reset(); } - blockVals.reset(); - } - spillVals.reset(); + spillVals.reset(); + } while (this.running && blockVals.nextBlock()); + } else { + T1 val1; + T2 val2; + + // for all blocks + do { + // for all values from the spilling side + while (this.running && ((val2 = spillVals.next()) != null)) { + // for all values in the block + while ((val1 = blockVals.next()) != null) { + collector.collect(crosser.cross(val1, serializer2.copy(val2))); + } + blockVals.reset(); + } + spillVals.reset(); + } while (this.running && blockVals.nextBlock()); + } - while (this.running && blockVals.nextBlock()); } private void runBlockedOuterSecond() throws Exception { @@ -249,30 +276,45 @@ private void runBlockedOuterSecond() throws Exception { this.taskContext.getOwningNepheleTask()); this.blockIter = blockVals; - T1 val1; - final T1 val1Reuse = serializer1.createInstance(); - T1 val1Copy = serializer1.createInstance(); - T2 val2; - final T2 val2Reuse = serializer2.createInstance(); - final CrossFunction crosser = this.taskContext.getStub(); final Collector collector = this.taskContext.getOutputCollector(); - - // for all blocks - do { - // for all values from the spilling side - while (this.running && ((val1 = spillVals.next(val1Reuse)) != null)) { - // for all values in the block - while (this.running && ((val2 = blockVals.next(val2Reuse)) != null)) { - val1Copy = serializer1.copy(val1, val1Copy); - collector.collect(crosser.cross(val1Copy, val2)); - //crosser.cross(val1Copy, val2, collector); + + if (objectReuseEnabled) { + final T1 val1Reuse = serializer1.createInstance(); + final T2 val2Reuse = serializer2.createInstance(); + T1 val1; + T2 val2; + + // for all blocks + do { + // for all values from the spilling side + while (this.running && ((val1 = spillVals.next(val1Reuse)) != null)) { + // for all values in the block + while (this.running && ((val2 = blockVals.next(val2Reuse)) != null)) { + collector.collect(crosser.cross(val1, val2)); + } + blockVals.reset(); } - blockVals.reset(); - } - spillVals.reset(); + spillVals.reset(); + } while (this.running && blockVals.nextBlock()); + } else { + T1 val1; + T2 val2; + + // for all blocks + do { + // for all values from the spilling side + while (this.running && ((val1 = spillVals.next()) != null)) { + // for all values in the block + while (this.running && ((val2 = blockVals.next()) != null)) { + collector.collect(crosser.cross(serializer1.copy(val1), val2)); + } + blockVals.reset(); + } + spillVals.reset(); + } while (this.running && blockVals.nextBlock()); + } - while (this.running && blockVals.nextBlock()); } private void runStreamedOuterFirst() throws Exception { @@ -292,24 +334,36 @@ private void runStreamedOuterFirst() throws Exception { this.taskContext.getOwningNepheleTask()); this.spillIter = spillVals; - T1 val1; - final T1 val1Reuse = serializer1.createInstance(); - T1 val1Copy = serializer1.createInstance(); - T2 val2; - final T2 val2Reuse = serializer2.createInstance(); - final CrossFunction crosser = this.taskContext.getStub(); final Collector collector = this.taskContext.getOutputCollector(); - - // for all blocks - while (this.running && ((val1 = in1.next(val1Reuse)) != null)) { - // for all values from the spilling side - while (this.running && ((val2 = spillVals.next(val2Reuse)) != null)) { - val1Copy = serializer1.copy(val1, val1Copy); - collector.collect(crosser.cross(val1Copy, val2)); - //crosser.cross(val1Copy, val2, collector); + + if (objectReuseEnabled) { + final T1 val1Reuse = serializer1.createInstance(); + final T2 val2Reuse = serializer2.createInstance(); + T1 val1; + T2 val2; + + // for all blocks + while (this.running && ((val1 = in1.next(val1Reuse)) != null)) { + // for all values from the spilling side + while (this.running && ((val2 = spillVals.next(val2Reuse)) != null)) { + collector.collect(crosser.cross(val1, val2)); + } + spillVals.reset(); } - spillVals.reset(); + } else { + T1 val1; + T2 val2; + + // for all blocks + while (this.running && ((val1 = in1.next()) != null)) { + // for all values from the spilling side + while (this.running && ((val2 = spillVals.next()) != null)) { + collector.collect(crosser.cross(serializer1.copy(val1), val2)); + } + spillVals.reset(); + } + } } @@ -328,25 +382,38 @@ private void runStreamedOuterSecond() throws Exception { in1, serializer1, this.memManager, this.taskContext.getIOManager(), this.memPagesForSpillingSide, this.taskContext.getOwningNepheleTask()); this.spillIter = spillVals; - - T1 val1; - final T1 val1Reuse = serializer1.createInstance(); - T2 val2; - final T2 val2Reuse = serializer2.createInstance(); - T2 val2Copy = serializer2.createInstance(); - + final CrossFunction crosser = this.taskContext.getStub(); final Collector collector = this.taskContext.getOutputCollector(); - - // for all blocks - while (this.running && (val2 = in2.next(val2Reuse)) != null) { - // for all values from the spilling side - while (this.running && (val1 = spillVals.next(val1Reuse)) != null) { - val2Copy = serializer2.copy(val2, val2Copy); - collector.collect(crosser.cross(val1, val2Copy)); - //crosser.cross(val1, val2Copy, collector); + + if (objectReuseEnabled) { + final T1 val1Reuse = serializer1.createInstance(); + final T2 val2Reuse = serializer2.createInstance(); + T1 val1; + T2 val2; + + // for all blocks + while (this.running && (val2 = in2.next(val2Reuse)) != null) { + // for all values from the spilling side + while (this.running && (val1 = spillVals.next(val1Reuse)) != null) { + collector.collect(crosser.cross(val1, val2)); + //crosser.cross(val1, val2Copy, collector); + } + spillVals.reset(); + } + } else { + T1 val1; + T2 val2; + + // for all blocks + while (this.running && (val2 = in2.next()) != null) { + // for all values from the spilling side + while (this.running && (val1 = spillVals.next()) != null) { + collector.collect(crosser.cross(val1, serializer2.copy(val2))); + } + spillVals.reset(); } - spillVals.reset(); + } } } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/DataSinkTask.java b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/DataSinkTask.java index 610ab0661f8c3..e3ffdba47208e 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/DataSinkTask.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/DataSinkTask.java @@ -19,7 +19,9 @@ package org.apache.flink.runtime.operators; +import org.apache.flink.api.common.ExecutionConfig; import org.apache.flink.runtime.plugable.DeserializationDelegate; +import org.apache.flink.util.InstantiationUtil; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import org.apache.flink.api.common.io.CleanupWhenUnsuccessful; @@ -41,6 +43,8 @@ import org.apache.flink.runtime.operators.util.TaskConfig; import org.apache.flink.util.MutableObjectIterator; +import java.io.IOException; + /** * DataSinkTask which is executed by a task manager. The task hands the data to an output format. * @@ -75,7 +79,6 @@ public class DataSinkTask extends AbstractInvokable { private volatile boolean taskCanceled; private volatile boolean cleanupCalled; - @Override public void registerInputOutput() { @@ -106,6 +109,22 @@ public void invoke() throws Exception if (LOG.isDebugEnabled()) { LOG.debug(getLogString("Starting data sink operator")); } + + ExecutionConfig executionConfig = new ExecutionConfig(); + try { + ExecutionConfig c = (ExecutionConfig) InstantiationUtil.readObjectFromConfig( + getJobConfiguration(), + ExecutionConfig.CONFIG_KEY, + this.getClass().getClassLoader()); + if (c != null) { + executionConfig = c; + } + } catch (IOException e) { + throw new RuntimeException("Could not load ExecutionConfig from Job Configuration: " + e); + } catch (ClassNotFoundException e) { + throw new RuntimeException("Could not load ExecutionConfig from Job Configuration: " + e); + } + boolean objectReuseEnabled = executionConfig.isObjectReuseEnabled(); try { @@ -150,10 +169,8 @@ public void invoke() throws Exception final TypeSerializer serializer = this.inputTypeSerializerFactory.getSerializer(); final MutableObjectIterator input = this.input; final OutputFormat format = this.format; - - - IT record = serializer.createInstance(); - + + // check if task has been canceled if (this.taskCanceled) { return; @@ -166,9 +183,20 @@ public void invoke() throws Exception // open format.open(this.getEnvironment().getIndexInSubtaskGroup(), this.getEnvironment().getCurrentNumberOfSubtasks()); - // work! - while (!this.taskCanceled && ((record = input.next(record)) != null)) { - format.writeRecord(record); + if (objectReuseEnabled) { + IT record = serializer.createInstance(); + + // work! + while (!this.taskCanceled && ((record = input.next(record)) != null)) { + format.writeRecord(record); + } + } else { + IT record; + + // work! + while (!this.taskCanceled && ((record = input.next()) != null)) { + format.writeRecord(record); + } } // close. We close here such that a regular close throwing an exception marks a task as failed. diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/DataSourceTask.java b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/DataSourceTask.java index 2db652f89015f..b50bf363efe0d 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/DataSourceTask.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/DataSourceTask.java @@ -18,12 +18,15 @@ package org.apache.flink.runtime.operators; +import java.io.IOException; import java.util.ArrayList; import java.util.HashMap; import java.util.Iterator; import java.util.List; import java.util.NoSuchElementException; +import org.apache.flink.api.common.ExecutionConfig; +import org.apache.flink.util.InstantiationUtil; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import org.apache.flink.api.common.accumulators.Accumulator; @@ -36,10 +39,8 @@ import org.apache.flink.runtime.io.network.api.BufferWriter; import org.apache.flink.runtime.jobgraph.tasks.AbstractInvokable; import org.apache.flink.runtime.jobgraph.tasks.InputSplitProvider; -import org.apache.flink.runtime.operators.chaining.ChainedCollectorMapDriver; import org.apache.flink.runtime.operators.chaining.ChainedDriver; import org.apache.flink.runtime.operators.chaining.ExceptionInChainedStubException; -import org.apache.flink.runtime.operators.shipping.OutputCollector; import org.apache.flink.runtime.operators.util.TaskConfig; import org.apache.flink.util.Collector; @@ -74,6 +75,23 @@ public class DataSourceTask extends AbstractInvokable { // cancel flag private volatile boolean taskCanceled = false; + private ExecutionConfig getExecutionConfig() { + ExecutionConfig executionConfig = new ExecutionConfig(); + try { + ExecutionConfig c = (ExecutionConfig) InstantiationUtil.readObjectFromConfig( + getJobConfiguration(), + ExecutionConfig.CONFIG_KEY, + this.getClass().getClassLoader()); + if (c != null) { + executionConfig = c; + } + } catch (IOException e) { + throw new RuntimeException("Could not load ExecutionConfig from Job Configuration: " + e); + } catch (ClassNotFoundException e) { + throw new RuntimeException("Could not load ExecutionConfig from Job Configuration: " + e); + } + return executionConfig; + } @Override public void registerInputOutput() { @@ -102,6 +120,27 @@ public void invoke() throws Exception { if (LOG.isDebugEnabled()) { LOG.debug(getLogString("Starting data source operator")); } + + ExecutionConfig executionConfig = new ExecutionConfig(); + try { + ExecutionConfig c = (ExecutionConfig) InstantiationUtil.readObjectFromConfig( + getJobConfiguration(), + ExecutionConfig.CONFIG_KEY, + this.getClass().getClassLoader()); + if (c != null) { + executionConfig = c; + } + } catch (IOException e) { + throw new RuntimeException("Could not load ExecutionConfig from Job Configuration: " + e); + } catch (ClassNotFoundException e) { + throw new RuntimeException("Could not load ExecutionConfig from Job Configuration: " + e); + } + + boolean objectReuseEnabled = executionConfig.isObjectReuseEnabled(); + + if (LOG.isDebugEnabled()) { + LOG.debug("DataSourceTask object reuse: " + (objectReuseEnabled ? "ENABLED" : "DISABLED") + "."); + } final TypeSerializer serializer = this.serializerFactory.getSerializer(); @@ -121,8 +160,6 @@ public void invoke() throws Exception { // get start and end final InputSplit split = splitIterator.next(); - OT record = serializer.createInstance(); - if (LOG.isDebugEnabled()) { LOG.debug(getLogString("Opening input split " + split.toString())); } @@ -137,48 +174,31 @@ public void invoke() throws Exception { } try { - // special case to make the loops tighter - if (this.output instanceof OutputCollector) { - final OutputCollector output = (OutputCollector) this.output; - - // as long as there is data to read - while (!this.taskCanceled && !format.reachedEnd()) { - - OT returned; - if ((returned = format.nextRecord(record)) != null) { - output.collect(returned); - record = returned; - } - } - } - else if (this.output instanceof ChainedCollectorMapDriver) { - @SuppressWarnings("unchecked") - final ChainedCollectorMapDriver output = (ChainedCollectorMapDriver) this.output; - + + final Collector output = this.output; + + if (objectReuseEnabled) { + OT reuse = serializer.createInstance(); + // as long as there is data to read while (!this.taskCanceled && !format.reachedEnd()) { - + OT returned; - if ((returned = format.nextRecord(record)) != null) { + if ((returned = format.nextRecord(reuse)) != null) { output.collect(returned); - record = returned; } } - } - else { - final Collector output = this.output; - + } else { // as long as there is data to read while (!this.taskCanceled && !format.reachedEnd()) { - + OT returned; - if ((returned = format.nextRecord(record)) != null) { + if ((returned = format.nextRecord(serializer.createInstance())) != null) { output.collect(returned); - record = returned; } } } - + if (LOG.isDebugEnabled() && !this.taskCanceled) { LOG.debug(getLogString("Closing input split " + split.toString())); } @@ -285,7 +305,7 @@ private void initInputFormat() { private void initOutputs(ClassLoader cl) throws Exception { this.chainedTasks = new ArrayList>(); this.eventualOutputs = new ArrayList(); - this.output = RegularPactTask.initOutputs(this, cl, this.config, this.chainedTasks, this.eventualOutputs); + this.output = RegularPactTask.initOutputs(this, cl, this.config, this.chainedTasks, this.eventualOutputs, getExecutionConfig()); } // ------------------------------------------------------------------------ diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/FlatMapDriver.java b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/FlatMapDriver.java index f4ae62d666033..d63a3e34a2151 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/FlatMapDriver.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/FlatMapDriver.java @@ -19,9 +19,12 @@ package org.apache.flink.runtime.operators; +import org.apache.flink.api.common.ExecutionConfig; import org.apache.flink.api.common.functions.FlatMapFunction; import org.apache.flink.util.Collector; import org.apache.flink.util.MutableObjectIterator; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; /** * Map task which is executed by a Nephele task manager. The task has a single @@ -37,12 +40,15 @@ * @param The mapper's output data type. */ public class FlatMapDriver implements PactDriver, OT> { - + + private static final Logger LOG = LoggerFactory.getLogger(FlatMapDriver.class); + private PactTaskContext, OT> taskContext; private volatile boolean running; - - + + private boolean objectReuseEnabled = false; + @Override public void setup(PactTaskContext, OT> context) { this.taskContext = context; @@ -68,8 +74,12 @@ public int getNumberOfDriverComparators() { @Override public void prepare() { - // nothing, since a mapper does not need any preparation - } + ExecutionConfig executionConfig = taskContext.getExecutionConfig(); + this.objectReuseEnabled = executionConfig.isObjectReuseEnabled(); + + if (LOG.isDebugEnabled()) { + LOG.debug("FlatMapDriver object reuse: " + (this.objectReuseEnabled ? "ENABLED" : "DISABLED") + "."); + } } @Override public void run() throws Exception { @@ -78,10 +88,19 @@ public void run() throws Exception { final FlatMapFunction function = this.taskContext.getStub(); final Collector output = this.taskContext.getOutputCollector(); - IT record = this.taskContext.getInputSerializer(0).getSerializer().createInstance(); + if (objectReuseEnabled) { + IT record = this.taskContext.getInputSerializer(0).getSerializer().createInstance(); + + + while (this.running && ((record = input.next(record)) != null)) { + function.flatMap(record, output); + } + } else { + IT record; - while (this.running && ((record = input.next(record)) != null)) { - function.flatMap(record, output); + while (this.running && ((record = input.next()) != null)) { + function.flatMap(record, output); + } } } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/GroupReduceCombineDriver.java b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/GroupReduceCombineDriver.java index 4323eae5ec955..8d8d5dc63e2cd 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/GroupReduceCombineDriver.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/GroupReduceCombineDriver.java @@ -19,6 +19,8 @@ package org.apache.flink.runtime.operators; +import org.apache.flink.api.common.ExecutionConfig; +import org.apache.flink.runtime.util.NonReusingKeyGroupedIterator; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import org.apache.flink.api.common.functions.FlatCombineFunction; @@ -31,7 +33,7 @@ import org.apache.flink.runtime.operators.sort.InMemorySorter; import org.apache.flink.runtime.operators.sort.NormalizedKeySorter; import org.apache.flink.runtime.operators.sort.QuickSort; -import org.apache.flink.runtime.util.KeyGroupedIterator; +import org.apache.flink.runtime.util.ReusingKeyGroupedIterator; import org.apache.flink.util.Collector; import org.apache.flink.util.MutableObjectIterator; @@ -72,6 +74,8 @@ public class GroupReduceCombineDriver implements PactDriver(this.serializer, this.sortingComparator.duplicate(), memory); } + + ExecutionConfig executionConfig = taskContext.getExecutionConfig(); + this.objectReuseEnabled = executionConfig.isObjectReuseEnabled(); + + if (LOG.isDebugEnabled()) { + LOG.debug("GroupReduceCombineDriver object reuse: " + (this.objectReuseEnabled ? "ENABLED" : "DISABLED") + "."); + } } @Override @@ -162,19 +173,37 @@ public void run() throws Exception { private void sortAndCombine() throws Exception { final InMemorySorter sorter = this.sorter; - if (!sorter.isEmpty()) { - this.sortAlgo.sort(sorter); + if (objectReuseEnabled) { + if (!sorter.isEmpty()) { + this.sortAlgo.sort(sorter); + + final ReusingKeyGroupedIterator keyIter = new ReusingKeyGroupedIterator(sorter.getIterator(), this.serializer, this.groupingComparator); - final KeyGroupedIterator keyIter = new KeyGroupedIterator(sorter.getIterator(), this.serializer, - this.groupingComparator); - final FlatCombineFunction combiner = this.combiner; - final Collector output = this.output; + final FlatCombineFunction combiner = this.combiner; + final Collector output = this.output; - // iterate over key groups - while (this.running && keyIter.nextKey()) { - combiner.combine(keyIter.getValues(), output); + // iterate over key groups + while (this.running && keyIter.nextKey()) { + combiner.combine(keyIter.getValues(), output); + } } + } else { + if (!sorter.isEmpty()) { + this.sortAlgo.sort(sorter); + + final NonReusingKeyGroupedIterator keyIter = new NonReusingKeyGroupedIterator(sorter.getIterator(), this.serializer, this.groupingComparator); + + + final FlatCombineFunction combiner = this.combiner; + final Collector output = this.output; + + // iterate over key groups + while (this.running && keyIter.nextKey()) { + combiner.combine(keyIter.getValues(), output); + } + } + } } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/GroupReduceDriver.java b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/GroupReduceDriver.java index 2ec987377e706..9d9f9946bd0f5 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/GroupReduceDriver.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/GroupReduceDriver.java @@ -19,14 +19,15 @@ package org.apache.flink.runtime.operators; +import org.apache.flink.api.common.ExecutionConfig; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import org.apache.flink.api.common.functions.GroupReduceFunction; import org.apache.flink.api.common.typeutils.TypeComparator; import org.apache.flink.api.common.typeutils.TypeSerializer; import org.apache.flink.runtime.operators.util.TaskConfig; -import org.apache.flink.runtime.util.KeyGroupedIterator; -import org.apache.flink.runtime.util.KeyGroupedIteratorImmutable; +import org.apache.flink.runtime.util.ReusingKeyGroupedIterator; +import org.apache.flink.runtime.util.NonReusingKeyGroupedIterator; import org.apache.flink.util.Collector; import org.apache.flink.util.MutableObjectIterator; @@ -51,11 +52,11 @@ public class GroupReduceDriver implements PactDriver serializer; private TypeComparator comparator; - - private boolean mutableObjectMode = false; - + private volatile boolean running; + private boolean objectReuseEnabled = false; + // ------------------------------------------------------------------------ @Override @@ -92,11 +93,12 @@ public void prepare() throws Exception { this.serializer = this.taskContext.getInputSerializer(0).getSerializer(); this.comparator = this.taskContext.getDriverComparator(0); this.input = this.taskContext.getInput(0); - - this.mutableObjectMode = config.getMutableObjectMode(); - + + ExecutionConfig executionConfig = taskContext.getExecutionConfig(); + this.objectReuseEnabled = executionConfig.isObjectReuseEnabled(); + if (LOG.isDebugEnabled()) { - LOG.debug("GroupReduceDriver uses " + (this.mutableObjectMode ? "MUTABLE" : "IMMUTABLE") + " object mode."); + LOG.debug("GroupReduceDriver object reuse: " + (this.objectReuseEnabled ? "ENABLED" : "DISABLED") + "."); } } @@ -110,15 +112,15 @@ public void run() throws Exception { final GroupReduceFunction stub = this.taskContext.getStub(); final Collector output = this.taskContext.getOutputCollector(); - if (mutableObjectMode) { - final KeyGroupedIterator iter = new KeyGroupedIterator(this.input, this.serializer, this.comparator); + if (objectReuseEnabled) { + final ReusingKeyGroupedIterator iter = new ReusingKeyGroupedIterator(this.input, this.serializer, this.comparator); // run stub implementation while (this.running && iter.nextKey()) { stub.reduce(iter.getValues(), output); } } else { - final KeyGroupedIteratorImmutable iter = new KeyGroupedIteratorImmutable(this.input, this.serializer, this.comparator); + final NonReusingKeyGroupedIterator iter = new NonReusingKeyGroupedIterator(this.input, this.serializer, this.comparator); // run stub implementation while (this.running && iter.nextKey()) { stub.reduce(iter.getValues(), output); diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/JoinWithSolutionSetFirstDriver.java b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/JoinWithSolutionSetFirstDriver.java index 4f9f1ba2ff0c6..a1c8a4a0c5593 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/JoinWithSolutionSetFirstDriver.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/JoinWithSolutionSetFirstDriver.java @@ -18,6 +18,7 @@ package org.apache.flink.runtime.operators; +import org.apache.flink.api.common.ExecutionConfig; import org.apache.flink.api.common.functions.FlatJoinFunction; import org.apache.flink.api.common.operators.util.JoinHashMap; import org.apache.flink.api.common.typeutils.TypeComparator; @@ -48,7 +49,9 @@ public class JoinWithSolutionSetFirstDriver implements ResettableP private IT2 probeSideRecord; protected volatile boolean running; - + + private boolean objectReuseEnabled = false; + // -------------------------------------------------------------------------------------------- @Override @@ -125,10 +128,15 @@ else if (table instanceof JoinHashMap) { TypeComparatorFactory probeSideComparatorFactory = config.getDriverComparator(0, classLoader); this.probeSideComparator = probeSideComparatorFactory.createComparator(); - - solutionSideRecord = solutionSetSerializer.createInstance(); - probeSideRecord = probeSideSerializer.createInstance(); - + + ExecutionConfig executionConfig = taskContext.getExecutionConfig(); + objectReuseEnabled = executionConfig.isObjectReuseEnabled(); + + if (objectReuseEnabled) { + solutionSideRecord = solutionSetSerializer.createInstance(); + probeSideRecord = probeSideSerializer.createInstance(); + } + TypePairComparatorFactory factory = taskContext.getTaskConfig().getPairComparatorFactory(taskContext.getUserCodeClassLoader()); pairComparator = factory.createComparator21(solutionSetComparator, this.probeSideComparator); } @@ -146,33 +154,60 @@ public void run() throws Exception { final Collector collector = taskContext.getOutputCollector(); final MutableObjectIterator probeSideInput = taskContext.getInput(0); - - IT2 probeSideRecord = this.probeSideRecord; - - if (hashTable != null) { - final CompactingHashTable join = hashTable; - final CompactingHashTable.HashTableProber prober = join.getProber(probeSideComparator, pairComparator); - - IT1 buildSideRecord = this.solutionSideRecord; - - while (this.running && ((probeSideRecord = probeSideInput.next(probeSideRecord)) != null)) { - buildSideRecord = prober.getMatchFor(probeSideRecord, buildSideRecord); - joinFunction.join(buildSideRecord, probeSideRecord, collector); + + if (objectReuseEnabled) { + IT2 probeSideRecord = this.probeSideRecord; + + if (hashTable != null) { + final CompactingHashTable join = hashTable; + final CompactingHashTable.HashTableProber prober = join.getProber(probeSideComparator, pairComparator); + + + IT1 buildSideRecord = this.solutionSideRecord; + + while (this.running && ((probeSideRecord = probeSideInput.next(probeSideRecord)) != null)) { + buildSideRecord = prober.getMatchFor(probeSideRecord, buildSideRecord); + joinFunction.join(buildSideRecord, probeSideRecord, collector); + } + } else if (objectMap != null) { + final JoinHashMap hashTable = this.objectMap; + final JoinHashMap.Prober prober = this.objectMap.createProber(probeSideComparator, pairComparator); + final TypeSerializer buildSerializer = hashTable.getBuildSerializer(); + + while (this.running && ((probeSideRecord = probeSideInput.next(probeSideRecord)) != null)) { + IT1 match = prober.lookupMatch(probeSideRecord); + joinFunction.join(buildSerializer.copy(match), probeSideRecord, collector); + } + } else { + throw new RuntimeException(); } - } - else if (objectMap != null) { - final JoinHashMap hashTable = this.objectMap; - final JoinHashMap.Prober prober = this.objectMap.createProber(probeSideComparator, pairComparator); - final TypeSerializer buildSerializer = hashTable.getBuildSerializer(); - - while (this.running && ((probeSideRecord = probeSideInput.next(probeSideRecord)) != null)) { - IT1 match = prober.lookupMatch(probeSideRecord); - joinFunction.join(buildSerializer.copy(match), probeSideRecord, collector); + } else { + IT2 probeSideRecord; + + if (hashTable != null) { + final CompactingHashTable join = hashTable; + final CompactingHashTable.HashTableProber prober = join.getProber(probeSideComparator, pairComparator); + + + IT1 buildSideRecord; + + while (this.running && ((probeSideRecord = probeSideInput.next()) != null)) { + buildSideRecord = prober.getMatchFor(probeSideRecord); + joinFunction.join(buildSideRecord, probeSideRecord, collector); + } + } else if (objectMap != null) { + final JoinHashMap hashTable = this.objectMap; + final JoinHashMap.Prober prober = this.objectMap.createProber(probeSideComparator, pairComparator); + final TypeSerializer buildSerializer = hashTable.getBuildSerializer(); + + while (this.running && ((probeSideRecord = probeSideInput.next()) != null)) { + IT1 match = prober.lookupMatch(probeSideRecord); + joinFunction.join(buildSerializer.copy(match), probeSideRecord, collector); + } + } else { + throw new RuntimeException(); } } - else { - throw new RuntimeException(); - } } @Override diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/JoinWithSolutionSetSecondDriver.java b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/JoinWithSolutionSetSecondDriver.java index 7866cabc6cfab..32a75dc90889a 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/JoinWithSolutionSetSecondDriver.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/JoinWithSolutionSetSecondDriver.java @@ -18,6 +18,7 @@ package org.apache.flink.runtime.operators; +import org.apache.flink.api.common.ExecutionConfig; import org.apache.flink.api.common.functions.FlatJoinFunction; import org.apache.flink.api.common.operators.util.JoinHashMap; import org.apache.flink.api.common.typeutils.TypeComparator; @@ -48,7 +49,9 @@ public class JoinWithSolutionSetSecondDriver implements Resettable private IT1 probeSideRecord; protected volatile boolean running; - + + private boolean objectReuseEnabled = false; + // -------------------------------------------------------------------------------------------- @Override @@ -128,8 +131,13 @@ else if (table instanceof JoinHashMap) { this.probeSideComparator = probeSideComparatorFactory.createComparator(); - solutionSideRecord = solutionSetSerializer.createInstance(); - probeSideRecord = probeSideSerializer.createInstance(); + ExecutionConfig executionConfig = taskContext.getExecutionConfig(); + objectReuseEnabled = executionConfig.isObjectReuseEnabled(); + + if (objectReuseEnabled) { + solutionSideRecord = solutionSetSerializer.createInstance(); + probeSideRecord = probeSideSerializer.createInstance(); + } TypePairComparatorFactory factory = taskContext.getTaskConfig().getPairComparatorFactory(taskContext.getUserCodeClassLoader()); pairComparator = factory.createComparator12(this.probeSideComparator, solutionSetComparator); @@ -148,32 +156,60 @@ public void run() throws Exception { final FlatJoinFunction joinFunction = taskContext.getStub(); final Collector collector = taskContext.getOutputCollector(); final MutableObjectIterator probeSideInput = taskContext.getInput(0); - - IT1 probeSideRecord = this.probeSideRecord; - - if (hashTable != null) { - final CompactingHashTable join = hashTable; - final CompactingHashTable.HashTableProber prober = join.getProber(probeSideComparator, pairComparator); - - IT2 buildSideRecord = this.solutionSideRecord; - - while (this.running && ((probeSideRecord = probeSideInput.next(probeSideRecord)) != null)) { - buildSideRecord = prober.getMatchFor(probeSideRecord, buildSideRecord); - joinFunction.join(probeSideRecord, buildSideRecord, collector); + + if (objectReuseEnabled) { + IT1 probeSideRecord = this.probeSideRecord; + + if (hashTable != null) { + final CompactingHashTable join = hashTable; + final CompactingHashTable.HashTableProber prober = join.getProber(probeSideComparator, pairComparator); + + + IT2 buildSideRecord = this.solutionSideRecord; + + while (this.running && ((probeSideRecord = probeSideInput.next(probeSideRecord)) != null)) { + buildSideRecord = prober.getMatchFor(probeSideRecord, buildSideRecord); + joinFunction.join(probeSideRecord, buildSideRecord, collector); + } + } else if (objectMap != null) { + final JoinHashMap hashTable = this.objectMap; + final JoinHashMap.Prober prober = this.objectMap.createProber(probeSideComparator, pairComparator); + final TypeSerializer buildSerializer = hashTable.getBuildSerializer(); + + while (this.running && ((probeSideRecord = probeSideInput.next(probeSideRecord)) != null)) { + IT2 match = prober.lookupMatch(probeSideRecord); + joinFunction.join(probeSideRecord, buildSerializer.copy(match), collector); + } + } else { + throw new RuntimeException(); } - } - else if (objectMap != null) { - final JoinHashMap hashTable = this.objectMap; - final JoinHashMap.Prober prober = this.objectMap.createProber(probeSideComparator, pairComparator); - final TypeSerializer buildSerializer = hashTable.getBuildSerializer(); - - while (this.running && ((probeSideRecord = probeSideInput.next(probeSideRecord)) != null)) { - IT2 match = prober.lookupMatch(probeSideRecord); - joinFunction.join(probeSideRecord, buildSerializer.copy(match), collector); + } else { + IT1 probeSideRecord; + + if (hashTable != null) { + final CompactingHashTable join = hashTable; + final CompactingHashTable.HashTableProber prober = join.getProber(probeSideComparator, pairComparator); + + + IT2 buildSideRecord; + + while (this.running && ((probeSideRecord = probeSideInput.next()) != null)) { + buildSideRecord = prober.getMatchFor(probeSideRecord); + joinFunction.join(probeSideRecord, buildSideRecord, collector); + } + } else if (objectMap != null) { + final JoinHashMap hashTable = this.objectMap; + final JoinHashMap.Prober prober = this.objectMap.createProber(probeSideComparator, pairComparator); + final TypeSerializer buildSerializer = hashTable.getBuildSerializer(); + + while (this.running && ((probeSideRecord = probeSideInput.next()) != null)) { + IT2 match = prober.lookupMatch(probeSideRecord); + joinFunction.join(probeSideRecord, buildSerializer.copy(match), collector); + } + } else { + throw new RuntimeException(); } - } - else { - throw new RuntimeException(); + } } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/MapDriver.java b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/MapDriver.java index 2d989546b031c..d6679dd844d65 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/MapDriver.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/MapDriver.java @@ -19,6 +19,7 @@ package org.apache.flink.runtime.operators; +import org.apache.flink.api.common.ExecutionConfig; import org.apache.flink.api.common.functions.MapFunction; import org.apache.flink.util.Collector; import org.apache.flink.util.MutableObjectIterator; @@ -41,12 +42,17 @@ public class MapDriver implements PactDriver, OT> { private PactTaskContext, OT> taskContext; private volatile boolean running; + + private boolean objectReuseEnabled = false; @Override public void setup(PactTaskContext, OT> context) { this.taskContext = context; this.running = true; + + ExecutionConfig executionConfig = taskContext.getExecutionConfig(); + this.objectReuseEnabled = executionConfig.isObjectReuseEnabled(); } @Override diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/MapPartitionDriver.java b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/MapPartitionDriver.java index 41aa312df1610..401c951709493 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/MapPartitionDriver.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/MapPartitionDriver.java @@ -19,10 +19,14 @@ package org.apache.flink.runtime.operators; +import org.apache.flink.api.common.ExecutionConfig; import org.apache.flink.api.common.functions.MapPartitionFunction; -import org.apache.flink.runtime.util.MutableToRegularIteratorWrapper; +import org.apache.flink.runtime.util.NonReusingMutableToRegularIteratorWrapper; +import org.apache.flink.runtime.util.ReusingMutableToRegularIteratorWrapper; import org.apache.flink.util.Collector; import org.apache.flink.util.MutableObjectIterator; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; /** * MapPartition task which is executed by a Nephele task manager. The task has a single @@ -39,8 +43,12 @@ */ public class MapPartitionDriver implements PactDriver, OT> { + private static final Logger LOG = LoggerFactory.getLogger(MapPartitionDriver.class); + private PactTaskContext, OT> taskContext; + private boolean objectReuseEnabled = false; + @Override public void setup(PactTaskContext, OT> context) { this.taskContext = context; @@ -65,7 +73,12 @@ public int getNumberOfDriverComparators() { @Override public void prepare() { - // nothing, since a mapper does not need any preparation + ExecutionConfig executionConfig = taskContext.getExecutionConfig(); + this.objectReuseEnabled = executionConfig.isObjectReuseEnabled(); + + if (LOG.isDebugEnabled()) { + LOG.debug("MapPartitionDriver object reuse: " + (this.objectReuseEnabled ? "ENABLED" : "DISABLED") + "."); + } } @Override @@ -75,8 +88,15 @@ public void run() throws Exception { final MapPartitionFunction function = this.taskContext.getStub(); final Collector output = this.taskContext.getOutputCollector(); - final MutableToRegularIteratorWrapper inIter = new MutableToRegularIteratorWrapper(input, this.taskContext.getInputSerializer(0).getSerializer()); - function.mapPartition(inIter, output); + if (objectReuseEnabled) { + final ReusingMutableToRegularIteratorWrapper inIter = new ReusingMutableToRegularIteratorWrapper(input, this.taskContext.getInputSerializer(0).getSerializer()); + + function.mapPartition(inIter, output); + } else { + final NonReusingMutableToRegularIteratorWrapper inIter = new NonReusingMutableToRegularIteratorWrapper(input, this.taskContext.getInputSerializer(0).getSerializer()); + + function.mapPartition(inIter, output); + } } @Override diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/MatchDriver.java b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/MatchDriver.java index 82f24d72f5450..2d051ad4c872c 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/MatchDriver.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/MatchDriver.java @@ -19,6 +19,9 @@ package org.apache.flink.runtime.operators; +import org.apache.flink.api.common.ExecutionConfig; +import org.apache.flink.runtime.operators.hash.NonReusingBuildFirstHashMatchIterator; +import org.apache.flink.runtime.operators.hash.NonReusingBuildSecondHashMatchIterator; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import org.apache.flink.api.common.functions.FlatJoinFunction; @@ -27,8 +30,8 @@ import org.apache.flink.api.common.typeutils.TypeSerializer; import org.apache.flink.runtime.io.disk.iomanager.IOManager; import org.apache.flink.runtime.memorymanager.MemoryManager; -import org.apache.flink.runtime.operators.hash.BuildFirstHashMatchIterator; -import org.apache.flink.runtime.operators.hash.BuildSecondHashMatchIterator; +import org.apache.flink.runtime.operators.hash.ReusingBuildFirstHashMatchIterator; +import org.apache.flink.runtime.operators.hash.ReusingBuildSecondHashMatchIterator; import org.apache.flink.runtime.operators.sort.MergeMatchIterator; import org.apache.flink.runtime.operators.util.JoinTaskIterator; import org.apache.flink.runtime.operators.util.TaskConfig; @@ -53,7 +56,9 @@ public class MatchDriver implements PactDriver matchIterator; // the iterator that does the actual matching protected volatile boolean running; - + + private boolean objectReuseEnabled = false; + // ------------------------------------------------------------------------ @Override @@ -96,7 +101,7 @@ public void prepare() throws Exception{ final MutableObjectIterator in1 = this.taskContext.getInput(0); final MutableObjectIterator in2 = this.taskContext.getInput(1); - + // get the key positions and types final TypeSerializer serializer1 = this.taskContext.getInputSerializer(0).getSerializer(); final TypeSerializer serializer2 = this.taskContext.getInputSerializer(1).getSerializer(); @@ -109,25 +114,44 @@ public void prepare() throws Exception{ throw new Exception("Missing pair comparator factory for Match driver"); } + ExecutionConfig executionConfig = taskContext.getExecutionConfig(); + this.objectReuseEnabled = executionConfig.isObjectReuseEnabled(); + + if (LOG.isDebugEnabled()) { + LOG.debug("MatchDriver object reuse: " + (this.objectReuseEnabled ? "ENABLED" : "DISABLED") + "."); + } + // create and return MatchTaskIterator according to provided local strategy. - switch (ls) { - case MERGE: - this.matchIterator = new MergeMatchIterator(in1, in2, serializer1, comparator1, - serializer2, comparator2, pairComparatorFactory.createComparator12(comparator1, comparator2), - memoryManager, ioManager, numPages, this.taskContext.getOwningNepheleTask()); - break; - case HYBRIDHASH_BUILD_FIRST: - this.matchIterator = new BuildFirstHashMatchIterator(in1, in2, serializer1, comparator1, - serializer2, comparator2, pairComparatorFactory.createComparator21(comparator1, comparator2), - memoryManager, ioManager, this.taskContext.getOwningNepheleTask(), fractionAvailableMemory); - break; - case HYBRIDHASH_BUILD_SECOND: - this.matchIterator = new BuildSecondHashMatchIterator(in1, in2, serializer1, comparator1, - serializer2, comparator2, pairComparatorFactory.createComparator12(comparator1, comparator2), - memoryManager, ioManager, this.taskContext.getOwningNepheleTask(), fractionAvailableMemory); - break; - default: - throw new Exception("Unsupported driver strategy for Match driver: " + ls.name()); + if (this.objectReuseEnabled) { + switch (ls) { + case MERGE: + this.matchIterator = new MergeMatchIterator(in1, in2, serializer1, comparator1, serializer2, comparator2, pairComparatorFactory.createComparator12(comparator1, comparator2), memoryManager, ioManager, numPages, this.taskContext.getOwningNepheleTask()); + + break; + case HYBRIDHASH_BUILD_FIRST: + this.matchIterator = new ReusingBuildFirstHashMatchIterator(in1, in2, serializer1, comparator1, serializer2, comparator2, pairComparatorFactory.createComparator21(comparator1, comparator2), memoryManager, ioManager, this.taskContext.getOwningNepheleTask(), fractionAvailableMemory); + break; + case HYBRIDHASH_BUILD_SECOND: + this.matchIterator = new ReusingBuildSecondHashMatchIterator(in1, in2, serializer1, comparator1, serializer2, comparator2, pairComparatorFactory.createComparator12(comparator1, comparator2), memoryManager, ioManager, this.taskContext.getOwningNepheleTask(), fractionAvailableMemory); + break; + default: + throw new Exception("Unsupported driver strategy for Match driver: " + ls.name()); + } + } else { + switch (ls) { + case MERGE: + this.matchIterator = new MergeMatchIterator(in1, in2, serializer1, comparator1, serializer2, comparator2, pairComparatorFactory.createComparator12(comparator1, comparator2), memoryManager, ioManager, numPages, this.taskContext.getOwningNepheleTask()); + + break; + case HYBRIDHASH_BUILD_FIRST: + this.matchIterator = new NonReusingBuildFirstHashMatchIterator(in1, in2, serializer1, comparator1, serializer2, comparator2, pairComparatorFactory.createComparator21(comparator1, comparator2), memoryManager, ioManager, this.taskContext.getOwningNepheleTask(), fractionAvailableMemory); + break; + case HYBRIDHASH_BUILD_SECOND: + this.matchIterator = new NonReusingBuildSecondHashMatchIterator(in1, in2, serializer1, comparator1, serializer2, comparator2, pairComparatorFactory.createComparator12(comparator1, comparator2), memoryManager, ioManager, this.taskContext.getOwningNepheleTask(), fractionAvailableMemory); + break; + default: + throw new Exception("Unsupported driver strategy for Match driver: " + ls.name()); + } } // open MatchTaskIterator - this triggers the sorting or hash-table building diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/NoOpDriver.java b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/NoOpDriver.java index 3eeb2d58431ed..1fb4813266aa2 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/NoOpDriver.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/NoOpDriver.java @@ -19,9 +19,13 @@ package org.apache.flink.runtime.operators; +import org.apache.flink.api.common.ExecutionConfig; import org.apache.flink.api.common.functions.AbstractRichFunction; +import org.apache.flink.api.common.typeutils.TypeSerializer; import org.apache.flink.util.Collector; import org.apache.flink.util.MutableObjectIterator; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; /** * A driver that does nothing but forward data from its input to its output. @@ -29,12 +33,15 @@ * @param The data type. */ public class NoOpDriver implements PactDriver { - + + private static final Logger LOG = LoggerFactory.getLogger(MapPartitionDriver.class); + private PactTaskContext taskContext; private volatile boolean running; - - + + private boolean objectReuseEnabled = false; + @Override public void setup(PactTaskContext context) { this.taskContext = context; @@ -57,17 +64,34 @@ public int getNumberOfDriverComparators() { } @Override - public void prepare() {} + public void prepare() { + ExecutionConfig executionConfig = taskContext.getExecutionConfig(); + this.objectReuseEnabled = executionConfig.isObjectReuseEnabled(); + + if (LOG.isDebugEnabled()) { + LOG.debug("NoOpDriver object reuse: " + (this.objectReuseEnabled ? "ENABLED" : "DISABLED") + "."); + } + } @Override public void run() throws Exception { // cache references on the stack final MutableObjectIterator input = this.taskContext.getInput(0); final Collector output = this.taskContext.getOutputCollector(); - T record = this.taskContext.getInputSerializer(0).getSerializer().createInstance(); - while (this.running && ((record = input.next(record)) != null)) { - output.collect(record); + if (objectReuseEnabled) { + T record = this.taskContext.getInputSerializer(0).getSerializer().createInstance(); + + while (this.running && ((record = input.next(record)) != null)) { + output.collect(record); + } + } else { + T record; + TypeSerializer serializer = this.taskContext.getInputSerializer(0).getSerializer(); + while (this.running && ((record = input.next(serializer.createInstance())) != null)) { + output.collect(record); + } + } } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/PactTaskContext.java b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/PactTaskContext.java index f6d46fa1b4c0b..f6d2f4a96d4a0 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/PactTaskContext.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/PactTaskContext.java @@ -19,6 +19,7 @@ package org.apache.flink.runtime.operators; +import org.apache.flink.api.common.ExecutionConfig; import org.apache.flink.api.common.typeutils.TypeComparator; import org.apache.flink.api.common.typeutils.TypeSerializerFactory; import org.apache.flink.runtime.io.disk.iomanager.IOManager; @@ -43,7 +44,9 @@ public interface PactTaskContext { TaskConfig getTaskConfig(); - + + ExecutionConfig getExecutionConfig(); + ClassLoader getUserCodeClassLoader(); MemoryManager getMemoryManager(); diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/ReduceCombineDriver.java b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/ReduceCombineDriver.java index 35fb73949a1e6..f3ea7004d1cbc 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/ReduceCombineDriver.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/ReduceCombineDriver.java @@ -22,6 +22,7 @@ import java.io.IOException; import java.util.List; +import org.apache.flink.api.common.ExecutionConfig; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import org.apache.flink.api.common.functions.ReduceFunction; @@ -62,16 +63,17 @@ public class ReduceCombineDriver implements PactDriver, T> private Collector output; - private MemoryManager memManager; private InMemorySorter sorter; private QuickSort sortAlgo = new QuickSort(); - - + private boolean running; + private boolean objectReuseEnabled = false; + + // ------------------------------------------------------------------------ @Override @@ -124,6 +126,13 @@ public void prepare() throws Exception { } else { this.sorter = new NormalizedKeySorter(this.serializer, this.comparator.duplicate(), memory); } + + ExecutionConfig executionConfig = taskContext.getExecutionConfig(); + this.objectReuseEnabled = executionConfig.isObjectReuseEnabled(); + + if (LOG.isDebugEnabled()) { + LOG.debug("ReduceCombineDriver object reuse: " + (this.objectReuseEnabled ? "ENABLED" : "DISABLED") + "."); + } } @Override @@ -172,26 +181,60 @@ private void sortAndCombine() throws Exception { final Collector output = this.output; final MutableObjectIterator input = sorter.getIterator(); - - T value = input.next(serializer.createInstance()); - - // iterate over key groups - while (this.running && value != null) { - comparator.setReference(value); - T res = value; - - // iterate within a key group - while ((value = input.next(serializer.createInstance())) != null) { - if (comparator.equalToReference(value)) { - // same group, reduce - res = function.reduce(res, value); - } else { - // new key group - break; + + if (objectReuseEnabled) { + // We only need two objects. The user function is expected to return + // the first input as the result. The output value is also expected + // to have the same key fields as the input elements. + + T reuse1 = serializer.createInstance(); + T reuse2 = serializer.createInstance(); + + T value = input.next(reuse1); + + // iterate over key groups + while (this.running && value != null) { + comparator.setReference(value); + T res = value; + + // iterate within a key group + while ((value = input.next(reuse2)) != null) { + if (comparator.equalToReference(value)) { + // same group, reduce + res = function.reduce(res, value); + } else { + // new key group + break; + } } + + output.collect(res); + + if (value != null) { + value = serializer.copy(value, reuse1); + } + } + } else { + T value = input.next(serializer.createInstance()); + + // iterate over key groups + while (this.running && value != null) { + comparator.setReference(value); + T res = value; + + // iterate within a key group + while ((value = input.next(serializer.createInstance())) != null) { + if (comparator.equalToReference(value)) { + // same group, reduce + res = function.reduce(res, value); + } else { + // new key group + break; + } + } + + output.collect(res); } - - output.collect(res); } } } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/ReduceDriver.java b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/ReduceDriver.java index 9cdcdda20e939..6528f3011c376 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/ReduceDriver.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/ReduceDriver.java @@ -19,6 +19,7 @@ package org.apache.flink.runtime.operators; +import org.apache.flink.api.common.ExecutionConfig; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import org.apache.flink.api.common.functions.ReduceFunction; @@ -52,6 +53,8 @@ public class ReduceDriver implements PactDriver, T> { private volatile boolean running; + private boolean objectReuseEnabled = false; + // ------------------------------------------------------------------------ @Override @@ -88,6 +91,13 @@ public void prepare() throws Exception { this.serializer = this.taskContext.getInputSerializer(0).getSerializer(); this.comparator = this.taskContext.getDriverComparator(0); this.input = this.taskContext.getInput(0); + + ExecutionConfig executionConfig = taskContext.getExecutionConfig(); + this.objectReuseEnabled = executionConfig.isObjectReuseEnabled(); + + if (LOG.isDebugEnabled()) { + LOG.debug("ReduceDriver object reuse: " + (this.objectReuseEnabled ? "ENABLED" : "DISABLED") + "."); + } } @Override @@ -104,26 +114,60 @@ public void run() throws Exception { final ReduceFunction function = this.taskContext.getStub(); final Collector output = this.taskContext.getOutputCollector(); - - T value = input.next(serializer.createInstance()); - - // iterate over key groups - while (this.running && value != null) { - comparator.setReference(value); - T res = value; - - // iterate within a key group - while ((value = input.next(serializer.createInstance())) != null) { - if (comparator.equalToReference(value)) { - // same group, reduce - res = function.reduce(res, value); - } else { - // new key group - break; + + if (objectReuseEnabled) { + // We only need two objects. The user function is expected to return + // the first input as the result. The output value is also expected + // to have the same key fields as the input elements. + + T reuse1 = serializer.createInstance(); + T reuse2 = serializer.createInstance(); + + T value = input.next(reuse1); + + // iterate over key groups + while (this.running && value != null) { + comparator.setReference(value); + T res = value; + + // iterate within a key group + while ((value = input.next(reuse2)) != null) { + if (comparator.equalToReference(value)) { + // same group, reduce + res = function.reduce(res, value); + } else { + // new key group + break; + } + } + + output.collect(res); + + if (value != null) { + value = serializer.copy(value, reuse1); } } - - output.collect(res); + } else { + T value = input.next(serializer.createInstance()); + + // iterate over key groups + while (this.running && value != null) { + comparator.setReference(value); + T res = value; + + // iterate within a key group + while ((value = input.next(serializer.createInstance())) != null) { + if (comparator.equalToReference(value)) { + // same group, reduce + res = function.reduce(res, value); + } else { + // new key group + break; + } + } + + output.collect(res); + } } } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/RegularPactTask.java b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/RegularPactTask.java index 74bc4cbd963b3..db36f6d838bf8 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/RegularPactTask.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/RegularPactTask.java @@ -20,6 +20,7 @@ import akka.actor.ActorRef; import org.apache.flink.runtime.messages.JobManagerMessages; +import org.apache.flink.api.common.ExecutionConfig; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import org.apache.flink.api.common.accumulators.Accumulator; @@ -1057,7 +1058,7 @@ protected void initOutputs() throws Exception { ClassLoader userCodeClassLoader = getUserCodeClassLoader(); - this.output = initOutputs(this, userCodeClassLoader, this.config, this.chainedTasks, this.eventualOutputs); + this.output = initOutputs(this, userCodeClassLoader, this.config, this.chainedTasks, this.eventualOutputs, this.getExecutionConfig()); } public DistributedRuntimeUDFContext createRuntimeContext(String taskName) { @@ -1075,6 +1076,25 @@ public TaskConfig getTaskConfig() { return this.config; } + @Override + public ExecutionConfig getExecutionConfig() { + try { + ExecutionConfig c = (ExecutionConfig) InstantiationUtil.readObjectFromConfig( + getOwningNepheleTask().getJobConfiguration(), + ExecutionConfig.CONFIG_KEY, + this.getClass().getClassLoader()); + if (c != null) { + return c; + } else { + return new ExecutionConfig(); + } + } catch (IOException e) { + throw new RuntimeException("Could not load ExecutionConfig from Job Configuration: " + e); + } catch (ClassNotFoundException e) { + throw new RuntimeException("Could not load ExecutionConfig from Job Configuration: " + e); + } + } + @Override public MemoryManager getMemoryManager() { return getEnvironment().getMemoryManager(); @@ -1318,7 +1338,7 @@ public static Collector getOutputCollector(AbstractInvokable task, TaskCo */ @SuppressWarnings("unchecked") public static Collector initOutputs(AbstractInvokable nepheleTask, ClassLoader cl, TaskConfig config, - List> chainedTasksTarget, List eventualOutputs) + List> chainedTasksTarget, List eventualOutputs, ExecutionConfig executionConfig) throws Exception { final int numOutputs = config.getNumOutputs(); @@ -1355,7 +1375,7 @@ public static Collector initOutputs(AbstractInvokable nepheleTask, ClassL previous = getOutputCollector(nepheleTask, chainedStubConf, cl, eventualOutputs, chainedStubConf.getNumOutputs()); } - ct.setup(chainedStubConf, taskName, previous, nepheleTask, cl); + ct.setup(chainedStubConf, taskName, previous, nepheleTask, cl, executionConfig); chainedTasksTarget.add(0, ct); previous = ct; diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/chaining/ChainedDriver.java b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/chaining/ChainedDriver.java index 28cebf06f5e83..c36e74231e8ec 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/chaining/ChainedDriver.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/chaining/ChainedDriver.java @@ -18,6 +18,7 @@ package org.apache.flink.runtime.operators.chaining; +import org.apache.flink.api.common.ExecutionConfig; import org.apache.flink.api.common.functions.Function; import org.apache.flink.api.common.functions.RuntimeContext; import org.apache.flink.runtime.execution.Environment; @@ -43,9 +44,13 @@ public abstract class ChainedDriver implements Collector { private DistributedRuntimeUDFContext udfContext; + protected ExecutionConfig executionConfig; + + protected boolean objectReuseEnabled = false; + public void setup(TaskConfig config, String taskName, Collector outputCollector, - AbstractInvokable parent, ClassLoader userCodeClassLoader) + AbstractInvokable parent, ClassLoader userCodeClassLoader, ExecutionConfig executionConfig) { this.config = config; this.taskName = taskName; @@ -60,6 +65,9 @@ public void setup(TaskConfig config, String taskName, Collector outputCollec env.getIndexInSubtaskGroup(), userCodeClassLoader, env.getCopyTask()); } + this.executionConfig = executionConfig; + this.objectReuseEnabled = executionConfig.isObjectReuseEnabled(); + setup(parent); } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/chaining/SynchronousChainedCombineDriver.java b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/chaining/SynchronousChainedCombineDriver.java index 5492635a197d3..65426ef562d68 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/chaining/SynchronousChainedCombineDriver.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/chaining/SynchronousChainedCombineDriver.java @@ -38,11 +38,17 @@ import org.apache.flink.runtime.operators.sort.InMemorySorter; import org.apache.flink.runtime.operators.sort.NormalizedKeySorter; import org.apache.flink.runtime.operators.sort.QuickSort; -import org.apache.flink.runtime.util.KeyGroupedIterator; +import org.apache.flink.runtime.util.NonReusingKeyGroupedIterator; +import org.apache.flink.runtime.util.ReusingKeyGroupedIterator; import org.apache.flink.util.Collector; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; public class SynchronousChainedCombineDriver extends ChainedDriver { + private static final Logger LOG = LoggerFactory.getLogger(SynchronousChainedCombineDriver.class); + + /** * Fix length records with a length below this threshold will be in-place sorted, if possible. */ @@ -110,6 +116,10 @@ public void openTask() throws Exception { } else { this.sorter = new NormalizedKeySorter(this.serializer, this.sortingComparator.duplicate(), memory); } + + if (LOG.isDebugEnabled()) { + LOG.debug("SynchronousChainedCombineDriver object reuse: " + (this.objectReuseEnabled ? "ENABLED" : "DISABLED") + "."); + } } @Override @@ -183,19 +193,37 @@ public void close() { private void sortAndCombine() throws Exception { final InMemorySorter sorter = this.sorter; - if (!sorter.isEmpty()) { - this.sortAlgo.sort(sorter); - // run the combiner - final KeyGroupedIterator keyIter = new KeyGroupedIterator(sorter.getIterator(), this.serializer, - this.groupingComparator); + if (objectReuseEnabled) { + if (!sorter.isEmpty()) { + this.sortAlgo.sort(sorter); + // run the combiner + final ReusingKeyGroupedIterator keyIter = new ReusingKeyGroupedIterator(sorter.getIterator(), this.serializer, this.groupingComparator); + + + // cache references on the stack + final FlatCombineFunction stub = this.combiner; + final Collector output = this.outputCollector; + + // run stub implementation + while (this.running && keyIter.nextKey()) { + stub.combine(keyIter.getValues(), output); + } + } + } else { + if (!sorter.isEmpty()) { + this.sortAlgo.sort(sorter); + // run the combiner + final NonReusingKeyGroupedIterator keyIter = new NonReusingKeyGroupedIterator(sorter.getIterator(), this.serializer, this.groupingComparator); + - // cache references on the stack - final FlatCombineFunction stub = this.combiner; - final Collector output = this.outputCollector; + // cache references on the stack + final FlatCombineFunction stub = this.combiner; + final Collector output = this.outputCollector; - // run stub implementation - while (this.running && keyIter.nextKey()) { - stub.combine(keyIter.getValues(), output); + // run stub implementation + while (this.running && keyIter.nextKey()) { + stub.combine(keyIter.getValues(), output); + } } } } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/hash/CompactingHashTable.java b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/hash/CompactingHashTable.java index 4c39f28a7cbcd..3dd400fab390f 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/hash/CompactingHashTable.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/hash/CompactingHashTable.java @@ -1332,7 +1332,7 @@ private HashTableProber(TypeComparator probeTypeComparator, TypePairComparat super(probeTypeComparator, pairComparator); } - public T getMatchFor(PT probeSideRecord, T targetForMatch) { + public T getMatchFor(PT probeSideRecord, T reuse) { if(closed.get()) { return null; } @@ -1372,13 +1372,13 @@ public T getMatchFor(PT probeSideRecord, T targetForMatch) { // deserialize the key to check whether it is really equal, or whether we had only a hash collision try { - targetForMatch = p.readRecordAt(pointer, targetForMatch); + reuse = p.readRecordAt(pointer, reuse); - if (this.pairComparator.equalToReference(targetForMatch)) { + if (this.pairComparator.equalToReference(reuse)) { this.partition = p; this.bucket = bucket; this.pointerOffsetInBucket = pointerOffset; - return targetForMatch; + return reuse; } } catch (IOException e) { @@ -1404,7 +1404,80 @@ public T getMatchFor(PT probeSideRecord, T targetForMatch) { numInSegment = 0; } } - + + public T getMatchFor(PT probeSideRecord) { + if(closed.get()) { + return null; + } + final int searchHashCode = hash(this.probeTypeComparator.hash(probeSideRecord)); + + final int posHashCode = searchHashCode % numBuckets; + + // get the bucket for the given hash code + MemorySegment bucket = buckets[posHashCode >> bucketsPerSegmentBits]; + int bucketInSegmentOffset = (posHashCode & bucketsPerSegmentMask) << NUM_INTRA_BUCKET_BITS; + + // get the basic characteristics of the bucket + final int partitionNumber = bucket.get(bucketInSegmentOffset + HEADER_PARTITION_OFFSET); + final InMemoryPartition p = partitions.get(partitionNumber); + final MemorySegment[] overflowSegments = p.overflowSegments; + + this.pairComparator.setReference(probeSideRecord); + + int countInSegment = bucket.getInt(bucketInSegmentOffset + HEADER_COUNT_OFFSET); + int numInSegment = 0; + int posInSegment = bucketInSegmentOffset + BUCKET_HEADER_LENGTH; + + // loop over all segments that are involved in the bucket (original bucket plus overflow buckets) + while (true) { + + while (numInSegment < countInSegment) { + + final int thisCode = bucket.getInt(posInSegment); + posInSegment += HASH_CODE_LEN; + + // check if the hash code matches + if (thisCode == searchHashCode) { + // get the pointer to the pair + final int pointerOffset = bucketInSegmentOffset + BUCKET_POINTER_START_OFFSET + (numInSegment * POINTER_LEN); + final long pointer = bucket.getLong(pointerOffset); + numInSegment++; + + // deserialize the key to check whether it is really equal, or whether we had only a hash collision + try { + T result = p.readRecordAt(pointer); + + if (this.pairComparator.equalToReference(result)) { + this.partition = p; + this.bucket = bucket; + this.pointerOffsetInBucket = pointerOffset; + return result; + } + } + catch (IOException e) { + throw new RuntimeException("Error deserializing record from the hashtable: " + e.getMessage(), e); + } + } + else { + numInSegment++; + } + } + + // this segment is done. check if there is another chained bucket + final long forwardPointer = bucket.getLong(bucketInSegmentOffset + HEADER_FORWARD_OFFSET); + if (forwardPointer == BUCKET_FORWARD_POINTER_NOT_SET) { + return null; + } + + final int overflowSegNum = (int) (forwardPointer >>> 32); + bucket = overflowSegments[overflowSegNum]; + bucketInSegmentOffset = (int) (forwardPointer & 0xffffffff); + countInSegment = bucket.getInt(bucketInSegmentOffset + HEADER_COUNT_OFFSET); + posInSegment = bucketInSegmentOffset + BUCKET_HEADER_LENGTH; + numInSegment = 0; + } + } + public void updateMatch(T record) throws IOException { if(closed.get()) { return; diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/hash/HashMatchIteratorBase.java b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/hash/HashMatchIteratorBase.java new file mode 100644 index 0000000000000..4e0112ab806ef --- /dev/null +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/hash/HashMatchIteratorBase.java @@ -0,0 +1,51 @@ +/* + * 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.flink.runtime.operators.hash; + +import org.apache.flink.api.common.typeutils.TypeComparator; +import org.apache.flink.api.common.typeutils.TypePairComparator; +import org.apache.flink.api.common.typeutils.TypeSerializer; +import org.apache.flink.core.memory.MemorySegment; +import org.apache.flink.runtime.io.disk.iomanager.IOManager; +import org.apache.flink.runtime.jobgraph.tasks.AbstractInvokable; +import org.apache.flink.runtime.memorymanager.MemoryAllocationException; +import org.apache.flink.runtime.memorymanager.MemoryManager; + +import java.util.List; + +/** + * Common methods for all Hash Join Iterators. + */ +public class HashMatchIteratorBase { + public MutableHashTable getHashJoin( + TypeSerializer buildSideSerializer, + TypeComparator buildSideComparator, + TypeSerializer probeSideSerializer, + TypeComparator probeSideComparator, + TypePairComparator pairComparator, + MemoryManager memManager, + IOManager ioManager, + AbstractInvokable ownerTask, + double memoryFraction) throws MemoryAllocationException { + + final int numPages = memManager.computeNumberOfPages(memoryFraction); + final List memorySegments = memManager.allocatePages(ownerTask, numPages); + return new MutableHashTable(buildSideSerializer, probeSideSerializer, buildSideComparator, probeSideComparator, pairComparator, memorySegments, ioManager); + } + +} diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/hash/InMemoryPartition.java b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/hash/InMemoryPartition.java index ca3eb4ebdc353..7fb997e0fa9c8 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/hash/InMemoryPartition.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/hash/InMemoryPartition.java @@ -235,6 +235,11 @@ public T readRecordAt(long pointer, T reuse) throws IOException { this.readView.setReadPosition(pointer); return this.serializer.deserialize(reuse, this.readView); } + + public T readRecordAt(long pointer) throws IOException { + this.readView.setReadPosition(pointer); + return this.serializer.deserialize(this.readView); + } /** * UNSAFE!! overwrites record diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/hash/NonReusingBuildFirstHashMatchIterator.java b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/hash/NonReusingBuildFirstHashMatchIterator.java new file mode 100644 index 0000000000000..c2d780500d1ff --- /dev/null +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/hash/NonReusingBuildFirstHashMatchIterator.java @@ -0,0 +1,152 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + + +package org.apache.flink.runtime.operators.hash; + +import org.apache.flink.api.common.functions.FlatJoinFunction; +import org.apache.flink.api.common.typeutils.TypeComparator; +import org.apache.flink.api.common.typeutils.TypePairComparator; +import org.apache.flink.api.common.typeutils.TypeSerializer; +import org.apache.flink.core.memory.MemorySegment; +import org.apache.flink.runtime.io.disk.iomanager.IOManager; +import org.apache.flink.runtime.jobgraph.tasks.AbstractInvokable; +import org.apache.flink.runtime.memorymanager.MemoryAllocationException; +import org.apache.flink.runtime.memorymanager.MemoryManager; +import org.apache.flink.runtime.operators.util.JoinTaskIterator; +import org.apache.flink.util.Collector; +import org.apache.flink.util.MutableObjectIterator; + +import java.io.IOException; +import java.util.List; + + +/** + * An implementation of the {@link org.apache.flink.runtime.operators.util.JoinTaskIterator} that uses a hybrid-hash-join + * internally to match the records with equal key. The build side of the hash is the first input of the match. + * This implementation DOES NOT reuse objects. + */ +public class NonReusingBuildFirstHashMatchIterator extends HashMatchIteratorBase implements JoinTaskIterator { + + protected final MutableHashTable hashJoin; + + protected final TypeSerializer probeSideSerializer; + + private final MemoryManager memManager; + + private final MutableObjectIterator firstInput; + + private final MutableObjectIterator secondInput; + + private volatile boolean running = true; + + // -------------------------------------------------------------------------------------------- + + public NonReusingBuildFirstHashMatchIterator( + MutableObjectIterator firstInput, + MutableObjectIterator secondInput, + TypeSerializer serializer1, + TypeComparator comparator1, + TypeSerializer serializer2, + TypeComparator comparator2, + TypePairComparator pairComparator, + MemoryManager memManager, IOManager ioManager, + AbstractInvokable ownerTask, + double memoryFraction) + throws MemoryAllocationException + { + this.memManager = memManager; + this.firstInput = firstInput; + this.secondInput = secondInput; + this.probeSideSerializer = serializer2; + + this.hashJoin = getHashJoin(serializer1, comparator1, serializer2, comparator2, + pairComparator, memManager, ioManager, ownerTask, memoryFraction); + } + + // -------------------------------------------------------------------------------------------- + + @Override + public void open() throws IOException, MemoryAllocationException, InterruptedException { + this.hashJoin.open(this.firstInput, this.secondInput); + } + + + @Override + public void close() { + // close the join + this.hashJoin.close(); + + // free the memory + final List segments = this.hashJoin.getFreedMemory(); + this.memManager.release(segments); + } + + @Override + public final boolean callWithNextKey(FlatJoinFunction matchFunction, Collector collector) + throws Exception + { + if (this.hashJoin.nextRecord()) + { + // we have a next record, get the iterators to the probe and build side values + final MutableHashTable.HashBucketIterator buildSideIterator = this.hashJoin.getBuildSideIterator(); + V1 nextBuildSideRecord; + + // get the first build side value + if ((nextBuildSideRecord = buildSideIterator.next()) != null) { + V1 tmpRec; + final V2 probeRecord = this.hashJoin.getCurrentProbeRecord(); + + // check if there is another build-side value + if ((tmpRec = buildSideIterator.next()) != null) { + // more than one build-side value --> copy the probe side + V2 probeCopy; + probeCopy = this.probeSideSerializer.copy(probeRecord); + + // call match on the first pair + matchFunction.join(nextBuildSideRecord, probeCopy, collector); + + // call match on the second pair + probeCopy = this.probeSideSerializer.copy(probeRecord); + matchFunction.join(tmpRec, probeCopy, collector); + + while (this.running && ((nextBuildSideRecord = buildSideIterator.next()) != null)) { + // call match on the next pair + // make sure we restore the value of the probe side record + probeCopy = this.probeSideSerializer.copy(probeRecord); + matchFunction.join(nextBuildSideRecord, probeCopy, collector); + } + } + else { + // only single pair matches + matchFunction.join(nextBuildSideRecord, probeRecord, collector); + } + } + return true; + } + else { + return false; + } + } + + @Override + public void abort() { + this.running = false; + this.hashJoin.abort(); + } +} diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/hash/NonReusingBuildFirstReOpenableHashMatchIterator.java b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/hash/NonReusingBuildFirstReOpenableHashMatchIterator.java new file mode 100644 index 0000000000000..ee870a67619b3 --- /dev/null +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/hash/NonReusingBuildFirstReOpenableHashMatchIterator.java @@ -0,0 +1,80 @@ +/* + * 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.flink.runtime.operators.hash; + +import org.apache.flink.api.common.typeutils.TypeComparator; +import org.apache.flink.api.common.typeutils.TypePairComparator; +import org.apache.flink.api.common.typeutils.TypeSerializer; +import org.apache.flink.core.memory.MemorySegment; +import org.apache.flink.runtime.io.disk.iomanager.IOManager; +import org.apache.flink.runtime.jobgraph.tasks.AbstractInvokable; +import org.apache.flink.runtime.memorymanager.MemoryAllocationException; +import org.apache.flink.runtime.memorymanager.MemoryManager; +import org.apache.flink.util.MutableObjectIterator; + +import java.io.IOException; +import java.util.List; + +public class NonReusingBuildFirstReOpenableHashMatchIterator extends NonReusingBuildFirstHashMatchIterator { + + + private final ReOpenableMutableHashTable reopenHashTable; + + public NonReusingBuildFirstReOpenableHashMatchIterator( + MutableObjectIterator firstInput, + MutableObjectIterator secondInput, + TypeSerializer serializer1, + TypeComparator comparator1, + TypeSerializer serializer2, + TypeComparator comparator2, + TypePairComparator pairComparator, + MemoryManager memManager, + IOManager ioManager, + AbstractInvokable ownerTask, + double memoryFraction) + throws MemoryAllocationException + { + super(firstInput, secondInput, serializer1, comparator1, serializer2, + comparator2, pairComparator, memManager, ioManager, ownerTask, + memoryFraction); + reopenHashTable = (ReOpenableMutableHashTable) hashJoin; + } + + @Override + public MutableHashTable getHashJoin(TypeSerializer buildSideSerializer, TypeComparator buildSideComparator, + TypeSerializer probeSideSerializer, TypeComparator probeSideComparator, + TypePairComparator pairComparator, + MemoryManager memManager, IOManager ioManager, AbstractInvokable ownerTask, double memoryFraction) + throws MemoryAllocationException + { + final int numPages = memManager.computeNumberOfPages(memoryFraction); + final List memorySegments = memManager.allocatePages(ownerTask, numPages); + return new ReOpenableMutableHashTable(buildSideSerializer, probeSideSerializer, buildSideComparator, probeSideComparator, pairComparator, memorySegments, ioManager); + } + + /** + * Set new input for probe side + * @throws java.io.IOException + */ + public void reopenProbe(MutableObjectIterator probeInput) throws IOException { + reopenHashTable.reopenProbe(probeInput); + } + +} diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/hash/NonReusingBuildSecondHashMatchIterator.java b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/hash/NonReusingBuildSecondHashMatchIterator.java new file mode 100644 index 0000000000000..6099ac76295ae --- /dev/null +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/hash/NonReusingBuildSecondHashMatchIterator.java @@ -0,0 +1,150 @@ +/* + * 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.flink.runtime.operators.hash; + +import org.apache.flink.api.common.functions.FlatJoinFunction; +import org.apache.flink.api.common.typeutils.TypeComparator; +import org.apache.flink.api.common.typeutils.TypePairComparator; +import org.apache.flink.api.common.typeutils.TypeSerializer; +import org.apache.flink.core.memory.MemorySegment; +import org.apache.flink.runtime.io.disk.iomanager.IOManager; +import org.apache.flink.runtime.jobgraph.tasks.AbstractInvokable; +import org.apache.flink.runtime.memorymanager.MemoryAllocationException; +import org.apache.flink.runtime.memorymanager.MemoryManager; +import org.apache.flink.runtime.operators.util.JoinTaskIterator; +import org.apache.flink.util.Collector; +import org.apache.flink.util.MutableObjectIterator; + +import java.io.IOException; +import java.util.List; + + +/** + * An implementation of the {@link org.apache.flink.runtime.operators.util.JoinTaskIterator} that uses a hybrid-hash-join + * internally to match the records with equal key. The build side of the hash is the second input of the match. + */ +public class NonReusingBuildSecondHashMatchIterator extends HashMatchIteratorBase implements JoinTaskIterator { + + protected final MutableHashTable hashJoin; + + protected final TypeSerializer probeSideSerializer; + + private final MemoryManager memManager; + + private final MutableObjectIterator firstInput; + + private final MutableObjectIterator secondInput; + + private volatile boolean running = true; + + // -------------------------------------------------------------------------------------------- + + public NonReusingBuildSecondHashMatchIterator( + MutableObjectIterator firstInput, + MutableObjectIterator secondInput, + TypeSerializer serializer1, + TypeComparator comparator1, + TypeSerializer serializer2, + TypeComparator comparator2, + TypePairComparator pairComparator, + MemoryManager memManager, IOManager ioManager, + AbstractInvokable ownerTask, + double memoryFraction) + throws MemoryAllocationException + { + this.memManager = memManager; + this.firstInput = firstInput; + this.secondInput = secondInput; + this.probeSideSerializer = serializer1; + + this.hashJoin = getHashJoin(serializer2, comparator2, serializer1, + comparator1, pairComparator, memManager, ioManager, ownerTask, memoryFraction); + } + + // -------------------------------------------------------------------------------------------- + + @Override + public void open() throws IOException, MemoryAllocationException, InterruptedException { + this.hashJoin.open(this.secondInput, this.firstInput); + } + + @Override + public void close() { + // close the join + this.hashJoin.close(); + + // free the memory + final List segments = this.hashJoin.getFreedMemory(); + this.memManager.release(segments); + } + + @Override + public boolean callWithNextKey(FlatJoinFunction matchFunction, Collector collector) + throws Exception + { + if (this.hashJoin.nextRecord()) + { + // we have a next record, get the iterators to the probe and build side values + final MutableHashTable.HashBucketIterator buildSideIterator = this.hashJoin.getBuildSideIterator(); + V2 nextBuildSideRecord; + + // get the first build side value + if ((nextBuildSideRecord = buildSideIterator.next()) != null) { + V2 tmpRec; + final V1 probeRecord = this.hashJoin.getCurrentProbeRecord(); + + // check if there is another build-side value + if ((tmpRec = buildSideIterator.next()) != null) { + // more than one build-side value --> copy the probe side + V1 probeCopy; + probeCopy = this.probeSideSerializer.copy(probeRecord); + + // call match on the first pair + matchFunction.join(probeCopy, nextBuildSideRecord, collector); + + // call match on the second pair + probeCopy = this.probeSideSerializer.copy(probeRecord); + matchFunction.join(probeCopy, tmpRec, collector); + + while (this.running && ((nextBuildSideRecord = buildSideIterator.next()) != null)) { + // call match on the next pair + // make sure we restore the value of the probe side record + probeCopy = this.probeSideSerializer.copy(probeRecord); + matchFunction.join(probeCopy, nextBuildSideRecord, collector); + } + } + else { + // only single pair matches + matchFunction.join(probeRecord, nextBuildSideRecord, collector); + } + } + return true; + } + else { + return false; + } + } + + @Override + public void abort() { + this.running = false; + this.hashJoin.abort(); + } +} diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/hash/NonReusingBuildSecondReOpenableHashMatchIterator.java b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/hash/NonReusingBuildSecondReOpenableHashMatchIterator.java new file mode 100644 index 0000000000000..bc7e65b09b225 --- /dev/null +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/hash/NonReusingBuildSecondReOpenableHashMatchIterator.java @@ -0,0 +1,81 @@ +/* + * 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.flink.runtime.operators.hash; + +import org.apache.flink.api.common.typeutils.TypeComparator; +import org.apache.flink.api.common.typeutils.TypePairComparator; +import org.apache.flink.api.common.typeutils.TypeSerializer; +import org.apache.flink.core.memory.MemorySegment; +import org.apache.flink.runtime.io.disk.iomanager.IOManager; +import org.apache.flink.runtime.jobgraph.tasks.AbstractInvokable; +import org.apache.flink.runtime.memorymanager.MemoryAllocationException; +import org.apache.flink.runtime.memorymanager.MemoryManager; +import org.apache.flink.util.MutableObjectIterator; + +import java.io.IOException; +import java.util.List; + +public class NonReusingBuildSecondReOpenableHashMatchIterator extends NonReusingBuildSecondHashMatchIterator { + + + private final ReOpenableMutableHashTable reopenHashTable; + + public NonReusingBuildSecondReOpenableHashMatchIterator( + MutableObjectIterator firstInput, + MutableObjectIterator secondInput, + TypeSerializer serializer1, + TypeComparator comparator1, + TypeSerializer serializer2, + TypeComparator comparator2, + TypePairComparator pairComparator, + MemoryManager memManager, + IOManager ioManager, + AbstractInvokable ownerTask, + double memoryFraction) + throws MemoryAllocationException + { + super(firstInput, secondInput, serializer1, comparator1, serializer2, + comparator2, pairComparator, memManager, ioManager, ownerTask, memoryFraction); + reopenHashTable = (ReOpenableMutableHashTable) hashJoin; + } + + @Override + public MutableHashTable getHashJoin( + TypeSerializer buildSideSerializer, + TypeComparator buildSideComparator, + TypeSerializer probeSideSerializer, TypeComparator probeSideComparator, + TypePairComparator pairComparator, + MemoryManager memManager, IOManager ioManager, AbstractInvokable ownerTask, double memoryFraction) + throws MemoryAllocationException + { + final int numPages = memManager.computeNumberOfPages(memoryFraction); + final List memorySegments = memManager.allocatePages(ownerTask, numPages); + return new ReOpenableMutableHashTable(buildSideSerializer, probeSideSerializer, buildSideComparator, probeSideComparator, pairComparator, memorySegments, ioManager); + } + + /** + * Set new input for probe side + * @throws java.io.IOException + */ + public void reopenProbe(MutableObjectIterator probeInput) throws IOException { + reopenHashTable.reopenProbe(probeInput); + } + +} diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/hash/BuildFirstHashMatchIterator.java b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/hash/ReusingBuildFirstHashMatchIterator.java similarity index 69% rename from flink-runtime/src/main/java/org/apache/flink/runtime/operators/hash/BuildFirstHashMatchIterator.java rename to flink-runtime/src/main/java/org/apache/flink/runtime/operators/hash/ReusingBuildFirstHashMatchIterator.java index a65633fc89bbe..da76045016149 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/hash/BuildFirstHashMatchIterator.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/hash/ReusingBuildFirstHashMatchIterator.java @@ -38,18 +38,16 @@ /** * An implementation of the {@link org.apache.flink.runtime.operators.util.JoinTaskIterator} that uses a hybrid-hash-join - * internally to match the records with equal key. The build side of the hash is the first input of the match. + * internally to match the records with equal key. The build side of the hash is the first input of the match. */ -public class BuildFirstHashMatchIterator implements JoinTaskIterator { +public class ReusingBuildFirstHashMatchIterator extends HashMatchIteratorBase implements JoinTaskIterator { protected final MutableHashTable hashJoin; private final V1 nextBuildSideObject; - + private final V1 tempBuildSideRecord; - - private final V2 probeCopy; - + protected final TypeSerializer probeSideSerializer; private final MemoryManager memManager; @@ -62,11 +60,18 @@ public class BuildFirstHashMatchIterator implements JoinTaskIterator< // -------------------------------------------------------------------------------------------- - public BuildFirstHashMatchIterator(MutableObjectIterator firstInput, MutableObjectIterator secondInput, - TypeSerializer serializer1, TypeComparator comparator1, - TypeSerializer serializer2, TypeComparator comparator2, + public ReusingBuildFirstHashMatchIterator( + MutableObjectIterator firstInput, + MutableObjectIterator secondInput, + TypeSerializer serializer1, + TypeComparator comparator1, + TypeSerializer serializer2, + TypeComparator comparator2, TypePairComparator pairComparator, - MemoryManager memManager, IOManager ioManager, AbstractInvokable ownerTask, double memoryFraction) + MemoryManager memManager, + IOManager ioManager, + AbstractInvokable ownerTask, + double memoryFraction) throws MemoryAllocationException { this.memManager = memManager; @@ -76,10 +81,9 @@ public BuildFirstHashMatchIterator(MutableObjectIterator firstInput, Mutable this.nextBuildSideObject = serializer1.createInstance(); this.tempBuildSideRecord = serializer1.createInstance(); - this.probeCopy = serializer2.createInstance(); - - this.hashJoin = getHashJoin(serializer1, comparator1, serializer2, comparator2, pairComparator, - memManager, ioManager, ownerTask, memoryFraction); + + this.hashJoin = getHashJoin(serializer1, comparator1, serializer2, + comparator2, pairComparator, memManager, ioManager, ownerTask, memoryFraction); } // -------------------------------------------------------------------------------------------- @@ -117,22 +121,17 @@ public final boolean callWithNextKey(FlatJoinFunction matchFunction, // check if there is another build-side value if ((tmpRec = buildSideIterator.next(tmpRec)) != null) { - // more than one build-side value --> copy the probe side - V2 probeCopy = this.probeCopy; - probeCopy = this.probeSideSerializer.copy(probeRecord, probeCopy); - + // call match on the first pair - matchFunction.join(nextBuildSideRecord, probeCopy, collector); + matchFunction.join(nextBuildSideRecord, probeRecord, collector); // call match on the second pair - probeCopy = this.probeSideSerializer.copy(probeRecord, probeCopy); - matchFunction.join(tmpRec, probeCopy, collector); + matchFunction.join(tmpRec, probeRecord, collector); while (this.running && ((nextBuildSideRecord = buildSideIterator.next(nextBuildSideRecord)) != null)) { // call match on the next pair // make sure we restore the value of the probe side record - probeCopy = this.probeSideSerializer.copy(probeRecord, probeCopy); - matchFunction.join(nextBuildSideRecord, probeCopy, collector); + matchFunction.join(nextBuildSideRecord, probeRecord, collector); } } else { @@ -152,17 +151,4 @@ public void abort() { this.running = false; this.hashJoin.abort(); } - - // -------------------------------------------------------------------------------------------- - - public MutableHashTable getHashJoin(TypeSerializer buildSideSerializer, TypeComparator buildSideComparator, - TypeSerializer probeSideSerializer, TypeComparator probeSideComparator, - TypePairComparator pairComparator, - MemoryManager memManager, IOManager ioManager, AbstractInvokable ownerTask, double memoryFraction) - throws MemoryAllocationException - { - final int numPages = memManager.computeNumberOfPages(memoryFraction); - final List memorySegments = memManager.allocatePages(ownerTask, numPages); - return new MutableHashTable(buildSideSerializer, probeSideSerializer, buildSideComparator, probeSideComparator, pairComparator, memorySegments, ioManager); - } } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/hash/BuildFirstReOpenableHashMatchIterator.java b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/hash/ReusingBuildFirstReOpenableHashMatchIterator.java similarity index 90% rename from flink-runtime/src/main/java/org/apache/flink/runtime/operators/hash/BuildFirstReOpenableHashMatchIterator.java rename to flink-runtime/src/main/java/org/apache/flink/runtime/operators/hash/ReusingBuildFirstReOpenableHashMatchIterator.java index 9a05264a9a1b2..55012712307b2 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/hash/BuildFirstReOpenableHashMatchIterator.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/hash/ReusingBuildFirstReOpenableHashMatchIterator.java @@ -32,16 +32,18 @@ import org.apache.flink.runtime.memorymanager.MemoryManager; import org.apache.flink.util.MutableObjectIterator; -public class BuildFirstReOpenableHashMatchIterator extends BuildFirstHashMatchIterator { +public class ReusingBuildFirstReOpenableHashMatchIterator extends ReusingBuildFirstHashMatchIterator { private final ReOpenableMutableHashTable reopenHashTable; - public BuildFirstReOpenableHashMatchIterator( + public ReusingBuildFirstReOpenableHashMatchIterator( MutableObjectIterator firstInput, MutableObjectIterator secondInput, - TypeSerializer serializer1, TypeComparator comparator1, - TypeSerializer serializer2, TypeComparator comparator2, + TypeSerializer serializer1, + TypeComparator comparator1, + TypeSerializer serializer2, + TypeComparator comparator2, TypePairComparator pairComparator, MemoryManager memManager, IOManager ioManager, diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/hash/BuildSecondHashMatchIterator.java b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/hash/ReusingBuildSecondHashMatchIterator.java similarity index 72% rename from flink-runtime/src/main/java/org/apache/flink/runtime/operators/hash/BuildSecondHashMatchIterator.java rename to flink-runtime/src/main/java/org/apache/flink/runtime/operators/hash/ReusingBuildSecondHashMatchIterator.java index e1226dacb1517..a9435ef17c1ff 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/hash/BuildSecondHashMatchIterator.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/hash/ReusingBuildSecondHashMatchIterator.java @@ -40,7 +40,7 @@ * An implementation of the {@link org.apache.flink.runtime.operators.util.JoinTaskIterator} that uses a hybrid-hash-join * internally to match the records with equal key. The build side of the hash is the second input of the match. */ -public class BuildSecondHashMatchIterator implements JoinTaskIterator { +public class ReusingBuildSecondHashMatchIterator extends HashMatchIteratorBase implements JoinTaskIterator { protected final MutableHashTable hashJoin; @@ -48,8 +48,6 @@ public class BuildSecondHashMatchIterator implements JoinTaskIterator private final V2 tempBuildSideRecord; - private final V1 probeCopy; - protected final TypeSerializer probeSideSerializer; private final MemoryManager memManager; @@ -62,11 +60,18 @@ public class BuildSecondHashMatchIterator implements JoinTaskIterator // -------------------------------------------------------------------------------------------- - public BuildSecondHashMatchIterator(MutableObjectIterator firstInput, MutableObjectIterator secondInput, - TypeSerializer serializer1, TypeComparator comparator1, - TypeSerializer serializer2, TypeComparator comparator2, + public ReusingBuildSecondHashMatchIterator( + MutableObjectIterator firstInput, + MutableObjectIterator secondInput, + TypeSerializer serializer1, + TypeComparator comparator1, + TypeSerializer serializer2, + TypeComparator comparator2, TypePairComparator pairComparator, - MemoryManager memManager, IOManager ioManager, AbstractInvokable ownerTask, double memoryFraction) + MemoryManager memManager, + IOManager ioManager, + AbstractInvokable ownerTask, + double memoryFraction) throws MemoryAllocationException { this.memManager = memManager; @@ -76,8 +81,7 @@ public BuildSecondHashMatchIterator(MutableObjectIterator firstInput, Mutabl this.nextBuildSideObject = serializer2.createInstance(); this.tempBuildSideRecord = serializer2.createInstance(); - this.probeCopy = serializer1.createInstance(); - + this.hashJoin = getHashJoin(serializer2, comparator2, serializer1, comparator1, pairComparator, memManager, ioManager, ownerTask, memoryFraction); } @@ -116,22 +120,16 @@ public boolean callWithNextKey(FlatJoinFunction matchFunction, Collec // check if there is another build-side value if ((tmpRec = buildSideIterator.next(tmpRec)) != null) { - // more than one build-side value --> copy the probe side - V1 probeCopy = this.probeCopy; - probeCopy = this.probeSideSerializer.copy(probeRecord, probeCopy); - // call match on the first pair - matchFunction.join(probeCopy, nextBuildSideRecord, collector); + matchFunction.join(probeRecord, nextBuildSideRecord, collector); // call match on the second pair - probeCopy = this.probeSideSerializer.copy(probeRecord, probeCopy); - matchFunction.join(probeCopy, tmpRec, collector); + matchFunction.join(probeRecord, tmpRec, collector); while (this.running && ((nextBuildSideRecord = buildSideIterator.next(nextBuildSideRecord)) != null)) { // call match on the next pair // make sure we restore the value of the probe side record - probeCopy = this.probeSideSerializer.copy(probeRecord, probeCopy); - matchFunction.join(probeCopy, nextBuildSideRecord, collector); + matchFunction.join(probeRecord, nextBuildSideRecord, collector); } } else { @@ -151,16 +149,4 @@ public void abort() { this.running = false; this.hashJoin.abort(); } - - public MutableHashTable getHashJoin(TypeSerializer buildSideSerializer, TypeComparator buildSideComparator, - TypeSerializer probeSideSerializer, TypeComparator probeSideComparator, - TypePairComparator pairComparator, - MemoryManager memManager, IOManager ioManager, AbstractInvokable ownerTask, double memoryFraction) - throws MemoryAllocationException - { - final int numPages = memManager.computeNumberOfPages(memoryFraction); - final List memorySegments = memManager.allocatePages(ownerTask, numPages); - return new MutableHashTable(buildSideSerializer, probeSideSerializer, buildSideComparator, probeSideComparator, pairComparator, memorySegments, ioManager); - } - } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/hash/BuildSecondReOpenableHashMatchIterator.java b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/hash/ReusingBuildSecondReOpenableHashMatchIterator.java similarity index 90% rename from flink-runtime/src/main/java/org/apache/flink/runtime/operators/hash/BuildSecondReOpenableHashMatchIterator.java rename to flink-runtime/src/main/java/org/apache/flink/runtime/operators/hash/ReusingBuildSecondReOpenableHashMatchIterator.java index e00585121d0e5..559d20a31bb3a 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/hash/BuildSecondReOpenableHashMatchIterator.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/hash/ReusingBuildSecondReOpenableHashMatchIterator.java @@ -32,16 +32,18 @@ import org.apache.flink.runtime.memorymanager.MemoryManager; import org.apache.flink.util.MutableObjectIterator; -public class BuildSecondReOpenableHashMatchIterator extends BuildSecondHashMatchIterator { +public class ReusingBuildSecondReOpenableHashMatchIterator extends ReusingBuildSecondHashMatchIterator { private final ReOpenableMutableHashTable reopenHashTable; - public BuildSecondReOpenableHashMatchIterator( + public ReusingBuildSecondReOpenableHashMatchIterator( MutableObjectIterator firstInput, MutableObjectIterator secondInput, - TypeSerializer serializer1, TypeComparator comparator1, - TypeSerializer serializer2, TypeComparator comparator2, + TypeSerializer serializer1, + TypeComparator comparator1, + TypeSerializer serializer2, + TypeComparator comparator2, TypePairComparator pairComparator, MemoryManager memManager, IOManager ioManager, diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/sort/CombiningUnilateralSortMerger.java b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/sort/CombiningUnilateralSortMerger.java index 63e64c4a2a234..4f15abf246896 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/sort/CombiningUnilateralSortMerger.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/sort/CombiningUnilateralSortMerger.java @@ -43,7 +43,7 @@ import org.apache.flink.runtime.memorymanager.MemoryAllocationException; import org.apache.flink.runtime.memorymanager.MemoryManager; import org.apache.flink.runtime.util.EmptyMutableObjectIterator; -import org.apache.flink.runtime.util.KeyGroupedIterator; +import org.apache.flink.runtime.util.ReusingKeyGroupedIterator; import org.apache.flink.util.Collector; import org.apache.flink.util.MutableObjectIterator; import org.apache.flink.util.TraversableOnceException; @@ -453,7 +453,7 @@ protected ChannelWithBlockCount mergeChannels(List channe // the list with the target iterators final MergeIterator mergeIterator = getMergingIterator(channelIDs, readBuffers, channelAccesses); - final KeyGroupedIterator groupedIter = new KeyGroupedIterator(mergeIterator, this.serializer, this.comparator2); + final ReusingKeyGroupedIterator groupedIter = new ReusingKeyGroupedIterator(mergeIterator, this.serializer, this.comparator2); // create a new channel writer final FileIOChannel.ID mergedChannelID = this.ioManager.createChannel(); diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/sort/FixedLengthRecordSorter.java b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/sort/FixedLengthRecordSorter.java index f05694b693e6e..a3766e75bae91 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/sort/FixedLengthRecordSorter.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/sort/FixedLengthRecordSorter.java @@ -373,9 +373,7 @@ public T next() { this.currentInSegment++; try { - // This might blow up in our face, but we ignore the readWithNormalization/ - // writeWithNormalization methods for now. - return this.comp.readWithKeyDenormalization(null, this.in); + return this.comp.readWithKeyDenormalization(serializer.createInstance(), this.in); } catch (IOException ioe) { throw new RuntimeException(ioe); diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/sort/MergeMatchIterator.java b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/sort/MergeMatchIterator.java index d57a09e7f4c60..675758aa13e0a 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/sort/MergeMatchIterator.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/sort/MergeMatchIterator.java @@ -36,7 +36,7 @@ import org.apache.flink.runtime.operators.resettable.BlockResettableIterator; import org.apache.flink.runtime.operators.resettable.SpillingResettableIterator; import org.apache.flink.runtime.operators.util.JoinTaskIterator; -import org.apache.flink.runtime.util.KeyGroupedIterator; +import org.apache.flink.runtime.util.ReusingKeyGroupedIterator; import org.apache.flink.util.Collector; import org.apache.flink.util.MutableObjectIterator; @@ -56,9 +56,9 @@ public class MergeMatchIterator implements JoinTaskIterator comp; - private KeyGroupedIterator iterator1; + private ReusingKeyGroupedIterator iterator1; - private KeyGroupedIterator iterator2; + private ReusingKeyGroupedIterator iterator2; private final TypeSerializer serializer1; @@ -104,8 +104,8 @@ public MergeMatchIterator(MutableObjectIterator input1, MutableObjectIterato this.memoryManager = memoryManager; this.ioManager = ioManager; - this.iterator1 = new KeyGroupedIterator(input1, this.serializer1, comparator1.duplicate()); - this.iterator2 = new KeyGroupedIterator(input2, this.serializer2, comparator2.duplicate()); + this.iterator1 = new ReusingKeyGroupedIterator(input1, this.serializer1, comparator1.duplicate()); + this.iterator2 = new ReusingKeyGroupedIterator(input2, this.serializer2, comparator2.duplicate()); final int numPagesForSpiller = numMemoryPages > 20 ? 2 : 1; this.blockIt = new BlockResettableIterator(this.memoryManager, this.serializer2, @@ -190,8 +190,8 @@ public boolean callWithNextKey(final FlatJoinFunction matchFunction, // here, we have a common key! call the match function with the cross product of the // values - final KeyGroupedIterator.ValuesIterator values1 = this.iterator1.getValues(); - final KeyGroupedIterator.ValuesIterator values2 = this.iterator2.getValues(); + final ReusingKeyGroupedIterator.ValuesIterator values1 = this.iterator1.getValues(); + final ReusingKeyGroupedIterator.ValuesIterator values2 = this.iterator2.getValues(); final T1 firstV1 = values1.next(); final T2 firstV2 = values2.next(); diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/sort/SortMergeCoGroupIterator.java b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/sort/NonReusingSortMergeCoGroupIterator.java similarity index 86% rename from flink-runtime/src/main/java/org/apache/flink/runtime/operators/sort/SortMergeCoGroupIterator.java rename to flink-runtime/src/main/java/org/apache/flink/runtime/operators/sort/NonReusingSortMergeCoGroupIterator.java index 89872775d1e28..3d6b7442b132c 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/sort/SortMergeCoGroupIterator.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/sort/NonReusingSortMergeCoGroupIterator.java @@ -7,7 +7,7 @@ * "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, @@ -18,49 +18,51 @@ package org.apache.flink.runtime.operators.sort; -import java.io.IOException; -import java.util.Collections; - import org.apache.flink.api.common.typeutils.TypeComparator; import org.apache.flink.api.common.typeutils.TypePairComparator; import org.apache.flink.api.common.typeutils.TypeSerializer; import org.apache.flink.runtime.operators.util.CoGroupTaskIterator; -import org.apache.flink.runtime.util.KeyGroupedIterator; +import org.apache.flink.runtime.util.ReusingKeyGroupedIterator; import org.apache.flink.util.MutableObjectIterator; +import java.io.IOException; +import java.util.Collections; + + +public class NonReusingSortMergeCoGroupIterator implements CoGroupTaskIterator { -public class SortMergeCoGroupIterator implements CoGroupTaskIterator { - private static enum MatchStatus { NONE_REMAINED, FIRST_REMAINED, SECOND_REMAINED, FIRST_EMPTY, SECOND_EMPTY } - + // -------------------------------------------------------------------------------------------- - + private MatchStatus matchStatus; - + private Iterable firstReturn; - + private Iterable secondReturn; - + private TypePairComparator comp; - - private KeyGroupedIterator iterator1; - private KeyGroupedIterator iterator2; + private ReusingKeyGroupedIterator iterator1; + + private ReusingKeyGroupedIterator iterator2; // -------------------------------------------------------------------------------------------- - - public SortMergeCoGroupIterator(MutableObjectIterator input1, MutableObjectIterator input2, + + public NonReusingSortMergeCoGroupIterator( + MutableObjectIterator input1, + MutableObjectIterator input2, TypeSerializer serializer1, TypeComparator groupingComparator1, TypeSerializer serializer2, TypeComparator groupingComparator2, TypePairComparator pairComparator) - { + { this.comp = pairComparator; - - this.iterator1 = new KeyGroupedIterator(input1, serializer1, groupingComparator1); - this.iterator2 = new KeyGroupedIterator(input2, serializer2, groupingComparator2); + + this.iterator1 = new ReusingKeyGroupedIterator(input1, serializer1, groupingComparator1); + this.iterator2 = new ReusingKeyGroupedIterator(input2, serializer2, groupingComparator2); } @Override @@ -86,7 +88,7 @@ public Iterable getValues2() { public boolean next() throws IOException { boolean firstEmpty = true; boolean secondEmpty = true; - + if (this.matchStatus != MatchStatus.FIRST_EMPTY) { if (this.matchStatus == MatchStatus.FIRST_REMAINED) { // comparator is still set correctly @@ -130,7 +132,7 @@ else if (!firstEmpty && secondEmpty) { else { // both inputs are not empty final int comp = this.comp.compareToReference(this.iterator2.getCurrent()); - + if (0 == comp) { // keys match this.firstReturn = this.iterator1.getValues(); diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/sort/ReusingSortMergeCoGroupIterator.java b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/sort/ReusingSortMergeCoGroupIterator.java new file mode 100644 index 0000000000000..9638bb38b03c5 --- /dev/null +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/sort/ReusingSortMergeCoGroupIterator.java @@ -0,0 +1,158 @@ +/* + * 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.flink.runtime.operators.sort; + +import java.io.IOException; +import java.util.Collections; + +import org.apache.flink.api.common.typeutils.TypeComparator; +import org.apache.flink.api.common.typeutils.TypePairComparator; +import org.apache.flink.api.common.typeutils.TypeSerializer; +import org.apache.flink.runtime.operators.util.CoGroupTaskIterator; +import org.apache.flink.runtime.util.ReusingKeyGroupedIterator; +import org.apache.flink.util.MutableObjectIterator; + + +public class ReusingSortMergeCoGroupIterator implements CoGroupTaskIterator { + + private static enum MatchStatus { + NONE_REMAINED, FIRST_REMAINED, SECOND_REMAINED, FIRST_EMPTY, SECOND_EMPTY + } + + // -------------------------------------------------------------------------------------------- + + private MatchStatus matchStatus; + + private Iterable firstReturn; + + private Iterable secondReturn; + + private TypePairComparator comp; + + private ReusingKeyGroupedIterator iterator1; + + private ReusingKeyGroupedIterator iterator2; + + // -------------------------------------------------------------------------------------------- + + public ReusingSortMergeCoGroupIterator( + MutableObjectIterator input1, + MutableObjectIterator input2, TypeSerializer + serializer1, TypeComparator groupingComparator1, + TypeSerializer serializer2, + TypeComparator groupingComparator2, + TypePairComparator pairComparator) + { + + this.comp = pairComparator; + + this.iterator1 = new ReusingKeyGroupedIterator(input1, serializer1, groupingComparator1); + this.iterator2 = new ReusingKeyGroupedIterator(input2, serializer2, groupingComparator2); + } + + @Override + public void open() {} + + @Override + public void close() {} + + + @Override + public Iterable getValues1() { + return this.firstReturn; + } + + + @Override + public Iterable getValues2() { + return this.secondReturn; + } + + + @Override + public boolean next() throws IOException { + boolean firstEmpty = true; + boolean secondEmpty = true; + + if (this.matchStatus != MatchStatus.FIRST_EMPTY) { + if (this.matchStatus == MatchStatus.FIRST_REMAINED) { + // comparator is still set correctly + firstEmpty = false; + } else { + if (this.iterator1.nextKey()) { + this.comp.setReference(this.iterator1.getCurrent()); + firstEmpty = false; + } + } + } + + if (this.matchStatus != MatchStatus.SECOND_EMPTY) { + if (this.matchStatus == MatchStatus.SECOND_REMAINED) { + secondEmpty = false; + } else { + if (iterator2.nextKey()) { + secondEmpty = false; + } + } + } + + if (firstEmpty && secondEmpty) { + // both inputs are empty + return false; + } + else if (firstEmpty && !secondEmpty) { + // input1 is empty, input2 not + this.firstReturn = Collections.emptySet(); + this.secondReturn = this.iterator2.getValues(); + this.matchStatus = MatchStatus.FIRST_EMPTY; + return true; + } + else if (!firstEmpty && secondEmpty) { + // input1 is not empty, input 2 is empty + this.firstReturn = this.iterator1.getValues(); + this.secondReturn = Collections.emptySet(); + this.matchStatus = MatchStatus.SECOND_EMPTY; + return true; + } + else { + // both inputs are not empty + final int comp = this.comp.compareToReference(this.iterator2.getCurrent()); + + if (0 == comp) { + // keys match + this.firstReturn = this.iterator1.getValues(); + this.secondReturn = this.iterator2.getValues(); + this.matchStatus = MatchStatus.NONE_REMAINED; + } + else if (0 < comp) { + // key1 goes first + this.firstReturn = this.iterator1.getValues(); + this.secondReturn = Collections.emptySet(); + this.matchStatus = MatchStatus.SECOND_REMAINED; + } + else { + // key 2 goes first + this.firstReturn = Collections.emptySet(); + this.secondReturn = this.iterator2.getValues(); + this.matchStatus = MatchStatus.FIRST_REMAINED; + } + return true; + } + } +} diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/util/KeyGroupedIteratorImmutable.java b/flink-runtime/src/main/java/org/apache/flink/runtime/util/NonReusingKeyGroupedIterator.java similarity index 92% rename from flink-runtime/src/main/java/org/apache/flink/runtime/util/KeyGroupedIteratorImmutable.java rename to flink-runtime/src/main/java/org/apache/flink/runtime/util/NonReusingKeyGroupedIterator.java index 80ac2317a2d69..43d9bde097d2f 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/util/KeyGroupedIteratorImmutable.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/util/NonReusingKeyGroupedIterator.java @@ -30,7 +30,7 @@ * The KeyValueIterator returns a key and all values that belong to the key (share the same key). * */ -public final class KeyGroupedIteratorImmutable { +public final class NonReusingKeyGroupedIterator { private final MutableObjectIterator iterator; @@ -54,8 +54,8 @@ public final class KeyGroupedIteratorImmutable { * @param serializer The serializer for the data type iterated over. * @param comparator The comparator for the data type iterated over. */ - public KeyGroupedIteratorImmutable(MutableObjectIterator iterator, - TypeSerializer serializer, TypeComparator comparator) + public NonReusingKeyGroupedIterator(MutableObjectIterator iterator, TypeSerializer + serializer, TypeComparator comparator) { if (iterator == null || serializer == null || comparator == null) { throw new NullPointerException(); @@ -92,9 +92,9 @@ public boolean nextKey() throws IOException { if (this.valuesIterator != null) { // values was not entirely consumed. move to the next key // Required if user code / reduce() method did not read the whole value iterator. - E next = this.serializer.createInstance(); + E next; while (true) { - if ((next = this.iterator.next(next)) != null) { + if ((next = this.iterator.next()) != null) { if (!this.comparator.equalToReference(next)) { // the keys do not match, so we have a new group. store the current key this.comparator.setReference(next); @@ -117,7 +117,7 @@ public boolean nextKey() throws IOException { else { // first element // get the next element - E first = this.iterator.next(this.serializer.createInstance()); + E first = this.iterator.next(); if (first != null) { this.comparator.setReference(first); this.valuesIterator = new ValuesIterator(first); @@ -134,7 +134,7 @@ public boolean nextKey() throws IOException { private E advanceToNext() { try { - E next = this.iterator.next(serializer.createInstance()); + E next = this.iterator.next(); if (next != null) { if (comparator.equalToReference(next)) { // same key @@ -196,7 +196,7 @@ public boolean hasNext() { public E next() { if (this.next != null) { E current = this.next; - this.next = KeyGroupedIteratorImmutable.this.advanceToNext(); + this.next = NonReusingKeyGroupedIterator.this.advanceToNext(); return current; } else { throw new NoSuchElementException(); diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/util/NonReusingMutableToRegularIteratorWrapper.java b/flink-runtime/src/main/java/org/apache/flink/runtime/util/NonReusingMutableToRegularIteratorWrapper.java new file mode 100644 index 0000000000000..0db1670ea2c89 --- /dev/null +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/util/NonReusingMutableToRegularIteratorWrapper.java @@ -0,0 +1,96 @@ +/* + * 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.flink.runtime.util; + +import org.apache.flink.api.common.typeutils.TypeSerializer; +import org.apache.flink.util.MutableObjectIterator; +import org.apache.flink.util.TraversableOnceException; + +import java.io.IOException; +import java.util.Iterator; +import java.util.NoSuchElementException; + +/** + * This class wraps a {@link org.apache.flink.util.MutableObjectIterator} into a regular + * {@link java.util.Iterator}. It will always create new instances and not reuse objects. + */ +public class NonReusingMutableToRegularIteratorWrapper implements Iterator, Iterable { + + private final MutableObjectIterator source; + + private T current; + + private boolean currentIsAvailable; + + private boolean iteratorAvailable = true; + + private TypeSerializer serializer; + + public NonReusingMutableToRegularIteratorWrapper(MutableObjectIterator source, + TypeSerializer serializer) { + this.source = source; + this.current = null; + this.serializer = serializer; + } + + @Override + public boolean hasNext() { + if (currentIsAvailable) { + return true; + } else { + try { + if ((current = source.next()) != null) { + currentIsAvailable = true; + return true; + } else { + return false; + } + } catch (IOException ioex) { + throw new RuntimeException("Error reading next record: " + ioex.getMessage(), ioex); + } + } + } + + @Override + public T next() { + if (currentIsAvailable || hasNext()) { + currentIsAvailable = false; + return current; + } else { + throw new NoSuchElementException(); + } + } + + @Override + public void remove() { + throw new UnsupportedOperationException(); + } + + @Override + public Iterator iterator() { + if (iteratorAvailable) { + iteratorAvailable = false; + return this; + } + else { + throw new TraversableOnceException(); + } + } +} diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/util/KeyGroupedIterator.java b/flink-runtime/src/main/java/org/apache/flink/runtime/util/ReusingKeyGroupedIterator.java similarity index 86% rename from flink-runtime/src/main/java/org/apache/flink/runtime/util/KeyGroupedIterator.java rename to flink-runtime/src/main/java/org/apache/flink/runtime/util/ReusingKeyGroupedIterator.java index e0d1d9929dc8f..4dc9dd38ef487 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/util/KeyGroupedIterator.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/util/ReusingKeyGroupedIterator.java @@ -31,14 +31,16 @@ * The KeyValueIterator returns a key and all values that belong to the key (share the same key). * */ -public final class KeyGroupedIterator { +public final class ReusingKeyGroupedIterator { private final MutableObjectIterator iterator; private final TypeSerializer serializer; private final TypeComparator comparator; - + + private E reuse; + private E current; private E lookahead; @@ -57,8 +59,8 @@ public final class KeyGroupedIterator { * @param serializer The serializer for the data type iterated over. * @param comparator The comparator for the data type iterated over. */ - public KeyGroupedIterator(MutableObjectIterator iterator, - TypeSerializer serializer, TypeComparator comparator) + public ReusingKeyGroupedIterator(MutableObjectIterator iterator, TypeSerializer + serializer, TypeComparator comparator) { if (iterator == null || serializer == null || comparator == null) { throw new NullPointerException(); @@ -67,6 +69,7 @@ public KeyGroupedIterator(MutableObjectIterator iterator, this.iterator = iterator; this.serializer = serializer; this.comparator = comparator; + this.reuse = this.serializer.createInstance(); } /** @@ -83,7 +86,7 @@ public boolean nextKey() throws IOException this.valuesIterator = null; return false; } - this.current = this.serializer.createInstance(); + this.current = this.reuse; if ((this.current = this.iterator.next(this.current)) != null) { this.comparator.setReference(this.current); this.lookAheadHasNext = false; @@ -155,19 +158,19 @@ public ValuesIterator getValues() { public final class ValuesIterator implements Iterator, Iterable { - private final TypeSerializer serializer = KeyGroupedIterator.this.serializer; - private final TypeComparator comparator = KeyGroupedIterator.this.comparator; + private final TypeSerializer serializer = ReusingKeyGroupedIterator.this.serializer; + private final TypeComparator comparator = ReusingKeyGroupedIterator.this.comparator; private E staging = this.serializer.createInstance(); private boolean currentIsUnconsumed = false; private boolean iteratorAvailable = true; - + private ValuesIterator() {} @Override public boolean hasNext() { - if (KeyGroupedIterator.this.current == null || KeyGroupedIterator.this.lookAheadHasNext) { + if (ReusingKeyGroupedIterator.this.current == null || ReusingKeyGroupedIterator.this.lookAheadHasNext) { return false; } if (this.currentIsUnconsumed) { @@ -177,27 +180,27 @@ public boolean hasNext() { try { // read the next value into the staging record to make sure we keep the // current as it is in case the key changed - E stagingStaging = KeyGroupedIterator.this.iterator.next(this.staging); + E stagingStaging = ReusingKeyGroupedIterator.this.iterator.next(this.staging); if (stagingStaging != null) { this.staging = stagingStaging; if (this.comparator.equalToReference(this.staging)) { // same key, next value is in staging, so exchange staging with current final E tmp = this.staging; - this.staging = KeyGroupedIterator.this.current; - KeyGroupedIterator.this.current = tmp; + this.staging = ReusingKeyGroupedIterator.this.current; + ReusingKeyGroupedIterator.this.current = tmp; this.currentIsUnconsumed = true; return true; } else { // moved to the next key, no more values here - KeyGroupedIterator.this.lookAheadHasNext = true; - KeyGroupedIterator.this.lookahead = this.staging; - this.staging = KeyGroupedIterator.this.current; + ReusingKeyGroupedIterator.this.lookAheadHasNext = true; + ReusingKeyGroupedIterator.this.lookahead = this.staging; + this.staging = ReusingKeyGroupedIterator.this.current; return false; } } else { // backing iterator is consumed - KeyGroupedIterator.this.done = true; + ReusingKeyGroupedIterator.this.done = true; return false; } } @@ -214,7 +217,7 @@ public boolean hasNext() { public E next() { if (this.currentIsUnconsumed || hasNext()) { this.currentIsUnconsumed = false; - return KeyGroupedIterator.this.current; + return ReusingKeyGroupedIterator.this.current; } else { throw new NoSuchElementException(); } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/util/MutableToRegularIteratorWrapper.java b/flink-runtime/src/main/java/org/apache/flink/runtime/util/ReusingMutableToRegularIteratorWrapper.java similarity index 92% rename from flink-runtime/src/main/java/org/apache/flink/runtime/util/MutableToRegularIteratorWrapper.java rename to flink-runtime/src/main/java/org/apache/flink/runtime/util/ReusingMutableToRegularIteratorWrapper.java index 62b1be2de3ec6..b9e3e74d95149 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/util/MutableToRegularIteratorWrapper.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/util/ReusingMutableToRegularIteratorWrapper.java @@ -33,7 +33,7 @@ * whenever hasNext() returns (possibly with false), the previous obtained record is * still valid and cannot have been overwritten internally. */ -public class MutableToRegularIteratorWrapper implements Iterator, Iterable { +public class ReusingMutableToRegularIteratorWrapper implements Iterator, Iterable { private final MutableObjectIterator source; @@ -43,7 +43,8 @@ public class MutableToRegularIteratorWrapper implements Iterator, Iterable private boolean iteratorAvailable = true; - public MutableToRegularIteratorWrapper(MutableObjectIterator source, TypeSerializer serializer) { + public ReusingMutableToRegularIteratorWrapper(MutableObjectIterator source, + TypeSerializer serializer) { this.source = source; this.current = serializer.createInstance(); this.next = serializer.createInstance(); diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/CachedMatchTaskTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/CachedMatchTaskTest.java index 262a4e90800c9..a57287a7fe747 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/CachedMatchTaskTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/CachedMatchTaskTest.java @@ -22,6 +22,7 @@ import java.util.List; import java.util.concurrent.atomic.AtomicBoolean; +import org.apache.flink.api.common.ExecutionConfig; import org.apache.flink.api.common.functions.FlatJoinFunction; import org.apache.flink.api.common.functions.RichFlatJoinFunction; import org.apache.flink.api.common.typeutils.record.RecordComparator; @@ -56,8 +57,8 @@ public class CachedMatchTaskTest extends DriverTestBase outList = new ArrayList(); - public CachedMatchTaskTest() { - super(HASH_MEM, 2, SORT_MEM); + public CachedMatchTaskTest(ExecutionConfig config) { + super(config, HASH_MEM, 2, SORT_MEM); } @Test diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/CoGroupTaskExternalITCase.java b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/CoGroupTaskExternalITCase.java index 8e7d9d38f12b5..7f96954961e62 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/CoGroupTaskExternalITCase.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/CoGroupTaskExternalITCase.java @@ -18,6 +18,7 @@ package org.apache.flink.runtime.operators; +import org.apache.flink.api.common.ExecutionConfig; import org.junit.Assert; import org.apache.flink.api.common.functions.CoGroupFunction; import org.apache.flink.api.common.functions.RichCoGroupFunction; @@ -45,8 +46,8 @@ public class CoGroupTaskExternalITCase extends DriverTestBase>[])new Class[]{ IntValue.class }); - public CombineTaskExternalITCase() { - super(COMBINE_MEM, 0); + public CombineTaskExternalITCase(ExecutionConfig config) { + super(config, COMBINE_MEM, 0); combine_frac = (double)COMBINE_MEM/this.getMemoryManager().getMemorySize(); } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/CombineTaskTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/CombineTaskTest.java index 1f189175a90ab..515ce76e984cc 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/CombineTaskTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/CombineTaskTest.java @@ -21,6 +21,7 @@ import java.util.ArrayList; import java.util.concurrent.atomic.AtomicBoolean; +import org.apache.flink.api.common.ExecutionConfig; import org.junit.Assert; import org.apache.flink.api.common.functions.RichGroupReduceFunction; import org.apache.flink.api.common.typeutils.record.RecordComparator; @@ -49,8 +50,8 @@ public class CombineTaskTest extends DriverTestBase>[])new Class[]{ IntValue.class }); - public CombineTaskTest() { - super(COMBINE_MEM, 0); + public CombineTaskTest(ExecutionConfig config) { + super(config, COMBINE_MEM, 0); combine_frac = (double)COMBINE_MEM/this.getMemoryManager().getMemorySize(); } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/CrossTaskExternalITCase.java b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/CrossTaskExternalITCase.java index 1924489d642ea..b678ed179945b 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/CrossTaskExternalITCase.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/CrossTaskExternalITCase.java @@ -19,6 +19,7 @@ package org.apache.flink.runtime.operators; +import org.apache.flink.api.common.ExecutionConfig; import org.junit.Assert; import org.apache.flink.api.common.functions.CrossFunction; @@ -37,8 +38,8 @@ public class CrossTaskExternalITCase extends DriverTestBase outList = new ArrayList(); - public MatchTaskTest() { - super(HASH_MEM, NUM_SORTER, SORT_MEM); + public MatchTaskTest(ExecutionConfig config) { + super(config, HASH_MEM, NUM_SORTER, SORT_MEM); bnljn_frac = (double)BNLJN_MEM/this.getMemoryManager().getMemorySize(); hash_frac = (double)HASH_MEM/this.getMemoryManager().getMemorySize(); } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/ReduceTaskExternalITCase.java b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/ReduceTaskExternalITCase.java index eb26f5b06cf5c..d83e92ec07598 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/ReduceTaskExternalITCase.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/ReduceTaskExternalITCase.java @@ -21,6 +21,7 @@ import java.util.ArrayList; import java.util.List; +import org.apache.flink.api.common.ExecutionConfig; import org.junit.Assert; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -48,8 +49,8 @@ public class ReduceTaskExternalITCase extends DriverTestBase outList = new ArrayList(); - public ReduceTaskExternalITCase() { - super(0, 1, 3*1024*1024); + public ReduceTaskExternalITCase(ExecutionConfig config) { + super(config, 0, 1, 3*1024*1024); } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/ReduceTaskTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/ReduceTaskTest.java index c5a67627fb30a..964f646da04b2 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/ReduceTaskTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/ReduceTaskTest.java @@ -22,6 +22,7 @@ import java.util.List; import java.util.concurrent.atomic.AtomicBoolean; +import org.apache.flink.api.common.ExecutionConfig; import org.junit.Assert; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -52,8 +53,8 @@ public class ReduceTaskTest extends DriverTestBase outList = new ArrayList(); - public ReduceTaskTest() { - super(0, 1, 3*1024*1024); + public ReduceTaskTest(ExecutionConfig config) { + super(config, 0, 1, 3*1024*1024); } @Test diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/drivers/TestTaskContext.java b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/drivers/TestTaskContext.java index e5ece3fad8082..b614709734a7d 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/drivers/TestTaskContext.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/drivers/TestTaskContext.java @@ -19,6 +19,7 @@ package org.apache.flink.runtime.operators.drivers; +import org.apache.flink.api.common.ExecutionConfig; import org.apache.flink.api.common.typeutils.TypeComparator; import org.apache.flink.api.common.typeutils.TypeSerializer; import org.apache.flink.api.common.typeutils.TypeSerializerFactory; @@ -59,6 +60,8 @@ public class TestTaskContext implements PactTaskContext { private MemoryManager memoryManager; + private ExecutionConfig executionConfig = new ExecutionConfig(); + // -------------------------------------------------------------------------------------------- // Constructors // -------------------------------------------------------------------------------------------- @@ -132,6 +135,11 @@ public TaskConfig getTaskConfig() { return this.config; } + @Override + public ExecutionConfig getExecutionConfig() { + return executionConfig; + } + @Override public ClassLoader getUserCodeClassLoader() { return getClass().getClassLoader(); diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/hash/NonReusingHashMatchIteratorITCase.java b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/hash/NonReusingHashMatchIteratorITCase.java new file mode 100644 index 0000000000000..f4d225184ab59 --- /dev/null +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/hash/NonReusingHashMatchIteratorITCase.java @@ -0,0 +1,778 @@ +/* + * 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.flink.runtime.operators.hash; + +import org.apache.flink.api.common.functions.AbstractRichFunction; +import org.apache.flink.api.common.functions.FlatJoinFunction; +import org.apache.flink.api.common.typeutils.TypeComparator; +import org.apache.flink.api.common.typeutils.TypePairComparator; +import org.apache.flink.api.common.typeutils.TypeSerializer; +import org.apache.flink.api.common.typeutils.record.RecordComparator; +import org.apache.flink.api.common.typeutils.record.RecordPairComparator; +import org.apache.flink.api.common.typeutils.record.RecordSerializer; +import org.apache.flink.api.java.record.functions.JoinFunction; +import org.apache.flink.runtime.io.disk.iomanager.IOManager; +import org.apache.flink.runtime.io.disk.iomanager.IOManagerAsync; +import org.apache.flink.runtime.jobgraph.tasks.AbstractInvokable; +import org.apache.flink.runtime.memorymanager.DefaultMemoryManager; +import org.apache.flink.runtime.memorymanager.MemoryManager; +import org.apache.flink.runtime.operators.testutils.DiscardingOutputCollector; +import org.apache.flink.runtime.operators.testutils.DummyInvokable; +import org.apache.flink.runtime.operators.testutils.TestData; +import org.apache.flink.runtime.operators.testutils.TestData.Generator; +import org.apache.flink.runtime.operators.testutils.TestData.Generator.KeyMode; +import org.apache.flink.runtime.operators.testutils.TestData.Generator.ValueMode; +import org.apache.flink.runtime.operators.testutils.UniformIntPairGenerator; +import org.apache.flink.runtime.operators.testutils.UnionIterator; +import org.apache.flink.runtime.operators.testutils.types.IntPair; +import org.apache.flink.runtime.operators.testutils.types.IntPairComparator; +import org.apache.flink.runtime.operators.testutils.types.IntPairSerializer; +import org.apache.flink.types.IntValue; +import org.apache.flink.types.NullKeyFieldException; +import org.apache.flink.types.Record; +import org.apache.flink.types.Value; +import org.apache.flink.util.Collector; +import org.apache.flink.util.MutableObjectIterator; +import org.junit.After; +import org.junit.Assert; +import org.junit.Before; +import org.junit.Test; + +import java.util.ArrayList; +import java.util.Collection; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.Map.Entry; + +@SuppressWarnings({"serial", "deprecation"}) +public class NonReusingHashMatchIteratorITCase { + + private static final int MEMORY_SIZE = 16000000; // total memory + + private static final int INPUT_1_SIZE = 20000; + private static final int INPUT_2_SIZE = 1000; + + private static final long SEED1 = 561349061987311L; + private static final long SEED2 = 231434613412342L; + + private final AbstractInvokable parentTask = new DummyInvokable(); + + private IOManager ioManager; + private MemoryManager memoryManager; + + private TypeSerializer recordSerializer; + private TypeComparator record1Comparator; + private TypeComparator record2Comparator; + private TypePairComparator recordPairComparator; + + private TypeSerializer pairSerializer; + private TypeComparator pairComparator; + private TypePairComparator pairRecordPairComparator; + private TypePairComparator recordPairPairComparator; + + + @SuppressWarnings("unchecked") + @Before + public void beforeTest() { + this.recordSerializer = RecordSerializer.get(); + + this.record1Comparator = new RecordComparator(new int[] {0}, new Class[] {TestData.Key.class}); + this.record2Comparator = new RecordComparator(new int[] {0}, new Class[] {TestData.Key.class}); + + this.recordPairComparator = new RecordPairComparator(new int[] {0}, new int[] {0}, new Class[] {TestData.Key.class}); + + this.pairSerializer = new IntPairSerializer(); + this.pairComparator = new IntPairComparator(); + this.pairRecordPairComparator = new IntPairRecordPairComparator(); + this.recordPairPairComparator = new RecordIntPairPairComparator(); + + this.memoryManager = new DefaultMemoryManager(MEMORY_SIZE, 1); + this.ioManager = new IOManagerAsync(); + } + + @After + public void afterTest() { + if (this.ioManager != null) { + this.ioManager.shutdown(); + if (!this.ioManager.isProperlyShutDown()) { + Assert.fail("I/O manager failed to properly shut down."); + } + this.ioManager = null; + } + + if (this.memoryManager != null) { + Assert.assertTrue("Memory Leak: Not all memory has been returned to the memory manager.", + this.memoryManager.verifyEmpty()); + this.memoryManager.shutdown(); + this.memoryManager = null; + } + } + + + @Test + public void testBuildFirst() { + try { + Generator generator1 = new Generator(SEED1, 500, 4096, KeyMode.RANDOM, ValueMode.RANDOM_LENGTH); + Generator generator2 = new Generator(SEED2, 500, 2048, KeyMode.RANDOM, ValueMode.RANDOM_LENGTH); + + final TestData.GeneratorIterator input1 = new TestData.GeneratorIterator(generator1, INPUT_1_SIZE); + final TestData.GeneratorIterator input2 = new TestData.GeneratorIterator(generator2, INPUT_2_SIZE); + + // collect expected data + final Map> expectedMatchesMap = matchRecordValues( + collectRecordData(input1), + collectRecordData(input2)); + + final JoinFunction matcher = new RecordMatchRemovingJoin(expectedMatchesMap); + final Collector collector = new DiscardingOutputCollector(); + + // reset the generators + generator1.reset(); + generator2.reset(); + input1.reset(); + input2.reset(); + + // compare with iterator values + NonReusingBuildFirstHashMatchIterator iterator = + new NonReusingBuildFirstHashMatchIterator( + input1, input2, this.recordSerializer, this.record1Comparator, + this.recordSerializer, this.record2Comparator, this.recordPairComparator, + this.memoryManager, ioManager, this.parentTask, 1.0); + + iterator.open(); + + while (iterator.callWithNextKey(matcher, collector)); + + iterator.close(); + + // assert that each expected match was seen + for (Entry> entry : expectedMatchesMap.entrySet()) { + if (!entry.getValue().isEmpty()) { + Assert.fail("Collection for key " + entry.getKey() + " is not empty"); + } + } + } + catch (Exception e) { + e.printStackTrace(); + Assert.fail("An exception occurred during the test: " + e.getMessage()); + } + } + + @Test + public void testBuildFirstWithHighNumberOfCommonKeys() + { + // the size of the left and right inputs + final int INPUT_1_SIZE = 200; + final int INPUT_2_SIZE = 100; + + final int INPUT_1_DUPLICATES = 10; + final int INPUT_2_DUPLICATES = 2000; + final int DUPLICATE_KEY = 13; + + try { + Generator generator1 = new Generator(SEED1, 500, 4096, KeyMode.RANDOM, ValueMode.RANDOM_LENGTH); + Generator generator2 = new Generator(SEED2, 500, 2048, KeyMode.RANDOM, ValueMode.RANDOM_LENGTH); + + final TestData.GeneratorIterator gen1Iter = new TestData.GeneratorIterator(generator1, INPUT_1_SIZE); + final TestData.GeneratorIterator gen2Iter = new TestData.GeneratorIterator(generator2, INPUT_2_SIZE); + + final TestData.ConstantValueIterator const1Iter = new TestData.ConstantValueIterator(DUPLICATE_KEY, "LEFT String for Duplicate Keys", INPUT_1_DUPLICATES); + final TestData.ConstantValueIterator const2Iter = new TestData.ConstantValueIterator(DUPLICATE_KEY, "RIGHT String for Duplicate Keys", INPUT_2_DUPLICATES); + + final List> inList1 = new ArrayList>(); + inList1.add(gen1Iter); + inList1.add(const1Iter); + + final List> inList2 = new ArrayList>(); + inList2.add(gen2Iter); + inList2.add(const2Iter); + + MutableObjectIterator input1 = new UnionIterator(inList1); + MutableObjectIterator input2 = new UnionIterator(inList2); + + + // collect expected data + final Map> expectedMatchesMap = matchRecordValues( + collectRecordData(input1), + collectRecordData(input2)); + + // re-create the whole thing for actual processing + + // reset the generators and iterators + generator1.reset(); + generator2.reset(); + const1Iter.reset(); + const2Iter.reset(); + gen1Iter.reset(); + gen2Iter.reset(); + + inList1.clear(); + inList1.add(gen1Iter); + inList1.add(const1Iter); + + inList2.clear(); + inList2.add(gen2Iter); + inList2.add(const2Iter); + + input1 = new UnionIterator(inList1); + input2 = new UnionIterator(inList2); + + final JoinFunction matcher = new RecordMatchRemovingJoin(expectedMatchesMap); + final Collector collector = new DiscardingOutputCollector(); + + NonReusingBuildFirstHashMatchIterator iterator = + new NonReusingBuildFirstHashMatchIterator( + input1, input2, this.recordSerializer, this.record1Comparator, + this.recordSerializer, this.record2Comparator, this.recordPairComparator, + this.memoryManager, ioManager, this.parentTask, 1.0); + + iterator.open(); + + while (iterator.callWithNextKey(matcher, collector)); + + iterator.close(); + + // assert that each expected match was seen + for (Entry> entry : expectedMatchesMap.entrySet()) { + if (!entry.getValue().isEmpty()) { + Assert.fail("Collection for key " + entry.getKey() + " is not empty"); + } + } + } + catch (Exception e) { + e.printStackTrace(); + Assert.fail("An exception occurred during the test: " + e.getMessage()); + } + } + + @Test + public void testBuildSecond() { + try { + Generator generator1 = new Generator(SEED1, 500, 4096, KeyMode.RANDOM, ValueMode.RANDOM_LENGTH); + Generator generator2 = new Generator(SEED2, 500, 2048, KeyMode.RANDOM, ValueMode.RANDOM_LENGTH); + + final TestData.GeneratorIterator input1 = new TestData.GeneratorIterator(generator1, INPUT_1_SIZE); + final TestData.GeneratorIterator input2 = new TestData.GeneratorIterator(generator2, INPUT_2_SIZE); + + // collect expected data + final Map> expectedMatchesMap = matchRecordValues( + collectRecordData(input1), + collectRecordData(input2)); + + final JoinFunction matcher = new RecordMatchRemovingJoin(expectedMatchesMap); + final Collector collector = new DiscardingOutputCollector(); + + // reset the generators + generator1.reset(); + generator2.reset(); + input1.reset(); + input2.reset(); + + // compare with iterator values + NonReusingBuildSecondHashMatchIterator iterator = + new NonReusingBuildSecondHashMatchIterator( + input1, input2, this.recordSerializer, this.record1Comparator, + this.recordSerializer, this.record2Comparator, this.recordPairComparator, + this.memoryManager, ioManager, this.parentTask, 1.0); + + iterator.open(); + + while (iterator.callWithNextKey(matcher, collector)); + + iterator.close(); + + // assert that each expected match was seen + for (Entry> entry : expectedMatchesMap.entrySet()) { + if (!entry.getValue().isEmpty()) { + Assert.fail("Collection for key " + entry.getKey() + " is not empty"); + } + } + } + catch (Exception e) { + e.printStackTrace(); + Assert.fail("An exception occurred during the test: " + e.getMessage()); + } + } + + @Test + public void testBuildSecondWithHighNumberOfCommonKeys() + { + // the size of the left and right inputs + final int INPUT_1_SIZE = 200; + final int INPUT_2_SIZE = 100; + + final int INPUT_1_DUPLICATES = 10; + final int INPUT_2_DUPLICATES = 2000; + final int DUPLICATE_KEY = 13; + + try { + Generator generator1 = new Generator(SEED1, 500, 4096, KeyMode.RANDOM, ValueMode.RANDOM_LENGTH); + Generator generator2 = new Generator(SEED2, 500, 2048, KeyMode.RANDOM, ValueMode.RANDOM_LENGTH); + + final TestData.GeneratorIterator gen1Iter = new TestData.GeneratorIterator(generator1, INPUT_1_SIZE); + final TestData.GeneratorIterator gen2Iter = new TestData.GeneratorIterator(generator2, INPUT_2_SIZE); + + final TestData.ConstantValueIterator const1Iter = new TestData.ConstantValueIterator(DUPLICATE_KEY, "LEFT String for Duplicate Keys", INPUT_1_DUPLICATES); + final TestData.ConstantValueIterator const2Iter = new TestData.ConstantValueIterator(DUPLICATE_KEY, "RIGHT String for Duplicate Keys", INPUT_2_DUPLICATES); + + final List> inList1 = new ArrayList>(); + inList1.add(gen1Iter); + inList1.add(const1Iter); + + final List> inList2 = new ArrayList>(); + inList2.add(gen2Iter); + inList2.add(const2Iter); + + MutableObjectIterator input1 = new UnionIterator(inList1); + MutableObjectIterator input2 = new UnionIterator(inList2); + + + // collect expected data + final Map> expectedMatchesMap = matchRecordValues( + collectRecordData(input1), + collectRecordData(input2)); + + // re-create the whole thing for actual processing + + // reset the generators and iterators + generator1.reset(); + generator2.reset(); + const1Iter.reset(); + const2Iter.reset(); + gen1Iter.reset(); + gen2Iter.reset(); + + inList1.clear(); + inList1.add(gen1Iter); + inList1.add(const1Iter); + + inList2.clear(); + inList2.add(gen2Iter); + inList2.add(const2Iter); + + input1 = new UnionIterator(inList1); + input2 = new UnionIterator(inList2); + + final JoinFunction matcher = new RecordMatchRemovingJoin(expectedMatchesMap); + final Collector collector = new DiscardingOutputCollector(); + + NonReusingBuildSecondHashMatchIterator iterator = + new NonReusingBuildSecondHashMatchIterator( + input1, input2, this.recordSerializer, this.record1Comparator, + this.recordSerializer, this.record2Comparator, this.recordPairComparator, + this.memoryManager, ioManager, this.parentTask, 1.0); + + iterator.open(); + + while (iterator.callWithNextKey(matcher, collector)); + + iterator.close(); + + // assert that each expected match was seen + for (Entry> entry : expectedMatchesMap.entrySet()) { + if (!entry.getValue().isEmpty()) { + Assert.fail("Collection for key " + entry.getKey() + " is not empty"); + } + } + } + catch (Exception e) { + e.printStackTrace(); + Assert.fail("An exception occurred during the test: " + e.getMessage()); + } + } + + @Test + public void testBuildFirstWithMixedDataTypes() { + try { + MutableObjectIterator input1 = new UniformIntPairGenerator(500, 40, false); + + final Generator generator2 = new Generator(SEED2, 500, 2048, KeyMode.RANDOM, ValueMode.RANDOM_LENGTH); + final TestData.GeneratorIterator input2 = new TestData.GeneratorIterator(generator2, INPUT_2_SIZE); + + // collect expected data + final Map> expectedMatchesMap = matchRecordIntPairValues( + collectIntPairData(input1), + collectRecordData(input2)); + + final FlatJoinFunction matcher = new RecordIntPairMatchRemovingMatcher(expectedMatchesMap); + final Collector collector = new DiscardingOutputCollector(); + + // reset the generators + input1 = new UniformIntPairGenerator(500, 40, false); + generator2.reset(); + input2.reset(); + + // compare with iterator values + NonReusingBuildSecondHashMatchIterator iterator = + new NonReusingBuildSecondHashMatchIterator( + input1, input2, this.pairSerializer, this.pairComparator, + this.recordSerializer, this.record2Comparator, this.pairRecordPairComparator, + this.memoryManager, this.ioManager, this.parentTask, 1.0); + + iterator.open(); + + while (iterator.callWithNextKey(matcher, collector)); + + iterator.close(); + + // assert that each expected match was seen + for (Entry> entry : expectedMatchesMap.entrySet()) { + if (!entry.getValue().isEmpty()) { + Assert.fail("Collection for key " + entry.getKey() + " is not empty"); + } + } + } + catch (Exception e) { + e.printStackTrace(); + Assert.fail("An exception occurred during the test: " + e.getMessage()); + } + } + + @Test + public void testBuildSecondWithMixedDataTypes() { + try { + MutableObjectIterator input1 = new UniformIntPairGenerator(500, 40, false); + + final Generator generator2 = new Generator(SEED2, 500, 2048, KeyMode.RANDOM, ValueMode.RANDOM_LENGTH); + final TestData.GeneratorIterator input2 = new TestData.GeneratorIterator(generator2, INPUT_2_SIZE); + + // collect expected data + final Map> expectedMatchesMap = matchRecordIntPairValues( + collectIntPairData(input1), + collectRecordData(input2)); + + final FlatJoinFunction matcher = new RecordIntPairMatchRemovingMatcher(expectedMatchesMap); + final Collector collector = new DiscardingOutputCollector(); + + // reset the generators + input1 = new UniformIntPairGenerator(500, 40, false); + generator2.reset(); + input2.reset(); + + // compare with iterator values + NonReusingBuildFirstHashMatchIterator iterator = + new NonReusingBuildFirstHashMatchIterator( + input1, input2, this.pairSerializer, this.pairComparator, + this.recordSerializer, this.record2Comparator, this.recordPairPairComparator, + this.memoryManager, this.ioManager, this.parentTask, 1.0); + + iterator.open(); + + while (iterator.callWithNextKey(matcher, collector)); + + iterator.close(); + + // assert that each expected match was seen + for (Entry> entry : expectedMatchesMap.entrySet()) { + if (!entry.getValue().isEmpty()) { + Assert.fail("Collection for key " + entry.getKey() + " is not empty"); + } + } + } + catch (Exception e) { + e.printStackTrace(); + Assert.fail("An exception occurred during the test: " + e.getMessage()); + } + } + + // -------------------------------------------------------------------------------------------- + // Utilities + // -------------------------------------------------------------------------------------------- + + + + static Map> matchRecordValues( + Map> leftMap, + Map> rightMap) + { + Map> map = new HashMap>(); + + for (TestData.Key key : leftMap.keySet()) { + Collection leftValues = leftMap.get(key); + Collection rightValues = rightMap.get(key); + + if (rightValues == null) { + continue; + } + + if (!map.containsKey(key)) { + map.put(key, new ArrayList()); + } + + Collection matchedValues = map.get(key); + + for (TestData.Value leftValue : leftValues) { + for (TestData.Value rightValue : rightValues) { + matchedValues.add(new RecordMatch(leftValue, rightValue)); + } + } + } + + return map; + } + + static Map> matchRecordIntPairValues( + Map> leftMap, + Map> rightMap) + { + final Map> map = new HashMap>(); + + for (Integer i : leftMap.keySet()) { + + final TestData.Key key = new TestData.Key(i.intValue()); + + final Collection leftValues = leftMap.get(i); + final Collection rightValues = rightMap.get(key); + + if (rightValues == null) { + continue; + } + + if (!map.containsKey(key)) { + map.put(key, new ArrayList()); + } + + final Collection matchedValues = map.get(key); + + for (Integer v : leftValues) { + for (TestData.Value val : rightValues) { + matchedValues.add(new RecordIntPairMatch(v, val)); + } + } + } + + return map; + } + + + static Map> collectRecordData(MutableObjectIterator iter) + throws Exception + { + Map> map = new HashMap>(); + Record pair = new Record(); + + while ((pair = iter.next(pair)) != null) { + + TestData.Key key = pair.getField(0, TestData.Key.class); + if (!map.containsKey(key)) { + map.put(new TestData.Key(key.getKey()), new ArrayList()); + } + + Collection values = map.get(key); + values.add(new TestData.Value(pair.getField(1, TestData.Value.class).getValue())); + } + + return map; + } + + static Map> collectIntPairData(MutableObjectIterator iter) + throws Exception + { + Map> map = new HashMap>(); + IntPair pair = new IntPair(); + + while ((pair = iter.next(pair)) != null) { + + final int key = pair.getKey(); + final int value = pair.getValue(); + if (!map.containsKey(key)) { + map.put(key, new ArrayList()); + } + + Collection values = map.get(key); + values.add(value); + } + + return map; + } + + /** + * Private class used for storage of the expected matches in a hash-map. + */ + static class RecordMatch { + + private final Value left; + private final Value right; + + public RecordMatch(Value left, Value right) { + this.left = left; + this.right = right; + } + + @Override + public boolean equals(Object obj) { + RecordMatch o = (RecordMatch) obj; + return this.left.equals(o.left) && this.right.equals(o.right); + } + + @Override + public int hashCode() { + return this.left.hashCode() ^ this.right.hashCode(); + } + + @Override + public String toString() { + return left + ", " + right; + } + } + + /** + * Private class used for storage of the expected matches in a hash-map. + */ + static class RecordIntPairMatch + { + private final int left; + private final Value right; + + public RecordIntPairMatch(int left, Value right) { + this.left = left; + this.right = right; + } + + @Override + public boolean equals(Object obj) { + RecordIntPairMatch o = (RecordIntPairMatch) obj; + return this.left == o.left && this.right.equals(o.right); + } + + @Override + public int hashCode() { + return this.left ^ this.right.hashCode(); + } + + @Override + public String toString() { + return left + ", " + right; + } + } + + static final class RecordMatchRemovingJoin extends JoinFunction + { + private final Map> toRemoveFrom; + + protected RecordMatchRemovingJoin(Map> map) { + this.toRemoveFrom = map; + } + + @Override + public void join(Record rec1, Record rec2, Collector out) throws Exception + { + TestData.Key key = rec1.getField(0, TestData.Key.class); + TestData.Value value1 = rec1.getField(1, TestData.Value.class); + TestData.Value value2 = rec2.getField(1, TestData.Value.class); + //System.err.println("rec1 key = "+key+" rec2 key= "+rec2.getField(0, TestData.Key.class)); + Collection matches = this.toRemoveFrom.get(key); + if (matches == null) { + Assert.fail("Match " + key + " - " + value1 + ":" + value2 + " is unexpected."); + } + + Assert.assertTrue("Produced match was not contained: " + key + " - " + value1 + ":" + value2, + matches.remove(new RecordMatch(value1, value2))); + + if (matches.isEmpty()) { + this.toRemoveFrom.remove(key); + } + } + } + + static final class RecordIntPairMatchRemovingMatcher extends AbstractRichFunction implements FlatJoinFunction + { + private final Map> toRemoveFrom; + + protected RecordIntPairMatchRemovingMatcher(Map> map) { + this.toRemoveFrom = map; + } + + @Override + public void join(IntPair rec1, Record rec2, Collector out) throws Exception + { + final int k = rec1.getKey(); + final int v = rec1.getValue(); + + final TestData.Key key = rec2.getField(0, TestData.Key.class); + final TestData.Value value = rec2.getField(1, TestData.Value.class); + + Assert.assertTrue("Key does not match for matching IntPair Record combination.", k == key.getKey()); + + Collection matches = this.toRemoveFrom.get(key); + if (matches == null) { + Assert.fail("Match " + key + " - " + v + ":" + value + " is unexpected."); + } + + Assert.assertTrue("Produced match was not contained: " + key + " - " + v + ":" + value, + matches.remove(new RecordIntPairMatch(v, value))); + + if (matches.isEmpty()) { + this.toRemoveFrom.remove(key); + } + } + } + + static final class IntPairRecordPairComparator extends TypePairComparator + { + private int reference; + + @Override + public void setReference(IntPair reference) { + this.reference = reference.getKey(); + } + + @Override + public boolean equalToReference(Record candidate) { + try { + final IntValue i = candidate.getField(0, IntValue.class); + return i.getValue() == this.reference; + } catch (NullPointerException npex) { + throw new NullKeyFieldException(); + } + } + + @Override + public int compareToReference(Record candidate) { + try { + final IntValue i = candidate.getField(0, IntValue.class); + return i.getValue() - this.reference; + } catch (NullPointerException npex) { + throw new NullKeyFieldException(); + } + } + } + + static final class RecordIntPairPairComparator extends TypePairComparator + { + private int reference; + + @Override + public void setReference(Record reference) { + this.reference = reference.getField(0, IntValue.class).getValue(); + } + + @Override + public boolean equalToReference(IntPair candidate) { + return this.reference == candidate.getKey(); + } + + @Override + public int compareToReference(IntPair candidate) { + return candidate.getKey() - this.reference; + } + } +} diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/hash/NonReusingReOpenableHashTableITCase.java b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/hash/NonReusingReOpenableHashTableITCase.java new file mode 100644 index 0000000000000..5012d1e6f0737 --- /dev/null +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/hash/NonReusingReOpenableHashTableITCase.java @@ -0,0 +1,533 @@ +/* + * 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.flink.runtime.operators.hash; + +import org.apache.flink.api.common.typeutils.TypeComparator; +import org.apache.flink.api.common.typeutils.TypePairComparator; +import org.apache.flink.api.common.typeutils.TypeSerializer; +import org.apache.flink.api.common.typeutils.record.RecordComparator; +import org.apache.flink.api.common.typeutils.record.RecordPairComparator; +import org.apache.flink.api.common.typeutils.record.RecordSerializer; +import org.apache.flink.api.java.record.functions.JoinFunction; +import org.apache.flink.core.memory.MemorySegment; +import org.apache.flink.runtime.io.disk.iomanager.IOManager; +import org.apache.flink.runtime.io.disk.iomanager.IOManagerAsync; +import org.apache.flink.runtime.jobgraph.tasks.AbstractInvokable; +import org.apache.flink.runtime.memorymanager.DefaultMemoryManager; +import org.apache.flink.runtime.memorymanager.MemoryAllocationException; +import org.apache.flink.runtime.memorymanager.MemoryManager; +import org.apache.flink.runtime.operators.hash.HashTableITCase.ConstantsKeyValuePairsIterator; +import org.apache.flink.runtime.operators.hash.MutableHashTable.HashBucketIterator; +import org.apache.flink.runtime.operators.hash.NonReusingHashMatchIteratorITCase.RecordMatch; +import org.apache.flink.runtime.operators.hash.NonReusingHashMatchIteratorITCase + .RecordMatchRemovingJoin; +import org.apache.flink.runtime.operators.testutils.DiscardingOutputCollector; +import org.apache.flink.runtime.operators.testutils.DummyInvokable; +import org.apache.flink.runtime.operators.testutils.TestData; +import org.apache.flink.runtime.operators.testutils.TestData.Generator; +import org.apache.flink.runtime.operators.testutils.TestData.Generator.KeyMode; +import org.apache.flink.runtime.operators.testutils.TestData.Generator.ValueMode; +import org.apache.flink.runtime.operators.testutils.TestData.Key; +import org.apache.flink.runtime.operators.testutils.UniformRecordGenerator; +import org.apache.flink.runtime.operators.testutils.UnionIterator; +import org.apache.flink.types.IntValue; +import org.apache.flink.types.Record; +import org.apache.flink.util.Collector; +import org.apache.flink.util.MutableObjectIterator; +import org.junit.After; +import org.junit.Assert; +import org.junit.Before; +import org.junit.Test; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.Collection; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.Map.Entry; + +import static org.junit.Assert.fail; + +/** + * Test specialized hash join that keeps the build side data (in memory and on hard disk) + * This is used for iterative tasks. + */ +@SuppressWarnings("deprecation") +public class NonReusingReOpenableHashTableITCase { + + private static final int PAGE_SIZE = 8 * 1024; + private static final long MEMORY_SIZE = PAGE_SIZE * 1000; // 100 Pages. + + private static final long SEED1 = 561349061987311L; + private static final long SEED2 = 231434613412342L; + + private static final int NUM_PROBES = 3; // number of reopenings of hash join + + private final AbstractInvokable parentTask = new DummyInvokable(); + + private IOManager ioManager; + private MemoryManager memoryManager; + + private TypeSerializer recordSerializer; + private TypeComparator record1Comparator; + private TypeComparator record2Comparator; + private TypePairComparator recordPairComparator; + + + + + private static final AbstractInvokable MEM_OWNER = new DummyInvokable(); + private TypeSerializer recordBuildSideAccesssor; + private TypeSerializer recordProbeSideAccesssor; + private TypeComparator recordBuildSideComparator; + private TypeComparator recordProbeSideComparator; + private TypePairComparator pactRecordComparator; + + @SuppressWarnings("unchecked") + @Before + public void beforeTest() + { + this.recordSerializer = RecordSerializer.get(); + + this.record1Comparator = new RecordComparator(new int[] {0}, new Class[] {Key.class}); + this.record2Comparator = new RecordComparator(new int[] {0}, new Class[] {Key.class}); + this.recordPairComparator = new RecordPairComparator(new int[] {0}, new int[] {0}, new Class[] {Key.class}); + + + final int[] keyPos = new int[] {0}; + final Class[] keyType = (Class[]) new Class[] { IntValue.class }; + + this.recordBuildSideAccesssor = RecordSerializer.get(); + this.recordProbeSideAccesssor = RecordSerializer.get(); + this.recordBuildSideComparator = new RecordComparator(keyPos, keyType); + this.recordProbeSideComparator = new RecordComparator(keyPos, keyType); + this.pactRecordComparator = new HashTableITCase.RecordPairComparatorFirstInt(); + + this.memoryManager = new DefaultMemoryManager(MEMORY_SIZE,1, PAGE_SIZE); + this.ioManager = new IOManagerAsync(); + } + + @After + public void afterTest() + { + if (this.ioManager != null) { + this.ioManager.shutdown(); + if (!this.ioManager.isProperlyShutDown()) { + Assert.fail("I/O manager failed to properly shut down."); + } + this.ioManager = null; + } + + if (this.memoryManager != null) { + Assert.assertTrue("Memory Leak: Not all memory has been returned to the memory manager.", + this.memoryManager.verifyEmpty()); + this.memoryManager.shutdown(); + this.memoryManager = null; + } + } + + + /** + * Test behavior with overflow buckets (Overflow buckets must be initialized correctly + * if the input is reopened again) + */ + @Test + public void testOverflow() { + + int buildSize = 1000; + int probeSize = 1000; + try { + Generator bgen = new Generator(SEED1, 200, 1024, KeyMode.RANDOM, ValueMode.FIX_LENGTH); + Generator pgen = new Generator(SEED2, 0, 1024, KeyMode.SORTED, ValueMode.FIX_LENGTH); + + final TestData.GeneratorIterator buildInput = new TestData.GeneratorIterator(bgen, buildSize); + final TestData.GeneratorIterator probeInput = new TestData.GeneratorIterator(pgen, probeSize); + doTest(buildInput,probeInput, bgen, pgen); + } + catch (Exception e) { + e.printStackTrace(); + Assert.fail("An exception occurred during the test: " + e.getMessage()); + } + } + + /** + * Verify proper operation if the build side is spilled to disk. + */ + @Test + public void testDoubleProbeSpilling() { + + int buildSize = 1000; + int probeSize = 1000; + try { + Generator bgen = new Generator(SEED1, 0, 1024, KeyMode.SORTED, ValueMode.FIX_LENGTH); + Generator pgen = new Generator(SEED2, 0, 1024, KeyMode.SORTED, ValueMode.FIX_LENGTH); + + final TestData.GeneratorIterator buildInput = new TestData.GeneratorIterator(bgen, buildSize); + final TestData.GeneratorIterator probeInput = new TestData.GeneratorIterator(pgen, probeSize); + doTest(buildInput,probeInput, bgen, pgen); + } + catch (Exception e) { + e.printStackTrace(); + Assert.fail("An exception occurred during the test: " + e.getMessage()); + } + } + + /** + * This test case verifies that hybrid hash join is able to handle multiple probe phases + * when the build side fits completely into memory. + */ + @Test + public void testDoubleProbeInMemory() { + + int buildSize = 1000; + int probeSize = 1000; + try { + Generator bgen = new Generator(SEED1, 0, 28, KeyMode.SORTED, ValueMode.FIX_LENGTH); + Generator pgen = new Generator(SEED2, 0, 28, KeyMode.SORTED, ValueMode.FIX_LENGTH); + + final TestData.GeneratorIterator buildInput = new TestData.GeneratorIterator(bgen, buildSize); + final TestData.GeneratorIterator probeInput = new TestData.GeneratorIterator(pgen, probeSize); + + doTest(buildInput,probeInput, bgen, pgen); + } + catch (Exception e) { + e.printStackTrace(); + Assert.fail("An exception occurred during the test: " + e.getMessage()); + } + } + + private void doTest(TestData.GeneratorIterator buildInput, TestData.GeneratorIterator probeInput, Generator bgen, Generator pgen) throws Exception { + // collect expected data + final Map> expectedFirstMatchesMap = NonReusingHashMatchIteratorITCase.matchRecordValues(NonReusingHashMatchIteratorITCase.collectRecordData(buildInput), NonReusingHashMatchIteratorITCase.collectRecordData(probeInput)); + + final List>> expectedNMatchesMapList = new ArrayList>>(NUM_PROBES); + final JoinFunction[] nMatcher = new RecordMatchRemovingJoin[NUM_PROBES]; + for(int i = 0; i < NUM_PROBES; i++) { + Map> tmp; + expectedNMatchesMapList.add(tmp = deepCopy(expectedFirstMatchesMap)); + nMatcher[i] = new RecordMatchRemovingJoin(tmp); + } + + final JoinFunction firstMatcher = new RecordMatchRemovingJoin(expectedFirstMatchesMap); + + final Collector collector = new DiscardingOutputCollector(); + + // reset the generators + bgen.reset(); + pgen.reset(); + buildInput.reset(); + probeInput.reset(); + + // compare with iterator values + NonReusingBuildFirstReOpenableHashMatchIterator iterator = + new NonReusingBuildFirstReOpenableHashMatchIterator( + buildInput, probeInput, this.recordSerializer, this.record1Comparator, + this.recordSerializer, this.record2Comparator, this.recordPairComparator, + this.memoryManager, ioManager, this.parentTask, 1.0); + + iterator.open(); + // do first join with both inputs + while (iterator.callWithNextKey(firstMatcher, collector)); + + // assert that each expected match was seen for the first input + for (Entry> entry : expectedFirstMatchesMap.entrySet()) { + if (!entry.getValue().isEmpty()) { + Assert.fail("Collection for key " + entry.getKey() + " is not empty"); + } + } + + for(int i = 0; i < NUM_PROBES; i++) { + pgen.reset(); + probeInput.reset(); + // prepare .. + iterator.reopenProbe(probeInput); + // .. and do second join + while (iterator.callWithNextKey(nMatcher[i], collector)); + + // assert that each expected match was seen for the second input + for (Entry> entry : expectedNMatchesMapList.get(i).entrySet()) { + if (!entry.getValue().isEmpty()) { + Assert.fail("Collection for key " + entry.getKey() + " is not empty"); + } + } + } + + iterator.close(); + } + + // + // + // Tests taken from HahTableITCase! + // + // + + private final MutableObjectIterator getProbeInput(final int numKeys, + final int probeValsPerKey, final int repeatedValue1, final int repeatedValue2) { + MutableObjectIterator probe1 = new UniformRecordGenerator(numKeys, probeValsPerKey, true); + MutableObjectIterator probe2 = new ConstantsKeyValuePairsIterator(repeatedValue1, 17, 5); + MutableObjectIterator probe3 = new ConstantsKeyValuePairsIterator(repeatedValue2, 23, 5); + List> probes = new ArrayList>(); + probes.add(probe1); + probes.add(probe2); + probes.add(probe3); + return new UnionIterator(probes); + } + + @Test + public void testSpillingHashJoinWithMassiveCollisions() throws IOException + { + // the following two values are known to have a hash-code collision on the initial level. + // we use them to make sure one partition grows over-proportionally large + final int REPEATED_VALUE_1 = 40559; + final int REPEATED_VALUE_2 = 92882; + final int REPEATED_VALUE_COUNT_BUILD = 200000; + final int REPEATED_VALUE_COUNT_PROBE = 5; + + final int NUM_KEYS = 1000000; + final int BUILD_VALS_PER_KEY = 3; + final int PROBE_VALS_PER_KEY = 10; + + // create a build input that gives 3 million pairs with 3 values sharing the same key, plus 400k pairs with two colliding keys + MutableObjectIterator build1 = new UniformRecordGenerator(NUM_KEYS, BUILD_VALS_PER_KEY, false); + MutableObjectIterator build2 = new ConstantsKeyValuePairsIterator(REPEATED_VALUE_1, 17, REPEATED_VALUE_COUNT_BUILD); + MutableObjectIterator build3 = new ConstantsKeyValuePairsIterator(REPEATED_VALUE_2, 23, REPEATED_VALUE_COUNT_BUILD); + List> builds = new ArrayList>(); + builds.add(build1); + builds.add(build2); + builds.add(build3); + MutableObjectIterator buildInput = new UnionIterator(builds); + + + + + // allocate the memory for the HashTable + List memSegments; + try { + memSegments = this.memoryManager.allocatePages(MEM_OWNER, 896); + } + catch (MemoryAllocationException maex) { + fail("Memory for the Join could not be provided."); + return; + } + + // create the map for validating the results + HashMap map = new HashMap(NUM_KEYS); + + // ---------------------------------------------------------------------------------------- + + final ReOpenableMutableHashTable join = new ReOpenableMutableHashTable( + this.recordBuildSideAccesssor, this.recordProbeSideAccesssor, + this.recordBuildSideComparator, this.recordProbeSideComparator, this.pactRecordComparator, + memSegments, ioManager); + + for(int probe = 0; probe < NUM_PROBES; probe++) { + // create a probe input that gives 10 million pairs with 10 values sharing a key + MutableObjectIterator probeInput = getProbeInput(NUM_KEYS, PROBE_VALS_PER_KEY, REPEATED_VALUE_1, REPEATED_VALUE_2); + if(probe == 0) { + join.open(buildInput, probeInput); + } else { + join.reopenProbe(probeInput); + } + + Record record; + final Record recordReuse = new Record(); + + while (join.nextRecord()) + { + int numBuildValues = 0; + + final Record probeRec = join.getCurrentProbeRecord(); + int key = probeRec.getField(0, IntValue.class).getValue(); + + HashBucketIterator buildSide = join.getBuildSideIterator(); + if ((record = buildSide.next(recordReuse)) != null) { + numBuildValues = 1; + Assert.assertEquals("Probe-side key was different than build-side key.", key, record.getField(0, IntValue.class).getValue()); + } + else { + fail("No build side values found for a probe key."); + } + while ((record = buildSide.next(record)) != null) { + numBuildValues++; + Assert.assertEquals("Probe-side key was different than build-side key.", key, record.getField(0, IntValue.class).getValue()); + } + + Long contained = map.get(key); + if (contained == null) { + contained = Long.valueOf(numBuildValues); + } + else { + contained = Long.valueOf(contained.longValue() + numBuildValues); + } + + map.put(key, contained); + } + } + + join.close(); + + Assert.assertEquals("Wrong number of keys", NUM_KEYS, map.size()); + for (Entry entry : map.entrySet()) { + long val = entry.getValue(); + int key = entry.getKey(); + + if( key == REPEATED_VALUE_1 || key == REPEATED_VALUE_2) { + Assert.assertEquals("Wrong number of values in per-key cross product for key " + key, + (PROBE_VALS_PER_KEY + REPEATED_VALUE_COUNT_PROBE) * (BUILD_VALS_PER_KEY + REPEATED_VALUE_COUNT_BUILD) * NUM_PROBES, val); + } else { + Assert.assertEquals("Wrong number of values in per-key cross product for key " + key, + PROBE_VALS_PER_KEY * BUILD_VALS_PER_KEY * NUM_PROBES, val); + } + } + + + // ---------------------------------------------------------------------------------------- + + this.memoryManager.release(join.getFreedMemory()); + } + + /* + * This test is basically identical to the "testSpillingHashJoinWithMassiveCollisions" test, only that the number + * of repeated values (causing bucket collisions) are large enough to make sure that their target partition no longer + * fits into memory by itself and needs to be repartitioned in the recursion again. + */ + @Test + public void testSpillingHashJoinWithTwoRecursions() throws IOException + { + // the following two values are known to have a hash-code collision on the first recursion level. + // we use them to make sure one partition grows over-proportionally large + final int REPEATED_VALUE_1 = 40559; + final int REPEATED_VALUE_2 = 92882; + final int REPEATED_VALUE_COUNT_BUILD = 200000; + final int REPEATED_VALUE_COUNT_PROBE = 5; + + final int NUM_KEYS = 1000000; + final int BUILD_VALS_PER_KEY = 3; + final int PROBE_VALS_PER_KEY = 10; + + // create a build input that gives 3 million pairs with 3 values sharing the same key, plus 400k pairs with two colliding keys + MutableObjectIterator build1 = new UniformRecordGenerator(NUM_KEYS, BUILD_VALS_PER_KEY, false); + MutableObjectIterator build2 = new ConstantsKeyValuePairsIterator(REPEATED_VALUE_1, 17, REPEATED_VALUE_COUNT_BUILD); + MutableObjectIterator build3 = new ConstantsKeyValuePairsIterator(REPEATED_VALUE_2, 23, REPEATED_VALUE_COUNT_BUILD); + List> builds = new ArrayList>(); + builds.add(build1); + builds.add(build2); + builds.add(build3); + MutableObjectIterator buildInput = new UnionIterator(builds); + + + // allocate the memory for the HashTable + List memSegments; + try { + memSegments = this.memoryManager.allocatePages(MEM_OWNER, 896); + } + catch (MemoryAllocationException maex) { + fail("Memory for the Join could not be provided."); + return; + } + + // create the map for validating the results + HashMap map = new HashMap(NUM_KEYS); + + // ---------------------------------------------------------------------------------------- + + final ReOpenableMutableHashTable join = new ReOpenableMutableHashTable( + this.recordBuildSideAccesssor, this.recordProbeSideAccesssor, + this.recordBuildSideComparator, this.recordProbeSideComparator, this.pactRecordComparator, + memSegments, ioManager); + for(int probe = 0; probe < NUM_PROBES; probe++) { + // create a probe input that gives 10 million pairs with 10 values sharing a key + MutableObjectIterator probeInput = getProbeInput(NUM_KEYS, PROBE_VALS_PER_KEY, REPEATED_VALUE_1, REPEATED_VALUE_2); + if(probe == 0) { + join.open(buildInput, probeInput); + } else { + join.reopenProbe(probeInput); + } + Record record; + final Record recordReuse = new Record(); + + while (join.nextRecord()) + { + int numBuildValues = 0; + + final Record probeRec = join.getCurrentProbeRecord(); + int key = probeRec.getField(0, IntValue.class).getValue(); + + HashBucketIterator buildSide = join.getBuildSideIterator(); + if ((record = buildSide.next(recordReuse)) != null) { + numBuildValues = 1; + Assert.assertEquals("Probe-side key was different than build-side key.", key, record.getField(0, IntValue.class).getValue()); + } + else { + fail("No build side values found for a probe key."); + } + while ((record = buildSide.next(recordReuse)) != null) { + numBuildValues++; + Assert.assertEquals("Probe-side key was different than build-side key.", key, record.getField(0, IntValue.class).getValue()); + } + + Long contained = map.get(key); + if (contained == null) { + contained = Long.valueOf(numBuildValues); + } + else { + contained = Long.valueOf(contained.longValue() + numBuildValues); + } + + map.put(key, contained); + } + } + + join.close(); + Assert.assertEquals("Wrong number of keys", NUM_KEYS, map.size()); + for (Entry entry : map.entrySet()) { + long val = entry.getValue(); + int key = entry.getKey(); + + if( key == REPEATED_VALUE_1 || key == REPEATED_VALUE_2) { + Assert.assertEquals("Wrong number of values in per-key cross product for key " + key, + (PROBE_VALS_PER_KEY + REPEATED_VALUE_COUNT_PROBE) * (BUILD_VALS_PER_KEY + REPEATED_VALUE_COUNT_BUILD) * NUM_PROBES, val); + } else { + Assert.assertEquals("Wrong number of values in per-key cross product for key " + key, + PROBE_VALS_PER_KEY * BUILD_VALS_PER_KEY * NUM_PROBES, val); + } + } + + + // ---------------------------------------------------------------------------------------- + + this.memoryManager.release(join.getFreedMemory()); + } + + + static Map> deepCopy(Map> expectedSecondMatchesMap) { + Map> copy = new HashMap>(expectedSecondMatchesMap.size()); + for(Entry> entry : expectedSecondMatchesMap.entrySet()) { + List matches = new ArrayList(entry.getValue().size()); + for(RecordMatch m : entry.getValue()) { + matches.add(m); + } + copy.put(entry.getKey(), matches); + } + return copy; + } + +} diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/hash/HashMatchIteratorITCase.java b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/hash/ReusingHashMatchIteratorITCase.java similarity index 96% rename from flink-runtime/src/test/java/org/apache/flink/runtime/operators/hash/HashMatchIteratorITCase.java rename to flink-runtime/src/test/java/org/apache/flink/runtime/operators/hash/ReusingHashMatchIteratorITCase.java index 21e686de70aec..18cd8d0da6825 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/hash/HashMatchIteratorITCase.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/hash/ReusingHashMatchIteratorITCase.java @@ -63,7 +63,7 @@ import org.junit.Test; @SuppressWarnings({"serial", "deprecation"}) -public class HashMatchIteratorITCase { +public class ReusingHashMatchIteratorITCase { private static final int MEMORY_SIZE = 16000000; // total memory @@ -151,8 +151,8 @@ public void testBuildFirst() { input2.reset(); // compare with iterator values - BuildFirstHashMatchIterator iterator = - new BuildFirstHashMatchIterator( + ReusingBuildFirstHashMatchIterator iterator = + new ReusingBuildFirstHashMatchIterator( input1, input2, this.recordSerializer, this.record1Comparator, this.recordSerializer, this.record2Comparator, this.recordPairComparator, this.memoryManager, ioManager, this.parentTask, 1.0); @@ -238,8 +238,8 @@ public void testBuildFirstWithHighNumberOfCommonKeys() final JoinFunction matcher = new RecordMatchRemovingJoin(expectedMatchesMap); final Collector collector = new DiscardingOutputCollector(); - BuildFirstHashMatchIterator iterator = - new BuildFirstHashMatchIterator( + ReusingBuildFirstHashMatchIterator iterator = + new ReusingBuildFirstHashMatchIterator( input1, input2, this.recordSerializer, this.record1Comparator, this.recordSerializer, this.record2Comparator, this.recordPairComparator, this.memoryManager, ioManager, this.parentTask, 1.0); @@ -287,8 +287,8 @@ public void testBuildSecond() { input2.reset(); // compare with iterator values - BuildSecondHashMatchIterator iterator = - new BuildSecondHashMatchIterator( + ReusingBuildSecondHashMatchIterator iterator = + new ReusingBuildSecondHashMatchIterator( input1, input2, this.recordSerializer, this.record1Comparator, this.recordSerializer, this.record2Comparator, this.recordPairComparator, this.memoryManager, ioManager, this.parentTask, 1.0); @@ -374,8 +374,8 @@ public void testBuildSecondWithHighNumberOfCommonKeys() final JoinFunction matcher = new RecordMatchRemovingJoin(expectedMatchesMap); final Collector collector = new DiscardingOutputCollector(); - BuildSecondHashMatchIterator iterator = - new BuildSecondHashMatchIterator( + ReusingBuildSecondHashMatchIterator iterator = + new ReusingBuildSecondHashMatchIterator( input1, input2, this.recordSerializer, this.record1Comparator, this.recordSerializer, this.record2Comparator, this.recordPairComparator, this.memoryManager, ioManager, this.parentTask, 1.0); @@ -421,8 +421,8 @@ public void testBuildFirstWithMixedDataTypes() { input2.reset(); // compare with iterator values - BuildSecondHashMatchIterator iterator = - new BuildSecondHashMatchIterator( + ReusingBuildSecondHashMatchIterator iterator = + new ReusingBuildSecondHashMatchIterator( input1, input2, this.pairSerializer, this.pairComparator, this.recordSerializer, this.record2Comparator, this.pairRecordPairComparator, this.memoryManager, this.ioManager, this.parentTask, 1.0); @@ -468,8 +468,8 @@ public void testBuildSecondWithMixedDataTypes() { input2.reset(); // compare with iterator values - BuildFirstHashMatchIterator iterator = - new BuildFirstHashMatchIterator( + ReusingBuildFirstHashMatchIterator iterator = + new ReusingBuildFirstHashMatchIterator( input1, input2, this.pairSerializer, this.pairComparator, this.recordSerializer, this.record2Comparator, this.recordPairPairComparator, this.memoryManager, this.ioManager, this.parentTask, 1.0); diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/hash/ReOpenableHashTableITCase.java b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/hash/ReusingReOpenableHashTableITCase.java similarity index 97% rename from flink-runtime/src/test/java/org/apache/flink/runtime/operators/hash/ReOpenableHashTableITCase.java rename to flink-runtime/src/test/java/org/apache/flink/runtime/operators/hash/ReusingReOpenableHashTableITCase.java index 71f1979484847..fd2f906b08285 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/hash/ReOpenableHashTableITCase.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/hash/ReusingReOpenableHashTableITCase.java @@ -43,8 +43,8 @@ import org.apache.flink.runtime.memorymanager.DefaultMemoryManager; import org.apache.flink.runtime.memorymanager.MemoryAllocationException; import org.apache.flink.runtime.memorymanager.MemoryManager; -import org.apache.flink.runtime.operators.hash.HashMatchIteratorITCase.RecordMatch; -import org.apache.flink.runtime.operators.hash.HashMatchIteratorITCase.RecordMatchRemovingJoin; +import org.apache.flink.runtime.operators.hash.ReusingHashMatchIteratorITCase.RecordMatch; +import org.apache.flink.runtime.operators.hash.ReusingHashMatchIteratorITCase.RecordMatchRemovingJoin; import org.apache.flink.runtime.operators.hash.HashTableITCase.ConstantsKeyValuePairsIterator; import org.apache.flink.runtime.operators.hash.MutableHashTable.HashBucketIterator; import org.apache.flink.runtime.operators.testutils.DiscardingOutputCollector; @@ -70,7 +70,7 @@ * This is used for iterative tasks. */ @SuppressWarnings("deprecation") -public class ReOpenableHashTableITCase { +public class ReusingReOpenableHashTableITCase { private static final int PAGE_SIZE = 8 * 1024; private static final long MEMORY_SIZE = PAGE_SIZE * 1000; // 100 Pages. @@ -215,9 +215,7 @@ public void testDoubleProbeInMemory() { private void doTest(TestData.GeneratorIterator buildInput, TestData.GeneratorIterator probeInput, Generator bgen, Generator pgen) throws Exception { // collect expected data - final Map> expectedFirstMatchesMap = HashMatchIteratorITCase.matchRecordValues( - HashMatchIteratorITCase.collectRecordData(buildInput), - HashMatchIteratorITCase.collectRecordData(probeInput)); + final Map> expectedFirstMatchesMap = ReusingHashMatchIteratorITCase.matchRecordValues(ReusingHashMatchIteratorITCase.collectRecordData(buildInput), ReusingHashMatchIteratorITCase.collectRecordData(probeInput)); final List>> expectedNMatchesMapList = new ArrayList>>(NUM_PROBES); final JoinFunction[] nMatcher = new RecordMatchRemovingJoin[NUM_PROBES]; @@ -238,8 +236,8 @@ private void doTest(TestData.GeneratorIterator buildInput, TestData.GeneratorIte probeInput.reset(); // compare with iterator values - BuildFirstReOpenableHashMatchIterator iterator = - new BuildFirstReOpenableHashMatchIterator( + ReusingBuildFirstReOpenableHashMatchIterator iterator = + new ReusingBuildFirstReOpenableHashMatchIterator( buildInput, probeInput, this.recordSerializer, this.record1Comparator, this.recordSerializer, this.record2Comparator, this.recordPairComparator, this.memoryManager, ioManager, this.parentTask, 1.0); diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/sort/CombiningUnilateralSortMergerITCase.java b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/sort/CombiningUnilateralSortMergerITCase.java index a9d17c5450c14..0ba98230f1e37 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/sort/CombiningUnilateralSortMergerITCase.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/sort/CombiningUnilateralSortMergerITCase.java @@ -44,7 +44,7 @@ import org.apache.flink.runtime.operators.testutils.TestData.Key; import org.apache.flink.runtime.operators.testutils.TestData.Generator.KeyMode; import org.apache.flink.runtime.operators.testutils.TestData.Generator.ValueMode; -import org.apache.flink.runtime.util.KeyGroupedIterator; +import org.apache.flink.runtime.util.ReusingKeyGroupedIterator; import org.apache.flink.types.IntValue; import org.apache.flink.types.Record; import org.apache.flink.util.Collector; @@ -336,7 +336,7 @@ public void close() throws Exception { private static Iterator getReducingIterator(MutableObjectIterator data, TypeSerializer serializer, TypeComparator comparator) { - final KeyGroupedIterator groupIter = new KeyGroupedIterator(data, serializer, comparator); + final ReusingKeyGroupedIterator groupIter = new ReusingKeyGroupedIterator(data, serializer, comparator); return new Iterator() { diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/sort/MassiveStringValueSortingITCase.java b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/sort/MassiveStringValueSortingITCase.java index f2a3fc7d974f5..f4ceed3f3d5f8 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/sort/MassiveStringValueSortingITCase.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/sort/MassiveStringValueSortingITCase.java @@ -279,6 +279,11 @@ public StringValue next(StringValue reuse) throws IOException { reuse.setValue(line); return reuse; } + + @Override + public StringValue next() throws IOException { + return next(new StringValue()); + } } private static final class StringValueTupleReaderMutableObjectIterator implements MutableObjectIterator> { @@ -306,6 +311,11 @@ public Tuple2 next(Tuple2 next() throws IOException { + return next(new Tuple2(new StringValue(), new StringValue[0])); + } } // -------------------------------------------------------------------------------------------- diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/sort/NonReusingSortMergeCoGroupIteratorITCase.java b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/sort/NonReusingSortMergeCoGroupIteratorITCase.java new file mode 100644 index 0000000000000..1a6884e3a02a0 --- /dev/null +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/sort/NonReusingSortMergeCoGroupIteratorITCase.java @@ -0,0 +1,227 @@ +/* + * 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.flink.runtime.operators.sort; + +import org.apache.flink.api.common.typeutils.TypeComparator; +import org.apache.flink.api.common.typeutils.TypePairComparator; +import org.apache.flink.api.common.typeutils.TypeSerializer; +import org.apache.flink.api.common.typeutils.record.RecordComparator; +import org.apache.flink.api.common.typeutils.record.RecordPairComparator; +import org.apache.flink.api.common.typeutils.record.RecordSerializer; +import org.apache.flink.runtime.operators.testutils.TestData; +import org.apache.flink.runtime.operators.testutils.TestData.Generator; +import org.apache.flink.runtime.operators.testutils.TestData.Generator.KeyMode; +import org.apache.flink.runtime.operators.testutils.TestData.Generator.ValueMode; +import org.apache.flink.types.Record; +import org.apache.flink.util.MutableObjectIterator; +import org.junit.Assert; +import org.junit.Before; +import org.junit.Test; + +import java.util.ArrayList; +import java.util.Collection; +import java.util.HashMap; +import java.util.HashSet; +import java.util.Iterator; +import java.util.List; +import java.util.Map; +import java.util.Set; + +/** + */ +public class NonReusingSortMergeCoGroupIteratorITCase +{ + // the size of the left and right inputs + private static final int INPUT_1_SIZE = 20000; + + private static final int INPUT_2_SIZE = 1000; + + // random seeds for the left and right input data generators + private static final long SEED1 = 561349061987311L; + + private static final long SEED2 = 231434613412342L; + + // left and right input data generators + private Generator generator1; + + private Generator generator2; + + // left and right input RecordReader mocks + private MutableObjectIterator reader1; + + private MutableObjectIterator reader2; + + + private TypeSerializer serializer1; + private TypeSerializer serializer2; + private TypeComparator comparator1; + private TypeComparator comparator2; + private TypePairComparator pairComparator; + + + @SuppressWarnings("unchecked") + @Before + public void beforeTest() { + this.serializer1 = RecordSerializer.get(); + this.serializer2 = RecordSerializer.get(); + this.comparator1 = new RecordComparator(new int[] {0}, new Class[]{TestData.Key.class}); + this.comparator2 = new RecordComparator(new int[] {0}, new Class[]{TestData.Key.class}); + this.pairComparator = new RecordPairComparator(new int[] {0}, new int[] {0}, new Class[]{TestData.Key.class}); + } + + @Test + public void testMerge() { + try { + + generator1 = new Generator(SEED1, 500, 4096, KeyMode.SORTED, ValueMode.RANDOM_LENGTH); + generator2 = new Generator(SEED2, 500, 2048, KeyMode.SORTED, ValueMode.RANDOM_LENGTH); + + reader1 = new TestData.GeneratorIterator(generator1, INPUT_1_SIZE); + reader2 = new TestData.GeneratorIterator(generator2, INPUT_2_SIZE); + + // collect expected data + Map> expectedValuesMap1 = collectData(generator1, INPUT_1_SIZE); + Map> expectedValuesMap2 = collectData(generator2, INPUT_2_SIZE); + Map>> expectedCoGroupsMap = coGroupValues(expectedValuesMap1, expectedValuesMap2); + + // reset the generators + generator1.reset(); + generator2.reset(); + + // compare with iterator values + NonReusingSortMergeCoGroupIterator iterator = new NonReusingSortMergeCoGroupIterator( + this.reader1, this.reader2, this.serializer1, this.comparator1, this.serializer2, this.comparator2, + this.pairComparator); + + iterator.open(); + + final TestData.Key key = new TestData.Key(); + while (iterator.next()) + { + Iterator iter1 = iterator.getValues1().iterator(); + Iterator iter2 = iterator.getValues2().iterator(); + + TestData.Value v1 = null; + TestData.Value v2 = null; + + if (iter1.hasNext()) { + Record rec = iter1.next(); + rec.getFieldInto(0, key); + v1 = rec.getField(1, TestData.Value.class); + } + else if (iter2.hasNext()) { + Record rec = iter2.next(); + rec.getFieldInto(0, key); + v2 = rec.getField(1, TestData.Value.class); + } + else { + Assert.fail("No input on both sides."); + } + + // assert that matches for this key exist + Assert.assertTrue("No matches for key " + key, expectedCoGroupsMap.containsKey(key)); + + Collection expValues1 = expectedCoGroupsMap.get(key).get(0); + Collection expValues2 = expectedCoGroupsMap.get(key).get(1); + + if (v1 != null) { + expValues1.remove(v1); + } + else { + expValues2.remove(v2); + } + + while(iter1.hasNext()) { + Record rec = iter1.next(); + Assert.assertTrue("Value not in expected set of first input", expValues1.remove(rec.getField(1, TestData.Value.class))); + } + Assert.assertTrue("Expected set of first input not empty", expValues1.isEmpty()); + + while(iter2.hasNext()) { + Record rec = iter2.next(); + Assert.assertTrue("Value not in expected set of second input", expValues2.remove(rec.getField(1, TestData.Value.class))); + } + Assert.assertTrue("Expected set of second input not empty", expValues2.isEmpty()); + + expectedCoGroupsMap.remove(key); + } + iterator.close(); + + Assert.assertTrue("Expected key set not empty", expectedCoGroupsMap.isEmpty()); + } + catch (Exception e) { + e.printStackTrace(); + Assert.fail("An exception occurred during the test: " + e.getMessage()); + } + } + + // -------------------------------------------------------------------------------------------- + + private Map>> coGroupValues( + Map> leftMap, + Map> rightMap) + { + Map>> map = new HashMap>>(1000); + + Set keySet = new HashSet(leftMap.keySet()); + keySet.addAll(rightMap.keySet()); + + for (TestData.Key key : keySet) { + Collection leftValues = leftMap.get(key); + Collection rightValues = rightMap.get(key); + ArrayList> list = new ArrayList>(2); + + if (leftValues == null) { + list.add(new ArrayList(0)); + } else { + list.add(leftValues); + } + + if (rightValues == null) { + list.add(new ArrayList(0)); + } else { + list.add(rightValues); + } + + map.put(key, list); + } + return map; + } + + private Map> collectData(Generator iter, int num) + throws Exception + { + Map> map = new HashMap>(); + Record pair = new Record(); + + for (int i = 0; i < num; i++) { + iter.next(pair); + TestData.Key key = pair.getField(0, TestData.Key.class); + + if (!map.containsKey(key)) { + map.put(new TestData.Key(key.getKey()), new ArrayList()); + } + + Collection values = map.get(key); + values.add(new TestData.Value(pair.getField(1, TestData.Value.class).getValue())); + } + return map; + } +} diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/sort/SortMergeCoGroupIteratorITCase.java b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/sort/ReusingSortMergeCoGroupIteratorITCase.java similarity index 97% rename from flink-runtime/src/test/java/org/apache/flink/runtime/operators/sort/SortMergeCoGroupIteratorITCase.java rename to flink-runtime/src/test/java/org/apache/flink/runtime/operators/sort/ReusingSortMergeCoGroupIteratorITCase.java index 9d6fafbeda51b..a487a6527dff3 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/sort/SortMergeCoGroupIteratorITCase.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/sort/ReusingSortMergeCoGroupIteratorITCase.java @@ -46,7 +46,7 @@ /** */ -public class SortMergeCoGroupIteratorITCase +public class ReusingSortMergeCoGroupIteratorITCase { // the size of the left and right inputs private static final int INPUT_1_SIZE = 20000; @@ -106,7 +106,7 @@ public void testMerge() { generator2.reset(); // compare with iterator values - SortMergeCoGroupIterator iterator = new SortMergeCoGroupIterator( + ReusingSortMergeCoGroupIterator iterator = new ReusingSortMergeCoGroupIterator( this.reader1, this.reader2, this.serializer1, this.comparator1, this.serializer2, this.comparator2, this.pairComparator); diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/testutils/DriverTestBase.java b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/testutils/DriverTestBase.java index 02206f6b71ec0..0e169ec7a3be1 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/testutils/DriverTestBase.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/testutils/DriverTestBase.java @@ -18,9 +18,14 @@ package org.apache.flink.runtime.operators.testutils; +import java.io.FileNotFoundException; +import java.io.IOException; import java.util.ArrayList; +import java.util.Collection; +import java.util.LinkedList; import java.util.List; +import org.apache.flink.api.common.ExecutionConfig; import org.junit.Assert; import org.apache.flink.api.common.functions.Function; import org.apache.flink.api.common.functions.util.FunctionUtils; @@ -43,7 +48,10 @@ import org.apache.flink.util.Collector; import org.apache.flink.util.MutableObjectIterator; import org.junit.After; +import org.junit.runner.RunWith; +import org.junit.runners.Parameterized; +@RunWith(Parameterized.class) public class DriverTestBase implements PactTaskContext { protected static final long DEFAULT_PER_SORT_MEM = 16 * 1024 * 1024; @@ -79,12 +87,14 @@ public class DriverTestBase implements PactTaskContext driver; private volatile boolean running; + + private ExecutionConfig executionConfig; - protected DriverTestBase(long memory, int maxNumSorters) { - this(memory, maxNumSorters, DEFAULT_PER_SORT_MEM); + protected DriverTestBase(ExecutionConfig executionConfig, long memory, int maxNumSorters) { + this(executionConfig, memory, maxNumSorters, DEFAULT_PER_SORT_MEM); } - protected DriverTestBase(long memory, int maxNumSorters, long perSortMemory) { + protected DriverTestBase(ExecutionConfig executionConfig, long memory, int maxNumSorters, long perSortMemory) { if (memory < 0 || maxNumSorters < 0 || perSortMemory < 0) { throw new IllegalArgumentException(); } @@ -104,6 +114,27 @@ protected DriverTestBase(long memory, int maxNumSorters, long perSortMemory) { this.config = new Configuration(); this.taskConfig = new TaskConfig(this.config); + + this.executionConfig = executionConfig; + } + + @Parameterized.Parameters + public static Collection getConfigurations() throws FileNotFoundException, IOException { + + LinkedList configs = new LinkedList(); + + ExecutionConfig withReuse = new ExecutionConfig(); + withReuse.enableObjectReuse(); + + ExecutionConfig withoutReuse = new ExecutionConfig(); + withoutReuse.disableObjectReuse(); + + Object[] a = { withoutReuse }; + configs.add(a); + Object[] b = { withReuse }; + configs.add(b); + + return configs; } public void addInput(MutableObjectIterator input) { @@ -141,16 +172,21 @@ public void setNumFileHandlesForSort(int numFileHandles) { @SuppressWarnings({"unchecked","rawtypes"}) public void testDriver(PactDriver driver, Class stubClass) throws Exception { - + testDriverInternal(driver, stubClass); + } + + @SuppressWarnings({"unchecked","rawtypes"}) + public void testDriverInternal(PactDriver driver, Class stubClass) throws Exception { + this.driver = driver; driver.setup(this); - + this.stub = (S)stubClass.newInstance(); - + // regular running logic this.running = true; boolean stubOpen = false; - + try { // run the data preparation try { @@ -159,7 +195,7 @@ public void testDriver(PactDriver driver, Class stubClass) throws Exception { catch (Throwable t) { throw new Exception("The data preparation caused an error: " + t.getMessage(), t); } - + // open stub implementation try { FunctionUtils.openFunction(this.stub, getTaskConfig().getStubParameters()); @@ -168,16 +204,16 @@ public void testDriver(PactDriver driver, Class stubClass) throws Exception { catch (Throwable t) { throw new Exception("The user defined 'open()' method caused an exception: " + t.getMessage(), t); } - + // run the user code driver.run(); - + // close. We close here such that a regular close throwing an exception marks a task as failed. if (this.running) { FunctionUtils.closeFunction (this.stub); stubOpen = false; } - + this.output.close(); } catch (Exception ex) { @@ -188,7 +224,7 @@ public void testDriver(PactDriver driver, Class stubClass) throws Exception { } catch (Throwable t) {} } - + // if resettable driver invoke treardown if (this.driver instanceof ResettablePactDriver) { final ResettablePactDriver resDriver = (ResettablePactDriver) this.driver; @@ -198,18 +234,18 @@ public void testDriver(PactDriver driver, Class stubClass) throws Exception { throw new Exception("Error while shutting down an iterative operator: " + t.getMessage(), t); } } - + // drop exception, if the task was canceled if (this.running) { throw ex; } - + } finally { driver.cleanup(); } } - + @SuppressWarnings({"unchecked","rawtypes"}) public void testResettableDriver(ResettablePactDriver driver, Class stubClass, int iterations) throws Exception { @@ -242,6 +278,13 @@ public void cancel() throws Exception { public TaskConfig getTaskConfig() { return this.taskConfig; } + + + + @Override + public ExecutionConfig getExecutionConfig() { + return executionConfig; + } @Override public ClassLoader getUserCodeClassLoader() { diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/testutils/MockEnvironment.java b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/testutils/MockEnvironment.java index 63a168b822559..0629ea08e8a2c 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/testutils/MockEnvironment.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/testutils/MockEnvironment.java @@ -55,6 +55,7 @@ import org.apache.flink.runtime.memorymanager.DefaultMemoryManager; import org.apache.flink.runtime.memorymanager.MemoryManager; import org.apache.flink.runtime.plugable.DeserializationDelegate; +import org.apache.flink.runtime.plugable.NonReusingDeserializationDelegate; import org.apache.flink.types.Record; import org.apache.flink.util.MutableObjectIterator; @@ -192,8 +193,15 @@ public void registerRecordAvailabilityListener(final RecordAvailabilityListener< public InputChannelResult readRecord(DeserializationDelegate target) throws IOException, InterruptedException { Record reuse = target != null ? target.getInstance() : null; + + // Handle NonReusingDeserializationDelegate, which by default + // does not have a Record instance + if (reuse == null && target != null) { + reuse = new Record(); + target.setInstance(reuse); + } - if ((reuse = it.next(reuse)) != null) { + if (it.next(reuse) != null) { // everything comes from the same source channel and buffer in this mock notifyRecordIsAvailable(0); return InputChannelResult.INTERMEDIATE_RECORD_FROM_BUFFER; diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/util/HashVsSortMiniBenchmark.java b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/util/HashVsSortMiniBenchmark.java index ba387762c2dd8..e760a1ef1f894 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/util/HashVsSortMiniBenchmark.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/util/HashVsSortMiniBenchmark.java @@ -30,8 +30,8 @@ import org.apache.flink.runtime.jobgraph.tasks.AbstractInvokable; import org.apache.flink.runtime.memorymanager.DefaultMemoryManager; import org.apache.flink.runtime.memorymanager.MemoryManager; -import org.apache.flink.runtime.operators.hash.BuildFirstHashMatchIterator; -import org.apache.flink.runtime.operators.hash.BuildSecondHashMatchIterator; +import org.apache.flink.runtime.operators.hash.ReusingBuildFirstHashMatchIterator; +import org.apache.flink.runtime.operators.hash.ReusingBuildSecondHashMatchIterator; import org.apache.flink.runtime.operators.sort.MergeMatchIterator; import org.apache.flink.runtime.operators.sort.UnilateralSortMerger; import org.apache.flink.runtime.operators.testutils.DiscardingOutputCollector; @@ -183,8 +183,8 @@ public void testBuildFirst() { long start = System.nanoTime(); // compare with iterator values - final BuildFirstHashMatchIterator iterator = - new BuildFirstHashMatchIterator( + final ReusingBuildFirstHashMatchIterator iterator = + new ReusingBuildFirstHashMatchIterator( input1, input2, this.serializer1.getSerializer(), this.comparator1, this.serializer2.getSerializer(), this.comparator2, this.pairComparator11, this.memoryManager, this.ioManager, this.parentTask, MEMORY_SIZE); @@ -222,8 +222,8 @@ public void testBuildSecond() { long start = System.nanoTime(); // compare with iterator values - BuildSecondHashMatchIterator iterator = - new BuildSecondHashMatchIterator( + ReusingBuildSecondHashMatchIterator iterator = + new ReusingBuildSecondHashMatchIterator( input1, input2, this.serializer1.getSerializer(), this.comparator1, this.serializer2.getSerializer(), this.comparator2, this.pairComparator11, this.memoryManager, this.ioManager, this.parentTask, MEMORY_SIZE); diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/util/KeyGroupedIteratorImmutableTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/util/NonReusingKeyGroupedIteratorTest.java similarity index 98% rename from flink-runtime/src/test/java/org/apache/flink/runtime/util/KeyGroupedIteratorImmutableTest.java rename to flink-runtime/src/test/java/org/apache/flink/runtime/util/NonReusingKeyGroupedIteratorTest.java index 3d1a80b55980a..9f651b173c167 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/util/KeyGroupedIteratorImmutableTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/util/NonReusingKeyGroupedIteratorTest.java @@ -37,11 +37,11 @@ * Test for the safe key grouped iterator, which advances in windows containing the same key and provides a sub-iterator * over the records with the same key. */ -public class KeyGroupedIteratorImmutableTest { +public class NonReusingKeyGroupedIteratorTest { private MutableObjectIterator sourceIter; // the iterator that provides the input - private KeyGroupedIteratorImmutable psi; // the grouping iterator, progressing in key steps + private NonReusingKeyGroupedIterator psi; // the grouping iterator, progressing in key steps @Before public void setup() @@ -98,7 +98,7 @@ public Record next() throws IOException { @SuppressWarnings("unchecked") final RecordComparator comparator = new RecordComparator(new int[] {0}, new Class[] {IntValue.class}); - this.psi = new KeyGroupedIteratorImmutable(this.sourceIter, serializer, comparator); + this.psi = new NonReusingKeyGroupedIterator(this.sourceIter, serializer, comparator); } @Test diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/util/KeyGroupedIteratorTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/util/ReusingKeyGroupedIteratorTest.java similarity index 98% rename from flink-runtime/src/test/java/org/apache/flink/runtime/util/KeyGroupedIteratorTest.java rename to flink-runtime/src/test/java/org/apache/flink/runtime/util/ReusingKeyGroupedIteratorTest.java index 39ff077e8e198..8a9f8bae4b803 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/util/KeyGroupedIteratorTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/util/ReusingKeyGroupedIteratorTest.java @@ -25,7 +25,6 @@ import org.apache.flink.api.common.typeutils.record.RecordComparator; import org.apache.flink.api.common.typeutils.record.RecordSerializer; -import org.apache.flink.runtime.util.KeyGroupedIterator; import org.apache.flink.types.IntValue; import org.apache.flink.types.Record; import org.apache.flink.types.StringValue; @@ -39,11 +38,11 @@ * Test for the key grouped iterator, which advances in windows containing the same key and provides a sub-iterator * over the records with the same key. */ -public class KeyGroupedIteratorTest { +public class ReusingKeyGroupedIteratorTest { private MutableObjectIterator sourceIter; // the iterator that provides the input - private KeyGroupedIterator psi; // the grouping iterator, progressing in key steps + private ReusingKeyGroupedIterator psi; // the grouping iterator, progressing in key steps @Before public void setup() { @@ -100,7 +99,7 @@ public Record next() throws IOException { @SuppressWarnings("unchecked") final RecordComparator comparator = new RecordComparator(new int[] {0}, new Class[] {IntValue.class}); - this.psi = new KeyGroupedIterator(this.sourceIter, serializer, comparator); + this.psi = new ReusingKeyGroupedIterator(this.sourceIter, serializer, comparator); } @Test diff --git a/flink-test-utils/src/main/java/org/apache/flink/test/util/JavaProgramTestBase.java b/flink-test-utils/src/main/java/org/apache/flink/test/util/JavaProgramTestBase.java index 3304c7dc586a4..5e3a0a8fc471b 100644 --- a/flink-test-utils/src/main/java/org/apache/flink/test/util/JavaProgramTestBase.java +++ b/flink-test-utils/src/main/java/org/apache/flink/test/util/JavaProgramTestBase.java @@ -85,7 +85,7 @@ protected boolean skipCollectionExecution() { // -------------------------------------------------------------------------------------------- @Test - public void testJob() throws Exception { + public void testJobWithObjectReuse() throws Exception { isCollectionExecution = false; startCluster(); @@ -102,6 +102,7 @@ public void testJob() throws Exception { // prepare the test environment TestEnvironment env = new TestEnvironment(this.executor, this.degreeOfParallelism); + env.getConfig().enableObjectReuse(); env.setAsContext(); // call the test program @@ -130,6 +131,54 @@ public void testJob() throws Exception { stopCluster(); } } + + @Test + public void testJobWithoutObjectReuse() throws Exception { + isCollectionExecution = false; + + startCluster(); + try { + // pre-submit + try { + preSubmit(); + } + catch (Exception e) { + System.err.println(e.getMessage()); + e.printStackTrace(); + Assert.fail("Pre-submit work caused an error: " + e.getMessage()); + } + + // prepare the test environment + TestEnvironment env = new TestEnvironment(this.executor, this.degreeOfParallelism); + env.getConfig().disableObjectReuse(); + env.setAsContext(); + + // call the test program + try { + testProgram(); + this.latestExecutionResult = env.latestResult; + } + catch (Exception e) { + System.err.println(e.getMessage()); + e.printStackTrace(); + Assert.fail("Error while calling the test program: " + e.getMessage()); + } + + Assert.assertNotNull("The test program never triggered an execution.", this.latestExecutionResult); + + // post-submit + try { + postSubmit(); + } + catch (Exception e) { + System.err.println(e.getMessage()); + e.printStackTrace(); + Assert.fail("Post-submit work caused an error: " + e.getMessage()); + } + } finally { + stopCluster(); + } + } @Test public void testJobCollectionExecution() throws Exception { diff --git a/flink-tests/src/test/java/org/apache/flink/test/iterative/CoGroupConnectedComponentsITCase.java b/flink-tests/src/test/java/org/apache/flink/test/iterative/CoGroupConnectedComponentsITCase.java index 60328fe842275..0a4673a736f36 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/iterative/CoGroupConnectedComponentsITCase.java +++ b/flink-tests/src/test/java/org/apache/flink/test/iterative/CoGroupConnectedComponentsITCase.java @@ -100,7 +100,7 @@ public void coGroup(Iterator candidates, Iterator current, Colle } Record old = current.next(); long oldId = old.getField(1, LongValue.class).getValue(); - + long minimumComponentID = Long.MAX_VALUE; while (candidates.hasNext()) { @@ -110,7 +110,7 @@ public void coGroup(Iterator candidates, Iterator current, Colle minimumComponentID = candidateComponentID; } } - + if (minimumComponentID < oldId) { newComponentId.setValue(minimumComponentID); old.setField(1, newComponentId); diff --git a/flink-tests/src/test/java/org/apache/flink/test/iterative/DependencyConnectedComponentsITCase.java b/flink-tests/src/test/java/org/apache/flink/test/iterative/DependencyConnectedComponentsITCase.java index b59fe4fa85882..15079ec7e01d9 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/iterative/DependencyConnectedComponentsITCase.java +++ b/flink-tests/src/test/java/org/apache/flink/test/iterative/DependencyConnectedComponentsITCase.java @@ -57,6 +57,9 @@ public DependencyConnectedComponentsITCase(){ @Override protected void preSubmit() throws Exception { + verticesInput.clear(); + edgesInput.clear(); + // vertices input verticesInput.add(new Tuple2(1l,1l)); verticesInput.add(new Tuple2(2l,2l)); diff --git a/flink-tests/src/test/java/org/apache/flink/test/javaApiOperators/ReduceITCase.java b/flink-tests/src/test/java/org/apache/flink/test/javaApiOperators/ReduceITCase.java index 065be67bae790..1168e3c1ea35f 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/javaApiOperators/ReduceITCase.java +++ b/flink-tests/src/test/java/org/apache/flink/test/javaApiOperators/ReduceITCase.java @@ -243,29 +243,6 @@ public void testReduceWithBroadcastSet() throws Exception { "111,6,55\n"; } - @Test - public void testReduceWithUDFThatReturnsTheSecondInputObject() throws Exception { - /* - * Reduce with UDF that returns the second input object (check mutable object handling) - */ - - final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - - DataSet> ds = CollectionDataSets.get3TupleDataSet(env); - DataSet> reduceDs = ds. - groupBy(1).reduce(new InputReturningTuple3Reduce()); - - reduceDs.writeAsCsv(resultPath); - env.execute(); - - expected = "1,1,Hi\n" + - "5,2,Hi again!\n" + - "15,3,Hi again!\n" + - "34,4,Hi again!\n" + - "65,5,Hi again!\n" + - "111,6,Hi again!\n"; - } - @Test public void testReduceATupleReturningKeySelector() throws Exception { /* @@ -451,20 +428,6 @@ public CustomType reduce(CustomType in1, CustomType in2) } } - public static class InputReturningTuple3Reduce implements ReduceFunction> { - private static final long serialVersionUID = 1L; - - @Override - public Tuple3 reduce( - Tuple3 in1, - Tuple3 in2) throws Exception { - - in2.f0 = in1.f0 + in2.f0; - in2.f2 = "Hi again!"; - return in2; - } - } - public static class AllAddingTuple3Reduce implements ReduceFunction> { private static final long serialVersionUID = 1L; private final Tuple3 out = new Tuple3(); diff --git a/flink-tests/src/test/java/org/apache/flink/test/operators/ObjectReuseITCase.java b/flink-tests/src/test/java/org/apache/flink/test/operators/ObjectReuseITCase.java new file mode 100644 index 0000000000000..c840dc00b3763 --- /dev/null +++ b/flink-tests/src/test/java/org/apache/flink/test/operators/ObjectReuseITCase.java @@ -0,0 +1,247 @@ +/* + * 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.flink.test.operators; + +import org.apache.flink.api.common.functions.GroupReduceFunction; +import org.apache.flink.api.common.functions.ReduceFunction; +import org.apache.flink.api.java.DataSet; +import org.apache.flink.api.java.ExecutionEnvironment; +import org.apache.flink.api.java.tuple.Tuple2; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.test.util.JavaProgramTestBase; +import org.apache.flink.util.Collector; +import org.junit.runner.RunWith; +import org.junit.runners.Parameterized; +import org.junit.runners.Parameterized.Parameters; + +import java.io.FileNotFoundException; +import java.io.IOException; +import java.util.ArrayList; +import java.util.Collection; +import java.util.LinkedList; +import java.util.List; + +/** + * These check whether the object-reuse execution mode does really reuse objects. + */ +@RunWith(Parameterized.class) +public class ObjectReuseITCase extends JavaProgramTestBase { + + private static int NUM_PROGRAMS = 3; + + private int curProgId = config.getInteger("ProgramId", -1); + private String resultPath; + private String expectedResult; + + public ObjectReuseITCase(Configuration config) { + super(config); + } + + @Override + protected void preSubmit() throws Exception { + resultPath = getTempDirPath("result"); + } + + @Override + protected void testProgram() throws Exception { + expectedResult = Progs.runProgram(curProgId, resultPath); + } + + @Override + protected void postSubmit() throws Exception { + compareResultsByLinesInMemory(expectedResult, resultPath); + } + + @Override + protected boolean skipCollectionExecution() { + return true; + } + + + @Parameters + public static Collection getConfigurations() throws FileNotFoundException, IOException { + + LinkedList tConfigs = new LinkedList(); + + for(int i=1; i <= NUM_PROGRAMS; i++) { + Configuration config = new Configuration(); + config.setInteger("ProgramId", i); + tConfigs.add(config); + } + + return toParameterList(tConfigs); + } + + private static class Progs { + + public static String runProgram(int progId, String resultPath) throws Exception { + + switch(progId) { + + case 1: { + + final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); + env.getConfig().enableObjectReuse(); + + DataSet> input = env.fromElements( + new Tuple2("a", 1), + new Tuple2("a", 2), + new Tuple2("a", 3), + new Tuple2("a", 4), + new Tuple2("a", 50)); + + DataSet> result = input.groupBy(0).reduce(new ReduceFunction>() { + + @Override + public Tuple2 reduce(Tuple2 value1, Tuple2 value2) throws + Exception { + value2.f1 += value1.f1; + return value2; + } + + }); + + result.writeAsCsv(resultPath); + env.execute(); + + // return expected result + return "a,100\n"; + + } + + case 2: { + + final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); + env.getConfig().enableObjectReuse(); + + DataSet> input = env.fromElements( + new Tuple2("a", 1), + new Tuple2("a", 2), + new Tuple2("a", 3), + new Tuple2("a", 4), + new Tuple2("a", 50)); + + DataSet> result = input.reduce(new ReduceFunction>() { + + @Override + public Tuple2 reduce( + Tuple2 value1, + Tuple2 value2) throws Exception { + value2.f1 += value1.f1; + return value2; + } + + }); + + result.writeAsCsv(resultPath); + env.execute(); + + // return expected result + return "a,100\n"; + + } + + case 3: { + + final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); + env.getConfig().enableObjectReuse(); + + DataSet> input = env.fromElements( + new Tuple2("a", 1), + new Tuple2("a", 2), + new Tuple2("a", 3), + new Tuple2("a", 4), + new Tuple2("a", 5)); + + DataSet> result = input.reduceGroup(new GroupReduceFunction, Tuple2>() { + + @Override + public void reduce(Iterable> values, Collector> out) throws Exception { + List> list = new ArrayList>(); + for (Tuple2 val : values) { + list.add(val); + } + + for (Tuple2 val : list) { + out.collect(val); + } + } + + }); + + result.writeAsCsv(resultPath); + env.execute(); + + // return expected result + return "a,4\n" + + "a,4\n" + + "a,5\n" + + "a,5\n" + + "a,5\n"; + + } + + case 4: { + + final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); + env.getConfig().enableObjectReuse(); + + DataSet> input = env.fromElements( + new Tuple2("a", 1), + new Tuple2("a", 2), + new Tuple2("a", 3), + new Tuple2("a", 4), + new Tuple2("a", 5)); + + DataSet> result = input.reduceGroup(new GroupReduceFunction, Tuple2>() { + + @Override + public void reduce(Iterable> values, Collector> out) throws Exception { + List> list = new ArrayList>(); + for (Tuple2 val : values) { + list.add(val); + } + + for (Tuple2 val : list) { + out.collect(val); + } + } + + }); + + result.writeAsCsv(resultPath); + env.execute(); + + // return expected result + return "a,4\n" + + "a,4\n" + + "a,5\n" + + "a,5\n" + + "a,5\n"; + + } + + default: + throw new IllegalArgumentException("Invalid program id"); + } + + } + + } +} From d529749c8f45af693efffe1f69860dae0bfe70bf Mon Sep 17 00:00:00 2001 From: Aljoscha Krettek Date: Thu, 11 Dec 2014 14:58:23 +0100 Subject: [PATCH 3/4] [FLINK-1285] Make Merge-Join aware of object-reuse setting This closes #259 --- .../flink/runtime/operators/MatchDriver.java | 7 +- ...=> NonReusingBlockResettableIterator.java} | 28 +- .../ReusingBlockResettableIterator.java | 100 +++++ .../sort/NonReusingMergeMatchIterator.java | 424 ++++++++++++++++++ ...or.java => ReusingMergeMatchIterator.java} | 22 +- ...onReusingBlockResettableIteratorTest.java} | 9 +- .../ReusingBlockResettableIteratorTest.java | 201 +++++++++ ...nReusingSortMergeMatchIteratorITCase.java} | 12 +- .../ReusingSortMergeMatchIteratorITCase.java | 371 +++++++++++++++ .../util/HashVsSortMiniBenchmark.java | 6 +- 10 files changed, 1137 insertions(+), 43 deletions(-) rename flink-runtime/src/main/java/org/apache/flink/runtime/operators/resettable/{BlockResettableIterator.java => NonReusingBlockResettableIterator.java} (86%) create mode 100644 flink-runtime/src/main/java/org/apache/flink/runtime/operators/resettable/ReusingBlockResettableIterator.java create mode 100644 flink-runtime/src/main/java/org/apache/flink/runtime/operators/sort/NonReusingMergeMatchIterator.java rename flink-runtime/src/main/java/org/apache/flink/runtime/operators/sort/{MergeMatchIterator.java => ReusingMergeMatchIterator.java} (95%) rename flink-runtime/src/test/java/org/apache/flink/runtime/operators/resettable/{BlockResettableIteratorTest.java => NonReusingBlockResettableIteratorTest.java} (93%) create mode 100644 flink-runtime/src/test/java/org/apache/flink/runtime/operators/resettable/ReusingBlockResettableIteratorTest.java rename flink-runtime/src/test/java/org/apache/flink/runtime/operators/sort/{SortMergeMatchIteratorITCase.java => NonReusingSortMergeMatchIteratorITCase.java} (97%) create mode 100644 flink-runtime/src/test/java/org/apache/flink/runtime/operators/sort/ReusingSortMergeMatchIteratorITCase.java diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/MatchDriver.java b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/MatchDriver.java index 2d051ad4c872c..f8e4a2951c56a 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/MatchDriver.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/MatchDriver.java @@ -22,6 +22,7 @@ import org.apache.flink.api.common.ExecutionConfig; import org.apache.flink.runtime.operators.hash.NonReusingBuildFirstHashMatchIterator; import org.apache.flink.runtime.operators.hash.NonReusingBuildSecondHashMatchIterator; +import org.apache.flink.runtime.operators.sort.NonReusingMergeMatchIterator; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import org.apache.flink.api.common.functions.FlatJoinFunction; @@ -32,7 +33,7 @@ import org.apache.flink.runtime.memorymanager.MemoryManager; import org.apache.flink.runtime.operators.hash.ReusingBuildFirstHashMatchIterator; import org.apache.flink.runtime.operators.hash.ReusingBuildSecondHashMatchIterator; -import org.apache.flink.runtime.operators.sort.MergeMatchIterator; +import org.apache.flink.runtime.operators.sort.ReusingMergeMatchIterator; import org.apache.flink.runtime.operators.util.JoinTaskIterator; import org.apache.flink.runtime.operators.util.TaskConfig; import org.apache.flink.util.Collector; @@ -125,7 +126,7 @@ public void prepare() throws Exception{ if (this.objectReuseEnabled) { switch (ls) { case MERGE: - this.matchIterator = new MergeMatchIterator(in1, in2, serializer1, comparator1, serializer2, comparator2, pairComparatorFactory.createComparator12(comparator1, comparator2), memoryManager, ioManager, numPages, this.taskContext.getOwningNepheleTask()); + this.matchIterator = new ReusingMergeMatchIterator(in1, in2, serializer1, comparator1, serializer2, comparator2, pairComparatorFactory.createComparator12(comparator1, comparator2), memoryManager, ioManager, numPages, this.taskContext.getOwningNepheleTask()); break; case HYBRIDHASH_BUILD_FIRST: @@ -140,7 +141,7 @@ public void prepare() throws Exception{ } else { switch (ls) { case MERGE: - this.matchIterator = new MergeMatchIterator(in1, in2, serializer1, comparator1, serializer2, comparator2, pairComparatorFactory.createComparator12(comparator1, comparator2), memoryManager, ioManager, numPages, this.taskContext.getOwningNepheleTask()); + this.matchIterator = new NonReusingMergeMatchIterator(in1, in2, serializer1, comparator1, serializer2, comparator2, pairComparatorFactory.createComparator12(comparator1, comparator2), memoryManager, ioManager, numPages, this.taskContext.getOwningNepheleTask()); break; case HYBRIDHASH_BUILD_FIRST: diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/resettable/BlockResettableIterator.java b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/resettable/NonReusingBlockResettableIterator.java similarity index 86% rename from flink-runtime/src/main/java/org/apache/flink/runtime/operators/resettable/BlockResettableIterator.java rename to flink-runtime/src/main/java/org/apache/flink/runtime/operators/resettable/NonReusingBlockResettableIterator.java index 0019c8c11f1b8..9d581ce661453 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/resettable/BlockResettableIterator.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/resettable/NonReusingBlockResettableIterator.java @@ -36,41 +36,37 @@ * access to the data in that block. * */ -public class BlockResettableIterator extends AbstractBlockResettableIterator implements ResettableIterator { +public class NonReusingBlockResettableIterator extends AbstractBlockResettableIterator implements ResettableIterator { - public static final Logger LOG = LoggerFactory.getLogger(BlockResettableIterator.class); + public static final Logger LOG = LoggerFactory.getLogger(NonReusingBlockResettableIterator.class); // ------------------------------------------------------------------------ protected Iterator input; - private T nextElement; + protected T nextElement; - private final T reuseElement; + protected T leftOverElement; - private T leftOverElement; + protected boolean readPhase; - private boolean readPhase; - - private boolean noMoreBlocks; + protected boolean noMoreBlocks; // ------------------------------------------------------------------------ - public BlockResettableIterator(MemoryManager memoryManager, Iterator input, - TypeSerializer serializer, int numPages, AbstractInvokable ownerTask) + public NonReusingBlockResettableIterator(MemoryManager memoryManager, Iterator input, + TypeSerializer serializer, int numPages, + AbstractInvokable ownerTask) throws MemoryAllocationException { this(memoryManager, serializer, numPages, ownerTask); this.input = input; } - public BlockResettableIterator(MemoryManager memoryManager, - TypeSerializer serializer, int numPages, AbstractInvokable ownerTask) + public NonReusingBlockResettableIterator(MemoryManager memoryManager, TypeSerializer serializer, int numPages, AbstractInvokable ownerTask) throws MemoryAllocationException { super(serializer, memoryManager, numPages, ownerTask); - - this.reuseElement = serializer.createInstance(); } // ------------------------------------------------------------------------ @@ -83,8 +79,6 @@ public void reopen(Iterator input) throws IOException { nextBlock(); } - - @Override public boolean hasNext() { @@ -92,7 +86,7 @@ public boolean hasNext() { if (this.nextElement == null) { if (this.readPhase) { // read phase, get next element from buffer - T tmp = getNextRecord(this.reuseElement); + T tmp = getNextRecord(); if (tmp != null) { this.nextElement = tmp; return true; diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/resettable/ReusingBlockResettableIterator.java b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/resettable/ReusingBlockResettableIterator.java new file mode 100644 index 0000000000000..baa0fb2b4357f --- /dev/null +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/resettable/ReusingBlockResettableIterator.java @@ -0,0 +1,100 @@ +/* + * 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.flink.runtime.operators.resettable; + +import org.apache.flink.api.common.typeutils.TypeSerializer; +import org.apache.flink.runtime.jobgraph.tasks.AbstractInvokable; +import org.apache.flink.runtime.memorymanager.MemoryAllocationException; +import org.apache.flink.runtime.memorymanager.MemoryManager; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.IOException; +import java.util.Iterator; + +/** + * Implementation of an iterator that fetches a block of data into main memory and offers resettable + * access to the data in that block. + * + */ +public class ReusingBlockResettableIterator extends NonReusingBlockResettableIterator { + + public static final Logger LOG = LoggerFactory.getLogger(ReusingBlockResettableIterator.class); + + private final T reuseElement; + + // ------------------------------------------------------------------------ + + public ReusingBlockResettableIterator(MemoryManager memoryManager, Iterator input, + TypeSerializer serializer, int numPages, + AbstractInvokable ownerTask) + throws MemoryAllocationException + { + this(memoryManager, serializer, numPages, ownerTask); + this.input = input; + } + + public ReusingBlockResettableIterator(MemoryManager memoryManager, TypeSerializer + serializer, int numPages, AbstractInvokable ownerTask) + throws MemoryAllocationException + { + super(memoryManager, serializer, numPages, ownerTask); + + this.reuseElement = serializer.createInstance(); + } + + // ------------------------------------------------------------------------ + + @Override + public boolean hasNext() { + try { + if (this.nextElement == null) { + if (this.readPhase) { + // read phase, get next element from buffer + T tmp = getNextRecord(this.reuseElement); + if (tmp != null) { + this.nextElement = tmp; + return true; + } else { + return false; + } + } else { + if (this.input.hasNext()) { + final T next = this.input.next(); + if (writeNextRecord(next)) { + this.nextElement = next; + return true; + } else { + this.leftOverElement = next; + return false; + } + } else { + this.noMoreBlocks = true; + return false; + } + } + } else { + return true; + } + } catch (IOException ioex) { + throw new RuntimeException("Error (de)serializing record in block resettable iterator.", ioex); + } + } +} diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/sort/NonReusingMergeMatchIterator.java b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/sort/NonReusingMergeMatchIterator.java new file mode 100644 index 0000000000000..70b6f9a0e788b --- /dev/null +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/sort/NonReusingMergeMatchIterator.java @@ -0,0 +1,424 @@ +/* + * 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.flink.runtime.operators.sort; + +import org.apache.flink.api.common.functions.FlatJoinFunction; +import org.apache.flink.api.common.typeutils.TypeComparator; +import org.apache.flink.api.common.typeutils.TypePairComparator; +import org.apache.flink.api.common.typeutils.TypeSerializer; +import org.apache.flink.core.memory.MemorySegment; +import org.apache.flink.runtime.io.disk.iomanager.IOManager; +import org.apache.flink.runtime.jobgraph.tasks.AbstractInvokable; +import org.apache.flink.runtime.memorymanager.MemoryAllocationException; +import org.apache.flink.runtime.memorymanager.MemoryManager; +import org.apache.flink.runtime.operators.resettable.NonReusingBlockResettableIterator; +import org.apache.flink.runtime.operators.resettable.SpillingResettableIterator; +import org.apache.flink.runtime.operators.util.JoinTaskIterator; +import org.apache.flink.runtime.util.NonReusingKeyGroupedIterator; +import org.apache.flink.util.Collector; +import org.apache.flink.util.MutableObjectIterator; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.IOException; +import java.util.Iterator; +import java.util.List; + + +/** + * An implementation of the {@link org.apache.flink.runtime.operators.util.JoinTaskIterator} that realizes the + * matching through a sort-merge join strategy. + */ +public class NonReusingMergeMatchIterator implements JoinTaskIterator { + + /** + * The log used by this iterator to log messages. + */ + private static final Logger LOG = LoggerFactory.getLogger(NonReusingMergeMatchIterator.class); + + // -------------------------------------------------------------------------------------------- + + private TypePairComparator comp; + + private NonReusingKeyGroupedIterator iterator1; + + private NonReusingKeyGroupedIterator iterator2; + + private final TypeSerializer serializer1; + + private final TypeSerializer serializer2; + + private final NonReusingBlockResettableIterator blockIt; // for N:M cross products with same key + + private final List memoryForSpillingIterator; + + private final MemoryManager memoryManager; + + private final IOManager ioManager; + + // -------------------------------------------------------------------------------------------- + + public NonReusingMergeMatchIterator( + MutableObjectIterator input1, + MutableObjectIterator input2, + TypeSerializer serializer1, TypeComparator comparator1, + TypeSerializer serializer2, TypeComparator comparator2, + TypePairComparator pairComparator, + MemoryManager memoryManager, + IOManager ioManager, + int numMemoryPages, + AbstractInvokable parentTask) + throws MemoryAllocationException + { + if (numMemoryPages < 2) { + throw new IllegalArgumentException("Merger needs at least 2 memory pages."); + } + + this.comp = pairComparator; + this.serializer1 = serializer1; + this.serializer2 = serializer2; + + this.memoryManager = memoryManager; + this.ioManager = ioManager; + + this.iterator1 = new NonReusingKeyGroupedIterator(input1, this.serializer1, comparator1.duplicate()); + this.iterator2 = new NonReusingKeyGroupedIterator(input2, this.serializer2, comparator2.duplicate()); + + final int numPagesForSpiller = numMemoryPages > 20 ? 2 : 1; + this.blockIt = new NonReusingBlockResettableIterator(this.memoryManager, this.serializer2, + (numMemoryPages - numPagesForSpiller), parentTask); + this.memoryForSpillingIterator = memoryManager.allocatePages(parentTask, numPagesForSpiller); + } + + + @Override + public void open() throws IOException {} + + + @Override + public void close() { + if (this.blockIt != null) { + try { + this.blockIt.close(); + } + catch (Throwable t) { + LOG.error("Error closing block memory iterator: " + t.getMessage(), t); + } + } + + this.memoryManager.release(this.memoryForSpillingIterator); + } + + + @Override + public void abort() { + close(); + } + + /** + * Calls the JoinFunction#match() method for all two key-value pairs that share the same key and come + * from different inputs. The output of the match() method is forwarded. + *

+ * This method first zig-zags between the two sorted inputs in order to find a common + * key, and then calls the match stub with the cross product of the values. + * + * @throws Exception Forwards all exceptions from the user code and the I/O system. + * + * @see org.apache.flink.runtime.operators.util.JoinTaskIterator#callWithNextKey(org.apache.flink.api.common.functions.FlatJoinFunction, org.apache.flink.util.Collector) + */ + @Override + public boolean callWithNextKey(final FlatJoinFunction matchFunction, final Collector collector) + throws Exception + { + if (!this.iterator1.nextKey() || !this.iterator2.nextKey()) { + // consume all remaining keys (hack to prevent remaining inputs during iterations, lets get rid of this soon) + while (this.iterator1.nextKey()); + while (this.iterator2.nextKey()); + + return false; + } + + final TypePairComparator comparator = this.comp; + comparator.setReference(this.iterator1.getCurrent()); + T2 current2 = this.iterator2.getCurrent(); + + // zig zag + while (true) { + // determine the relation between the (possibly composite) keys + final int comp = comparator.compareToReference(current2); + + if (comp == 0) { + break; + } + + if (comp < 0) { + if (!this.iterator2.nextKey()) { + return false; + } + current2 = this.iterator2.getCurrent(); + } + else { + if (!this.iterator1.nextKey()) { + return false; + } + comparator.setReference(this.iterator1.getCurrent()); + } + } + + // here, we have a common key! call the match function with the cross product of the + // values + final NonReusingKeyGroupedIterator.ValuesIterator values1 = this.iterator1.getValues(); + final NonReusingKeyGroupedIterator.ValuesIterator values2 = this.iterator2.getValues(); + + final T1 firstV1 = values1.next(); + final T2 firstV2 = values2.next(); + + final boolean v1HasNext = values1.hasNext(); + final boolean v2HasNext = values2.hasNext(); + + // check if one side is already empty + // this check could be omitted if we put this in MatchTask. + // then we can derive the local strategy (with build side). + + if (v1HasNext) { + if (v2HasNext) { + // both sides contain more than one value + // TODO: Decide which side to spill and which to block! + crossMwithNValues(firstV1, values1, firstV2, values2, matchFunction, collector); + } else { + crossSecond1withNValues(firstV2, firstV1, values1, matchFunction, collector); + } + } else { + if (v2HasNext) { + crossFirst1withNValues(firstV1, firstV2, values2, matchFunction, collector); + } else { + // both sides contain only one value + matchFunction.join(firstV1, firstV2, collector); + } + } + return true; + } + + /** + * Crosses a single value from the first input with N values, all sharing a common key. + * Effectively realizes a 1:N match (join). + * + * @param val1 The value form the 1 side. + * @param firstValN The first of the values from the N side. + * @param valsN Iterator over remaining N side values. + * + * @throws Exception Forwards all exceptions thrown by the stub. + */ + private void crossFirst1withNValues(final T1 val1, final T2 firstValN, + final Iterator valsN, final FlatJoinFunction matchFunction, final Collector collector) + throws Exception + { + T1 copy1 = this.serializer1.copy(val1); + matchFunction.join(copy1, firstValN, collector); + + // set copy and match first element + boolean more = true; + do { + final T2 nRec = valsN.next(); + + if (valsN.hasNext()) { + copy1 = this.serializer1.copy(val1); + matchFunction.join(copy1, nRec, collector); + } else { + matchFunction.join(val1, nRec, collector); + more = false; + } + } + while (more); + } + + /** + * Crosses a single value from the second side with N values, all sharing a common key. + * Effectively realizes a N:1 match (join). + * + * @param val1 The value form the 1 side. + * @param firstValN The first of the values from the N side. + * @param valsN Iterator over remaining N side values. + * + * @throws Exception Forwards all exceptions thrown by the stub. + */ + private void crossSecond1withNValues(T2 val1, T1 firstValN, + Iterator valsN, FlatJoinFunction matchFunction, Collector collector) + throws Exception + { + T2 copy2 = this.serializer2.copy(val1); + matchFunction.join(firstValN, copy2, collector); + + // set copy and match first element + boolean more = true; + do { + final T1 nRec = valsN.next(); + + if (valsN.hasNext()) { + copy2 = this.serializer2.copy(val1); + matchFunction.join(nRec, copy2, collector); + } else { + matchFunction.join(nRec, val1, collector); + more = false; + } + } + while (more); + } + + /** + * @param firstV1 + * @param spillVals + * @param firstV2 + * @param blockVals + */ + private void crossMwithNValues(final T1 firstV1, Iterator spillVals, + final T2 firstV2, final Iterator blockVals, + final FlatJoinFunction matchFunction, final Collector collector) + throws Exception + { + // ================================================== + // We have one first (head) element from both inputs (firstV1 and firstV2) + // We have an iterator for both inputs. + // we make the V1 side the spilling side and the V2 side the blocking side. + // In order to get the full cross product without unnecessary spilling, we do the + // following: + // 1) cross the heads + // 2) cross the head of the spilling side against the first block of the blocking side + // 3) cross the iterator of the spilling side with the head of the block side + // 4) cross the iterator of the spilling side with the first block + // --------------------------------------------------- + // If the blocking side has more than one block, we really need to make the spilling side fully + // resettable. For each further block on the block side, we do: + // 5) cross the head of the spilling side with the next block + // 6) cross the spilling iterator with the next block. + + // match the first values first + T1 copy1 = this.serializer1.copy(firstV1); + T2 blockHeadCopy = this.serializer2.copy(firstV2); + T1 spillHeadCopy = null; + + + // --------------- 1) Cross the heads ------------------- + matchFunction.join(copy1, firstV2, collector); + + // for the remaining values, we do a block-nested-loops join + SpillingResettableIterator spillIt = null; + + try { + // create block iterator on the second input + this.blockIt.reopen(blockVals); + + // ------------- 2) cross the head of the spilling side with the first block ------------------ + while (this.blockIt.hasNext()) { + final T2 nextBlockRec = this.blockIt.next(); + copy1 = this.serializer1.copy(firstV1); + matchFunction.join(copy1, nextBlockRec, collector); + } + this.blockIt.reset(); + + // spilling is required if the blocked input has data beyond the current block. + // in that case, create the spilling iterator + final Iterator leftSideIter; + final boolean spillingRequired = this.blockIt.hasFurtherInput(); + if (spillingRequired) + { + // more data than would fit into one block. we need to wrap the other side in a spilling iterator + // create spilling iterator on first input + spillIt = new SpillingResettableIterator(spillVals, this.serializer1, + this.memoryManager, this.ioManager, this.memoryForSpillingIterator); + leftSideIter = spillIt; + spillIt.open(); + + spillHeadCopy = this.serializer1.copy(firstV1); + } + else { + leftSideIter = spillVals; + } + + // cross the values in the v1 iterator against the current block + + while (leftSideIter.hasNext()) { + final T1 nextSpillVal = leftSideIter.next(); + copy1 = this.serializer1.copy(nextSpillVal); + + + // -------- 3) cross the iterator of the spilling side with the head of the block side -------- + T2 copy2 = this.serializer2.copy(blockHeadCopy); + matchFunction.join(copy1, copy2, collector); + + // -------- 4) cross the iterator of the spilling side with the first block -------- + while (this.blockIt.hasNext()) { + T2 nextBlockRec = this.blockIt.next(); + + // get instances of key and block value + copy1 = this.serializer1.copy(nextSpillVal); + matchFunction.join(copy1, nextBlockRec, collector); + } + // reset block iterator + this.blockIt.reset(); + } + + // if everything from the block-side fit into a single block, we are done. + // note that in this special case, we did not create a spilling iterator at all + if (!spillingRequired) { + return; + } + + // here we are, because we have more blocks on the block side + // loop as long as there are blocks from the blocked input + while (this.blockIt.nextBlock()) + { + // rewind the spilling iterator + spillIt.reset(); + + // ------------- 5) cross the head of the spilling side with the next block ------------ + while (this.blockIt.hasNext()) { + copy1 = this.serializer1.copy(spillHeadCopy); + final T2 nextBlockVal = blockIt.next(); + matchFunction.join(copy1, nextBlockVal, collector); + } + this.blockIt.reset(); + + // -------- 6) cross the spilling iterator with the next block. ------------------ + while (spillIt.hasNext()) + { + // get value from resettable iterator + final T1 nextSpillVal = spillIt.next(); + // cross value with block values + while (this.blockIt.hasNext()) { + // get instances of key and block value + final T2 nextBlockVal = this.blockIt.next(); + copy1 = this.serializer1.copy(nextSpillVal); + matchFunction.join(copy1, nextBlockVal, collector); + } + + // reset block iterator + this.blockIt.reset(); + } + // reset v1 iterator + spillIt.reset(); + } + } + finally { + if (spillIt != null) { + this.memoryForSpillingIterator.addAll(spillIt.close()); + } + } + } +} diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/sort/MergeMatchIterator.java b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/sort/ReusingMergeMatchIterator.java similarity index 95% rename from flink-runtime/src/main/java/org/apache/flink/runtime/operators/sort/MergeMatchIterator.java rename to flink-runtime/src/main/java/org/apache/flink/runtime/operators/sort/ReusingMergeMatchIterator.java index 675758aa13e0a..66beee1f807fb 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/sort/MergeMatchIterator.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/sort/ReusingMergeMatchIterator.java @@ -33,7 +33,7 @@ import org.apache.flink.runtime.jobgraph.tasks.AbstractInvokable; import org.apache.flink.runtime.memorymanager.MemoryAllocationException; import org.apache.flink.runtime.memorymanager.MemoryManager; -import org.apache.flink.runtime.operators.resettable.BlockResettableIterator; +import org.apache.flink.runtime.operators.resettable.NonReusingBlockResettableIterator; import org.apache.flink.runtime.operators.resettable.SpillingResettableIterator; import org.apache.flink.runtime.operators.util.JoinTaskIterator; import org.apache.flink.runtime.util.ReusingKeyGroupedIterator; @@ -45,12 +45,12 @@ * An implementation of the {@link JoinTaskIterator} that realizes the * matching through a sort-merge join strategy. */ -public class MergeMatchIterator implements JoinTaskIterator { +public class ReusingMergeMatchIterator implements JoinTaskIterator { /** * The log used by this iterator to log messages. */ - private static final Logger LOG = LoggerFactory.getLogger(MergeMatchIterator.class); + private static final Logger LOG = LoggerFactory.getLogger(ReusingMergeMatchIterator.class); // -------------------------------------------------------------------------------------------- @@ -72,7 +72,7 @@ public class MergeMatchIterator implements JoinTaskIterator blockIt; // for N:M cross products with same key + private final NonReusingBlockResettableIterator blockIt; // for N:M cross products with same key private final List memoryForSpillingIterator; @@ -82,10 +82,16 @@ public class MergeMatchIterator implements JoinTaskIterator input1, MutableObjectIterator input2, + public ReusingMergeMatchIterator( + MutableObjectIterator input1, + MutableObjectIterator input2, TypeSerializer serializer1, TypeComparator comparator1, - TypeSerializer serializer2, TypeComparator comparator2, TypePairComparator pairComparator, - MemoryManager memoryManager, IOManager ioManager, int numMemoryPages, AbstractInvokable parentTask) + TypeSerializer serializer2, TypeComparator comparator2, + TypePairComparator pairComparator, + MemoryManager memoryManager, + IOManager ioManager, + int numMemoryPages, + AbstractInvokable parentTask) throws MemoryAllocationException { if (numMemoryPages < 2) { @@ -108,7 +114,7 @@ public MergeMatchIterator(MutableObjectIterator input1, MutableObjectIterato this.iterator2 = new ReusingKeyGroupedIterator(input2, this.serializer2, comparator2.duplicate()); final int numPagesForSpiller = numMemoryPages > 20 ? 2 : 1; - this.blockIt = new BlockResettableIterator(this.memoryManager, this.serializer2, + this.blockIt = new NonReusingBlockResettableIterator(this.memoryManager, this.serializer2, (numMemoryPages - numPagesForSpiller), parentTask); this.memoryForSpillingIterator = memoryManager.allocatePages(parentTask, numPagesForSpiller); } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/resettable/BlockResettableIteratorTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/resettable/NonReusingBlockResettableIteratorTest.java similarity index 93% rename from flink-runtime/src/test/java/org/apache/flink/runtime/operators/resettable/BlockResettableIteratorTest.java rename to flink-runtime/src/test/java/org/apache/flink/runtime/operators/resettable/NonReusingBlockResettableIteratorTest.java index c51e53aa9e08c..5641f29c85910 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/resettable/BlockResettableIteratorTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/resettable/NonReusingBlockResettableIteratorTest.java @@ -29,7 +29,6 @@ import org.apache.flink.runtime.jobgraph.tasks.AbstractInvokable; import org.apache.flink.runtime.memorymanager.DefaultMemoryManager; import org.apache.flink.runtime.memorymanager.MemoryManager; -import org.apache.flink.runtime.operators.resettable.BlockResettableIterator; import org.apache.flink.runtime.operators.testutils.DummyInvokable; import org.apache.flink.types.IntValue; import org.apache.flink.types.Record; @@ -38,7 +37,7 @@ import org.junit.Test; -public class BlockResettableIteratorTest +public class NonReusingBlockResettableIteratorTest { private static final int MEMORY_CAPACITY = 3 * 128 * 1024; @@ -85,7 +84,7 @@ public void testSerialBlockResettableIterator() throws Exception { final AbstractInvokable memOwner = new DummyInvokable(); // create the resettable Iterator - final BlockResettableIterator iterator = new BlockResettableIterator( + final NonReusingBlockResettableIterator iterator = new NonReusingBlockResettableIterator( this.memman, this.reader, this.serializer, 1, memOwner); // open the iterator iterator.open(); @@ -124,7 +123,7 @@ public void testDoubleBufferedBlockResettableIterator() throws Exception { final AbstractInvokable memOwner = new DummyInvokable(); // create the resettable Iterator - final BlockResettableIterator iterator = new BlockResettableIterator( + final NonReusingBlockResettableIterator iterator = new NonReusingBlockResettableIterator( this.memman, this.reader, this.serializer, 2, memOwner); // open the iterator iterator.open(); @@ -164,7 +163,7 @@ public void testTwelveFoldBufferedBlockResettableIterator() throws Exception { final AbstractInvokable memOwner = new DummyInvokable(); // create the resettable Iterator - final BlockResettableIterator iterator = new BlockResettableIterator( + final NonReusingBlockResettableIterator iterator = new NonReusingBlockResettableIterator( this.memman, this.reader, this.serializer, 12, memOwner); // open the iterator iterator.open(); diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/resettable/ReusingBlockResettableIteratorTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/resettable/ReusingBlockResettableIteratorTest.java new file mode 100644 index 0000000000000..f0fadfe23b29f --- /dev/null +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/resettable/ReusingBlockResettableIteratorTest.java @@ -0,0 +1,201 @@ +/* + * 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.flink.runtime.operators.resettable; + +import org.apache.flink.api.common.typeutils.TypeSerializer; +import org.apache.flink.api.common.typeutils.record.RecordSerializer; +import org.apache.flink.runtime.jobgraph.tasks.AbstractInvokable; +import org.apache.flink.runtime.memorymanager.DefaultMemoryManager; +import org.apache.flink.runtime.memorymanager.MemoryManager; +import org.apache.flink.runtime.operators.testutils.DummyInvokable; +import org.apache.flink.types.IntValue; +import org.apache.flink.types.Record; +import org.junit.After; +import org.junit.Assert; +import org.junit.Before; +import org.junit.Test; + +import java.util.ArrayList; +import java.util.Iterator; +import java.util.List; + + +public class ReusingBlockResettableIteratorTest +{ + private static final int MEMORY_CAPACITY = 3 * 128 * 1024; + + private static final int NUM_VALUES = 20000; + + private MemoryManager memman; + + private Iterator reader; + + private List objects; + + private final TypeSerializer serializer = RecordSerializer.get(); + + @Before + public void startup() { + // set up IO and memory manager + this.memman = new DefaultMemoryManager(MEMORY_CAPACITY, 1); + + // create test objects + this.objects = new ArrayList(20000); + for (int i = 0; i < NUM_VALUES; ++i) { + this.objects.add(new Record(new IntValue(i))); + } + + // create the reader + this.reader = objects.iterator(); + } + + @After + public void shutdown() { + this.objects = null; + + // check that the memory manager got all segments back + if (!this.memman.verifyEmpty()) { + Assert.fail("A memory leak has occurred: Not all memory was properly returned to the memory manager."); + } + + this.memman.shutdown(); + this.memman = null; + } + + @Test + public void testSerialBlockResettableIterator() throws Exception + { + final AbstractInvokable memOwner = new DummyInvokable(); + // create the resettable Iterator + final ReusingBlockResettableIterator iterator = new ReusingBlockResettableIterator( + this.memman, this.reader, this.serializer, 1, memOwner); + // open the iterator + iterator.open(); + + // now test walking through the iterator + int lower = 0; + int upper = 0; + do { + lower = upper; + upper = lower; + // find the upper bound + while (iterator.hasNext()) { + Record target = iterator.next(); + int val = target.getField(0, IntValue.class).getValue(); + Assert.assertEquals(upper++, val); + } + // now reset the buffer a few times + for (int i = 0; i < 5; ++i) { + iterator.reset(); + int count = 0; + while (iterator.hasNext()) { + Record target = iterator.next(); + int val = target.getField(0, IntValue.class).getValue(); + Assert.assertEquals(lower + (count++), val); + } + Assert.assertEquals(upper - lower, count); + } + } while (iterator.nextBlock()); + Assert.assertEquals(NUM_VALUES, upper); + // close the iterator + iterator.close(); + } + + @Test + public void testDoubleBufferedBlockResettableIterator() throws Exception + { + final AbstractInvokable memOwner = new DummyInvokable(); + // create the resettable Iterator + final ReusingBlockResettableIterator iterator = new ReusingBlockResettableIterator( + this.memman, this.reader, this.serializer, 2, memOwner); + // open the iterator + iterator.open(); + + // now test walking through the iterator + int lower = 0; + int upper = 0; + do { + lower = upper; + upper = lower; + // find the upper bound + while (iterator.hasNext()) { + Record target = iterator.next(); + int val = target.getField(0, IntValue.class).getValue(); + Assert.assertEquals(upper++, val); + } + // now reset the buffer a few times + for (int i = 0; i < 5; ++i) { + iterator.reset(); + int count = 0; + while (iterator.hasNext()) { + Record target = iterator.next(); + int val = target.getField(0, IntValue.class).getValue(); + Assert.assertEquals(lower + (count++), val); + } + Assert.assertEquals(upper - lower, count); + } + } while (iterator.nextBlock()); + Assert.assertEquals(NUM_VALUES, upper); + + // close the iterator + iterator.close(); + } + + @Test + public void testTwelveFoldBufferedBlockResettableIterator() throws Exception + { + final AbstractInvokable memOwner = new DummyInvokable(); + // create the resettable Iterator + final ReusingBlockResettableIterator iterator = new ReusingBlockResettableIterator( + this.memman, this.reader, this.serializer, 12, memOwner); + // open the iterator + iterator.open(); + + // now test walking through the iterator + int lower = 0; + int upper = 0; + do { + lower = upper; + upper = lower; + // find the upper bound + while (iterator.hasNext()) { + Record target = iterator.next(); + int val = target.getField(0, IntValue.class).getValue(); + Assert.assertEquals(upper++, val); + } + // now reset the buffer a few times + for (int i = 0; i < 5; ++i) { + iterator.reset(); + int count = 0; + while (iterator.hasNext()) { + Record target = iterator.next(); + int val = target.getField(0, IntValue.class).getValue(); + Assert.assertEquals(lower + (count++), val); + } + Assert.assertEquals(upper - lower, count); + } + } while (iterator.nextBlock()); + Assert.assertEquals(NUM_VALUES, upper); + + // close the iterator + iterator.close(); + } + +} diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/sort/SortMergeMatchIteratorITCase.java b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/sort/NonReusingSortMergeMatchIteratorITCase.java similarity index 97% rename from flink-runtime/src/test/java/org/apache/flink/runtime/operators/sort/SortMergeMatchIteratorITCase.java rename to flink-runtime/src/test/java/org/apache/flink/runtime/operators/sort/NonReusingSortMergeMatchIteratorITCase.java index e12c4ac6427d3..e873416b2f99f 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/sort/SortMergeMatchIteratorITCase.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/sort/NonReusingSortMergeMatchIteratorITCase.java @@ -37,8 +37,6 @@ import org.apache.flink.runtime.jobgraph.tasks.AbstractInvokable; import org.apache.flink.runtime.memorymanager.DefaultMemoryManager; import org.apache.flink.runtime.memorymanager.MemoryManager; -import org.apache.flink.runtime.operators.sort.MergeIterator; -import org.apache.flink.runtime.operators.sort.MergeMatchIterator; import org.apache.flink.runtime.operators.testutils.DiscardingOutputCollector; import org.apache.flink.runtime.operators.testutils.DummyInvokable; import org.apache.flink.runtime.operators.testutils.TestData; @@ -55,7 +53,7 @@ import org.junit.Test; @SuppressWarnings("deprecation") -public class SortMergeMatchIteratorITCase { +public class NonReusingSortMergeMatchIteratorITCase { // total memory private static final int MEMORY_SIZE = 1024 * 1024 * 16; @@ -143,8 +141,8 @@ public void testMerge() { input2.reset(); // compare with iterator values - MergeMatchIterator iterator = - new MergeMatchIterator( + NonReusingMergeMatchIterator iterator = + new NonReusingMergeMatchIterator( input1, input2, this.serializer1, this.comparator1, this.serializer2, this.comparator2, this.pairComparator, this.memoryManager, this.ioManager, PAGES_FOR_BNLJN, this.parentTask); @@ -230,8 +228,8 @@ public void testMergeWithHighNumberOfCommonKeys() // we create this sort-merge iterator with little memory for the block-nested-loops fall-back to make sure it // needs to spill for the duplicate keys - MergeMatchIterator iterator = - new MergeMatchIterator( + NonReusingMergeMatchIterator iterator = + new NonReusingMergeMatchIterator( input1, input2, this.serializer1, this.comparator1, this.serializer2, this.comparator2, this.pairComparator, this.memoryManager, this.ioManager, PAGES_FOR_BNLJN, this.parentTask); diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/sort/ReusingSortMergeMatchIteratorITCase.java b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/sort/ReusingSortMergeMatchIteratorITCase.java new file mode 100644 index 0000000000000..dd7248d65ef39 --- /dev/null +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/sort/ReusingSortMergeMatchIteratorITCase.java @@ -0,0 +1,371 @@ +/* + * 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.flink.runtime.operators.sort; + +import org.apache.flink.api.common.typeutils.TypeComparator; +import org.apache.flink.api.common.typeutils.TypePairComparator; +import org.apache.flink.api.common.typeutils.TypeSerializer; +import org.apache.flink.api.common.typeutils.record.RecordComparator; +import org.apache.flink.api.common.typeutils.record.RecordPairComparator; +import org.apache.flink.api.common.typeutils.record.RecordSerializer; +import org.apache.flink.api.java.record.functions.JoinFunction; +import org.apache.flink.runtime.io.disk.iomanager.IOManager; +import org.apache.flink.runtime.io.disk.iomanager.IOManagerAsync; +import org.apache.flink.runtime.jobgraph.tasks.AbstractInvokable; +import org.apache.flink.runtime.memorymanager.DefaultMemoryManager; +import org.apache.flink.runtime.memorymanager.MemoryManager; +import org.apache.flink.runtime.operators.testutils.DiscardingOutputCollector; +import org.apache.flink.runtime.operators.testutils.DummyInvokable; +import org.apache.flink.runtime.operators.testutils.TestData; +import org.apache.flink.runtime.operators.testutils.TestData.Generator; +import org.apache.flink.runtime.operators.testutils.TestData.Generator.KeyMode; +import org.apache.flink.runtime.operators.testutils.TestData.Generator.ValueMode; +import org.apache.flink.types.Record; +import org.apache.flink.types.Value; +import org.apache.flink.util.Collector; +import org.apache.flink.util.MutableObjectIterator; +import org.junit.After; +import org.junit.Assert; +import org.junit.Before; +import org.junit.Test; + +import java.util.ArrayList; +import java.util.Collection; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.Map.Entry; + +@SuppressWarnings("deprecation") +public class ReusingSortMergeMatchIteratorITCase { + + // total memory + private static final int MEMORY_SIZE = 1024 * 1024 * 16; + private static final int PAGES_FOR_BNLJN = 2; + + // the size of the left and right inputs + private static final int INPUT_1_SIZE = 20000; + + private static final int INPUT_2_SIZE = 1000; + + // random seeds for the left and right input data generators + private static final long SEED1 = 561349061987311L; + + private static final long SEED2 = 231434613412342L; + + // dummy abstract task + private final AbstractInvokable parentTask = new DummyInvokable(); + + private IOManager ioManager; + private MemoryManager memoryManager; + + private TypeSerializer serializer1; + private TypeSerializer serializer2; + private TypeComparator comparator1; + private TypeComparator comparator2; + private TypePairComparator pairComparator; + + + @SuppressWarnings("unchecked") + @Before + public void beforeTest() { + this.serializer1 = RecordSerializer.get(); + this.serializer2 = RecordSerializer.get(); + this.comparator1 = new RecordComparator(new int[] {0}, new Class[]{TestData.Key.class}); + this.comparator2 = new RecordComparator(new int[] {0}, new Class[]{TestData.Key.class}); + this.pairComparator = new RecordPairComparator(new int[] {0}, new int[] {0}, new Class[]{TestData.Key.class}); + + this.memoryManager = new DefaultMemoryManager(MEMORY_SIZE, 1); + this.ioManager = new IOManagerAsync(); + } + + @After + public void afterTest() { + if (this.ioManager != null) { + this.ioManager.shutdown(); + if (!this.ioManager.isProperlyShutDown()) { + Assert.fail("I/O manager failed to properly shut down."); + } + this.ioManager = null; + } + + if (this.memoryManager != null) { + Assert.assertTrue("Memory Leak: Not all memory has been returned to the memory manager.", + this.memoryManager.verifyEmpty()); + this.memoryManager.shutdown(); + this.memoryManager = null; + } + } + + + + @Test + public void testMerge() { + try { + + final Generator generator1 = new Generator(SEED1, 500, 4096, KeyMode.SORTED, ValueMode.RANDOM_LENGTH); + final Generator generator2 = new Generator(SEED2, 500, 2048, KeyMode.SORTED, ValueMode.RANDOM_LENGTH); + + final TestData.GeneratorIterator input1 = new TestData.GeneratorIterator(generator1, INPUT_1_SIZE); + final TestData.GeneratorIterator input2 = new TestData.GeneratorIterator(generator2, INPUT_2_SIZE); + + // collect expected data + final Map> expectedMatchesMap = matchValues( + collectData(input1), + collectData(input2)); + + final JoinFunction matcher = new MatchRemovingMatcher(expectedMatchesMap); + + final Collector collector = new DiscardingOutputCollector(); + + // reset the generators + generator1.reset(); + generator2.reset(); + input1.reset(); + input2.reset(); + + // compare with iterator values + ReusingMergeMatchIterator iterator = + new ReusingMergeMatchIterator( + input1, input2, this.serializer1, this.comparator1, this.serializer2, this.comparator2, + this.pairComparator, this.memoryManager, this.ioManager, PAGES_FOR_BNLJN, this.parentTask); + + iterator.open(); + + while (iterator.callWithNextKey(matcher, collector)); + + iterator.close(); + + // assert that each expected match was seen + for (Entry> entry : expectedMatchesMap.entrySet()) { + Assert.assertTrue("Collection for key " + entry.getKey() + " is not empty", entry.getValue().isEmpty()); + } + } + catch (Exception e) { + e.printStackTrace(); + Assert.fail("An exception occurred during the test: " + e.getMessage()); + } + } + + @Test + public void testMergeWithHighNumberOfCommonKeys() + { + // the size of the left and right inputs + final int INPUT_1_SIZE = 200; + final int INPUT_2_SIZE = 100; + + final int INPUT_1_DUPLICATES = 10; + final int INPUT_2_DUPLICATES = 4000; + final int DUPLICATE_KEY = 13; + + try { + final Generator generator1 = new Generator(SEED1, 500, 4096, KeyMode.SORTED, ValueMode.RANDOM_LENGTH); + final Generator generator2 = new Generator(SEED2, 500, 2048, KeyMode.SORTED, ValueMode.RANDOM_LENGTH); + + final TestData.GeneratorIterator gen1Iter = new TestData.GeneratorIterator(generator1, INPUT_1_SIZE); + final TestData.GeneratorIterator gen2Iter = new TestData.GeneratorIterator(generator2, INPUT_2_SIZE); + + final TestData.ConstantValueIterator const1Iter = new TestData.ConstantValueIterator(DUPLICATE_KEY, "LEFT String for Duplicate Keys", INPUT_1_DUPLICATES); + final TestData.ConstantValueIterator const2Iter = new TestData.ConstantValueIterator(DUPLICATE_KEY, "RIGHT String for Duplicate Keys", INPUT_2_DUPLICATES); + + final List> inList1 = new ArrayList>(); + inList1.add(gen1Iter); + inList1.add(const1Iter); + + final List> inList2 = new ArrayList>(); + inList2.add(gen2Iter); + inList2.add(const2Iter); + + MutableObjectIterator input1 = new MergeIterator(inList1, serializer1, comparator1.duplicate()); + MutableObjectIterator input2 = new MergeIterator(inList2, serializer2, comparator2.duplicate()); + + // collect expected data + final Map> expectedMatchesMap = matchValues( + collectData(input1), + collectData(input2)); + + // re-create the whole thing for actual processing + + // reset the generators and iterators + generator1.reset(); + generator2.reset(); + const1Iter.reset(); + const2Iter.reset(); + gen1Iter.reset(); + gen2Iter.reset(); + + inList1.clear(); + inList1.add(gen1Iter); + inList1.add(const1Iter); + + inList2.clear(); + inList2.add(gen2Iter); + inList2.add(const2Iter); + + input1 = new MergeIterator(inList1, serializer1, comparator1.duplicate()); + input2 = new MergeIterator(inList2, serializer2, comparator2.duplicate()); + + final JoinFunction matcher = new MatchRemovingMatcher(expectedMatchesMap); + + final Collector collector = new DiscardingOutputCollector(); + + + // we create this sort-merge iterator with little memory for the block-nested-loops fall-back to make sure it + // needs to spill for the duplicate keys + ReusingMergeMatchIterator iterator = + new ReusingMergeMatchIterator( + input1, input2, this.serializer1, this.comparator1, this.serializer2, this.comparator2, + this.pairComparator, this.memoryManager, this.ioManager, PAGES_FOR_BNLJN, this.parentTask); + + iterator.open(); + + while (iterator.callWithNextKey(matcher, collector)); + + iterator.close(); + + // assert that each expected match was seen + for (Entry> entry : expectedMatchesMap.entrySet()) { + if (!entry.getValue().isEmpty()) { + Assert.fail("Collection for key " + entry.getKey() + " is not empty"); + } + } + } + catch (Exception e) { + e.printStackTrace(); + Assert.fail("An exception occurred during the test: " + e.getMessage()); + } + } + + + + // -------------------------------------------------------------------------------------------- + // Utilities + // -------------------------------------------------------------------------------------------- + + private Map> matchValues( + Map> leftMap, + Map> rightMap) + { + Map> map = new HashMap>(); + + for (TestData.Key key : leftMap.keySet()) { + Collection leftValues = leftMap.get(key); + Collection rightValues = rightMap.get(key); + + if (rightValues == null) { + continue; + } + + if (!map.containsKey(key)) { + map.put(key, new ArrayList()); + } + + Collection matchedValues = map.get(key); + + for (TestData.Value leftValue : leftValues) { + for (TestData.Value rightValue : rightValues) { + matchedValues.add(new Match(leftValue, rightValue)); + } + } + } + + return map; + } + + + private Map> collectData(MutableObjectIterator iter) + throws Exception + { + Map> map = new HashMap>(); + Record pair = new Record(); + + while ((pair = iter.next(pair)) != null) { + TestData.Key key = pair.getField(0, TestData.Key.class); + + if (!map.containsKey(key)) { + map.put(new TestData.Key(key.getKey()), new ArrayList()); + } + + Collection values = map.get(key); + values.add(new TestData.Value(pair.getField(1, TestData.Value.class).getValue())); + } + + return map; + } + + /** + * Private class used for storage of the expected matches in a hashmap. + */ + private static class Match { + private final Value left; + + private final Value right; + + public Match(Value left, Value right) { + this.left = left; + this.right = right; + } + + @Override + public boolean equals(Object obj) { + Match o = (Match) obj; + return this.left.equals(o.left) && this.right.equals(o.right); + } + + @Override + public int hashCode() { + return this.left.hashCode() ^ this.right.hashCode(); + } + + @Override + public String toString() { + return left + ", " + right; + } + } + + private static final class MatchRemovingMatcher extends JoinFunction { + private static final long serialVersionUID = 1L; + + private final Map> toRemoveFrom; + + protected MatchRemovingMatcher(Map> map) { + this.toRemoveFrom = map; + } + + @Override + public void join(Record rec1, Record rec2, Collector out) throws Exception { + TestData.Key key = rec1.getField(0, TestData.Key.class); + TestData.Value value1 = rec1.getField(1, TestData.Value.class); + TestData.Value value2 = rec2.getField(1, TestData.Value.class); + + Collection matches = this.toRemoveFrom.get(key); + if (matches == null) { + Assert.fail("Match " + key + " - " + value1 + ":" + value2 + " is unexpected."); + } + + boolean contained = matches.remove(new Match(value1, value2)); + if (!contained) { + Assert.fail("Produced match was not contained: " + key + " - " + value1 + ":" + value2); + } + if (matches.isEmpty()) { + this.toRemoveFrom.remove(key); + } + } + } +} diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/util/HashVsSortMiniBenchmark.java b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/util/HashVsSortMiniBenchmark.java index e760a1ef1f894..5d1ce7f4ac4e0 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/util/HashVsSortMiniBenchmark.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/util/HashVsSortMiniBenchmark.java @@ -32,7 +32,7 @@ import org.apache.flink.runtime.memorymanager.MemoryManager; import org.apache.flink.runtime.operators.hash.ReusingBuildFirstHashMatchIterator; import org.apache.flink.runtime.operators.hash.ReusingBuildSecondHashMatchIterator; -import org.apache.flink.runtime.operators.sort.MergeMatchIterator; +import org.apache.flink.runtime.operators.sort.ReusingMergeMatchIterator; import org.apache.flink.runtime.operators.sort.UnilateralSortMerger; import org.apache.flink.runtime.operators.testutils.DiscardingOutputCollector; import org.apache.flink.runtime.operators.testutils.DummyInvokable; @@ -143,8 +143,8 @@ public void testSortBothMerge() { final MutableObjectIterator sortedInput2 = sorter2.getIterator(); // compare with iterator values - MergeMatchIterator iterator = - new MergeMatchIterator(sortedInput1, sortedInput2, + ReusingMergeMatchIterator iterator = + new ReusingMergeMatchIterator(sortedInput1, sortedInput2, this.serializer1.getSerializer(), this.comparator1, this.serializer2.getSerializer(), this.comparator2, this.pairComparator11, this.memoryManager, this.ioManager, MEMORY_PAGES_FOR_MERGE, this.parentTask); From 26c9819edce13ab13d3789e5f825b7a31a12a987 Mon Sep 17 00:00:00 2001 From: Stephan Ewen Date: Wed, 7 Jan 2015 21:43:50 +0100 Subject: [PATCH 4/4] [FLINK-1285] Various cleanup of object reusing and non-reusing code. - The map driver now also supports this - In the merge iterator (merges sorted runs from external sort), we now always use the non-reusing code path, because the reusing codepath here implies in all cases additional instances to be held concurrently, and copy between elements, which voids the benefits of reusing elements. - For many utility iterators (in test cases), consolidates the logic between the two variants of the "next()" functions (one calls the other, where possible) - Eliminates a few copies between elements in the non-reusing parts (where possible) - Removes unused variables in the non-reusing variants (mainly serializers previously used to create instance or copy between instances) - Remove some unused types - Improves generic type safety (fewer raw types) --- .../org/apache/flink/api/common/Plan.java | 10 +- .../flink/util/MutableObjectIterator.java | 23 +- .../BroadcastVariableMaterialization.java | 5 +- .../iterative/io/HashPartitionIterator.java | 3 +- .../CoGroupWithSolutionSetFirstDriver.java | 2 +- .../CoGroupWithSolutionSetSecondDriver.java | 3 +- .../operators/GroupReduceCombineDriver.java | 8 +- .../runtime/operators/GroupReduceDriver.java | 3 +- .../flink/runtime/operators/MapDriver.java | 18 +- .../runtime/operators/RegularPactTask.java | 4 +- .../SynchronousChainedCombineDriver.java | 3 +- .../operators/hash/CompactingHashTable.java | 19 +- .../runtime/operators/hash/HashPartition.java | 15 +- .../sort/CombiningUnilateralSortMerger.java | 4 +- .../runtime/operators/sort/MergeIterator.java | 72 ++---- .../sort/NonReusingMergeMatchIterator.java | 7 +- .../operators/sort/UnilateralSortMerger.java | 6 +- .../operators/util/ReaderIterator.java | 14 +- .../plugable/DeserializationDelegate.java | 1 + .../util/EmptyMutableObjectIterator.java | 4 - .../util/KeyGroupedMutableObjectIterator.java | 208 ------------------ .../util/NonReusingKeyGroupedIterator.java | 15 +- ...eusingMutableToRegularIteratorWrapper.java | 7 +- .../util/RegularToMutableObjectIterator.java | 4 +- .../runtime/instance/InstanceManagerTest.java | 6 +- .../operators/hash/HashTableITCase.java | 39 +--- .../sort/MassiveStringSortingITCase.java | 11 +- .../operators/sort/MergeIteratorTest.java | 35 +-- ...onReusingSortMergeMatchIteratorITCase.java | 8 +- .../ReusingSortMergeMatchIteratorITCase.java | 8 +- .../operators/testutils/DriverTestBase.java | 2 +- .../operators/testutils/MockEnvironment.java | 1 - .../MutableObjectIteratorWrapper.java | 15 +- .../testutils/RandomIntPairGenerator.java | 5 +- .../runtime/operators/testutils/TestData.java | 50 ++--- .../testutils/UniformIntPairGenerator.java | 30 +-- .../testutils/UniformRecordGenerator.java | 38 +--- .../testutils/UniformStringPairGenerator.java | 41 +--- .../operators/testutils/types/StringPair.java | 3 +- .../NonReusingKeyGroupedIteratorTest.java | 4 +- 40 files changed, 159 insertions(+), 595 deletions(-) delete mode 100644 flink-runtime/src/main/java/org/apache/flink/runtime/util/KeyGroupedMutableObjectIterator.java diff --git a/flink-core/src/main/java/org/apache/flink/api/common/Plan.java b/flink-core/src/main/java/org/apache/flink/api/common/Plan.java index 4a975d25c24ac..fb50742d772fb 100644 --- a/flink-core/src/main/java/org/apache/flink/api/common/Plan.java +++ b/flink-core/src/main/java/org/apache/flink/api/common/Plan.java @@ -285,16 +285,18 @@ public String getPostPassClassName() { } /** - * Sets the runtime config object. - * @return + * Gets the execution config object. + * + * @return The execution config object. */ public ExecutionConfig getExecutionConfig() { return executionConfig; } /** - * Gets the runtime config object. - * @param executionConfig + * Sets the runtime config object defining execution parameters. + * + * @param executionConfig The execution config to use. */ public void setExecutionConfig(ExecutionConfig executionConfig) { this.executionConfig = executionConfig; diff --git a/flink-core/src/main/java/org/apache/flink/util/MutableObjectIterator.java b/flink-core/src/main/java/org/apache/flink/util/MutableObjectIterator.java index ea5ed78b17d91..d0d393eecfad1 100644 --- a/flink-core/src/main/java/org/apache/flink/util/MutableObjectIterator.java +++ b/flink-core/src/main/java/org/apache/flink/util/MutableObjectIterator.java @@ -20,11 +20,15 @@ import java.io.IOException; /** - * A simple iterator interface. The key differences to the {@link java.util.Iterator} are that this - * iterator also as a next() method that accepts an object into which it can - * place the content if the object is mutable, and that it consolidates the logic in a single - * next() function, rather than in two different functions such as - * hasNext() and next(). + * A simple iterator interface. The key differences to the {@link java.util.Iterator} are + * + *

    + *
  • It has two distinct next(), where one variant allows to pass an object that may + * be reused, if the type is mutable.
  • + *
  • It consolidates the logic in a single next() function, rather than + * splitting it over two different functions such as hasNext() and next() + *
  • + *
* * @param The element type of the collection iterated over. */ @@ -32,10 +36,10 @@ public interface MutableObjectIterator { /** * Gets the next element from the collection. The contents of that next element is put into the - * given target object. + * given reuse object, if the type is mutable. * * @param reuse The target object into which to place next element if E is mutable. - * @return The filled object or null if the iterator is exhausted + * @return The filled object or null if the iterator is exhausted. * * @throws IOException Thrown, if a problem occurred in the underlying I/O layer or in the * serialization / deserialization logic @@ -43,9 +47,10 @@ public interface MutableObjectIterator { public E next(E reuse) throws IOException; /** - * Gets the next element from the collection. The reader must create a new instance itself. + * Gets the next element from the collection. The iterator implementation + * must obtain a new instance. * - * @return The object or null if the iterator is exhausted + * @return The object or null if the iterator is exhausted. * * @throws IOException Thrown, if a problem occurred in the underlying I/O layer or in the * serialization / deserialization logic diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/broadcast/BroadcastVariableMaterialization.java b/flink-runtime/src/main/java/org/apache/flink/runtime/broadcast/BroadcastVariableMaterialization.java index 5b5f2f25c45d9..cd6223c1a050e 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/broadcast/BroadcastVariableMaterialization.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/broadcast/BroadcastVariableMaterialization.java @@ -30,6 +30,7 @@ import org.apache.flink.runtime.io.network.api.MutableReader; import org.apache.flink.runtime.operators.RegularPactTask; import org.apache.flink.runtime.operators.util.ReaderIterator; +import org.apache.flink.runtime.plugable.DeserializationDelegate; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -95,11 +96,11 @@ public void materializeVariable(MutableReader reader, TypeSerializerFactory> typedReader = (MutableReader>) reader; + @SuppressWarnings("unchecked") final TypeSerializer serializer = ((TypeSerializerFactory) serializerFactory).getSerializer(); - @SuppressWarnings("unchecked") final ReaderIterator readerIterator = new ReaderIterator(typedReader, serializer); if (materializer) { diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/iterative/io/HashPartitionIterator.java b/flink-runtime/src/main/java/org/apache/flink/runtime/iterative/io/HashPartitionIterator.java index 209fb796fb84c..93ae55fa8d169 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/iterative/io/HashPartitionIterator.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/iterative/io/HashPartitionIterator.java @@ -16,7 +16,6 @@ * limitations under the License. */ - package org.apache.flink.runtime.iterative.io; import java.io.EOFException; @@ -28,7 +27,7 @@ import org.apache.flink.util.MutableObjectIterator; /** - * {@link Iterator} over the buildside entries of a {@link HashPartition} + * {@link Iterator} over the build side entries of a {@link HashPartition} * * @param */ diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/CoGroupWithSolutionSetFirstDriver.java b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/CoGroupWithSolutionSetFirstDriver.java index a3c69a346c504..b27b6b9c3bbd4 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/CoGroupWithSolutionSetFirstDriver.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/CoGroupWithSolutionSetFirstDriver.java @@ -201,7 +201,7 @@ public void run() throws Exception { } } } else { - final NonReusingKeyGroupedIterator probeSideInput = new NonReusingKeyGroupedIterator(taskContext.getInput(0), probeSideSerializer, probeSideComparator); + final NonReusingKeyGroupedIterator probeSideInput = new NonReusingKeyGroupedIterator(taskContext.getInput(0), probeSideComparator); if (this.hashTable != null) { final CompactingHashTable join = hashTable; final CompactingHashTable.HashTableProber prober = join.getProber(this diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/CoGroupWithSolutionSetSecondDriver.java b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/CoGroupWithSolutionSetSecondDriver.java index 17fc47198dc6a..ba0f8f98eecb2 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/CoGroupWithSolutionSetSecondDriver.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/CoGroupWithSolutionSetSecondDriver.java @@ -200,7 +200,8 @@ public void run() throws Exception { } } } else { - final NonReusingKeyGroupedIterator probeSideInput = new NonReusingKeyGroupedIterator(taskContext.getInput(0), probeSideSerializer, probeSideComparator); + final NonReusingKeyGroupedIterator probeSideInput = + new NonReusingKeyGroupedIterator(taskContext.getInput(0), probeSideComparator); if (this.hashTable != null) { final CompactingHashTable join = hashTable; diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/GroupReduceCombineDriver.java b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/GroupReduceCombineDriver.java index 8d8d5dc63e2cd..be0c9c49da76e 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/GroupReduceCombineDriver.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/GroupReduceCombineDriver.java @@ -177,8 +177,8 @@ private void sortAndCombine() throws Exception { if (!sorter.isEmpty()) { this.sortAlgo.sort(sorter); - final ReusingKeyGroupedIterator keyIter = new ReusingKeyGroupedIterator(sorter.getIterator(), this.serializer, this.groupingComparator); - + final ReusingKeyGroupedIterator keyIter = + new ReusingKeyGroupedIterator(sorter.getIterator(), this.serializer, this.groupingComparator); final FlatCombineFunction combiner = this.combiner; final Collector output = this.output; @@ -192,8 +192,8 @@ private void sortAndCombine() throws Exception { if (!sorter.isEmpty()) { this.sortAlgo.sort(sorter); - final NonReusingKeyGroupedIterator keyIter = new NonReusingKeyGroupedIterator(sorter.getIterator(), this.serializer, this.groupingComparator); - + final NonReusingKeyGroupedIterator keyIter = + new NonReusingKeyGroupedIterator(sorter.getIterator(), this.groupingComparator); final FlatCombineFunction combiner = this.combiner; final Collector output = this.output; diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/GroupReduceDriver.java b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/GroupReduceDriver.java index 9d9f9946bd0f5..211622cec8cf1 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/GroupReduceDriver.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/GroupReduceDriver.java @@ -16,7 +16,6 @@ * limitations under the License. */ - package org.apache.flink.runtime.operators; import org.apache.flink.api.common.ExecutionConfig; @@ -120,7 +119,7 @@ public void run() throws Exception { } } else { - final NonReusingKeyGroupedIterator iter = new NonReusingKeyGroupedIterator(this.input, this.serializer, this.comparator); + final NonReusingKeyGroupedIterator iter = new NonReusingKeyGroupedIterator(this.input, this.comparator); // run stub implementation while (this.running && iter.nextKey()) { stub.reduce(iter.getValues(), output); diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/MapDriver.java b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/MapDriver.java index d6679dd844d65..d750fd96d6ab4 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/MapDriver.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/MapDriver.java @@ -16,7 +16,6 @@ * limitations under the License. */ - package org.apache.flink.runtime.operators; import org.apache.flink.api.common.ExecutionConfig; @@ -84,10 +83,19 @@ public void run() throws Exception { final MapFunction function = this.taskContext.getStub(); final Collector output = this.taskContext.getOutputCollector(); - IT record = this.taskContext.getInputSerializer(0).getSerializer().createInstance(); - - while (this.running && ((record = input.next(record)) != null)) { - output.collect(function.map(record)); + if (objectReuseEnabled) { + IT record = this.taskContext.getInputSerializer(0).getSerializer().createInstance(); + + while (this.running && ((record = input.next(record)) != null)) { + output.collect(function.map(record)); + } + } + else { + IT record = null; + + while (this.running && ((record = input.next()) != null)) { + output.collect(function.map(record)); + } } } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/RegularPactTask.java b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/RegularPactTask.java index db36f6d838bf8..34aaced604b69 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/RegularPactTask.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/RegularPactTask.java @@ -1088,9 +1088,7 @@ public ExecutionConfig getExecutionConfig() { } else { return new ExecutionConfig(); } - } catch (IOException e) { - throw new RuntimeException("Could not load ExecutionConfig from Job Configuration: " + e); - } catch (ClassNotFoundException e) { + } catch (Exception e) { throw new RuntimeException("Could not load ExecutionConfig from Job Configuration: " + e); } } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/chaining/SynchronousChainedCombineDriver.java b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/chaining/SynchronousChainedCombineDriver.java index 65426ef562d68..dde6fe63d9f08 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/chaining/SynchronousChainedCombineDriver.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/chaining/SynchronousChainedCombineDriver.java @@ -16,7 +16,6 @@ * limitations under the License. */ - package org.apache.flink.runtime.operators.chaining; import java.io.IOException; @@ -213,7 +212,7 @@ private void sortAndCombine() throws Exception { if (!sorter.isEmpty()) { this.sortAlgo.sort(sorter); // run the combiner - final NonReusingKeyGroupedIterator keyIter = new NonReusingKeyGroupedIterator(sorter.getIterator(), this.serializer, this.groupingComparator); + final NonReusingKeyGroupedIterator keyIter = new NonReusingKeyGroupedIterator(sorter.getIterator(), this.groupingComparator); // cache references on the stack diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/hash/CompactingHashTable.java b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/hash/CompactingHashTable.java index 3dd400fab390f..301aa82c783a9 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/hash/CompactingHashTable.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/hash/CompactingHashTable.java @@ -1220,28 +1220,11 @@ private EntryIterator(CompactingHashTable compactingHashTable) { @Override public T next(T reuse) throws IOException { - if(done || this.table.closed.get()) { - return null; - } else if(!cache.isEmpty()) { - reuse = cache.remove(cache.size()-1); - return reuse; - } else { - while(!done && cache.isEmpty()) { - done = !fillCache(); - } - if(!done) { - reuse = cache.remove(cache.size()-1); - return reuse; - } else { - return null; - } - } + return next(); } @Override public T next() throws IOException { - // This is just a copy of the above, I wanted to keep the two separate, - // in case we change something later. Plus, it keeps the diff clean... :D if(done || this.table.closed.get()) { return null; } else if(!cache.isEmpty()) { diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/hash/HashPartition.java b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/hash/HashPartition.java index 23a415ddd6085..14e4ae6abf218 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/hash/HashPartition.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/hash/HashPartition.java @@ -44,8 +44,8 @@ /** * - * @tparam BT The type of the build side records. - * @tparam PT The type of the probe side records. + * @param The type of the build side records. + * @param The type of the probe side records. */ public class HashPartition extends AbstractPagedInputView implements SeekableDataInputView { @@ -637,19 +637,12 @@ public final BT next() throws IOException } } - - protected final long getPointer() - { + protected final long getPointer() { return this.currentPointer; } - protected final int getCurrentHashCode() - { + protected final int getCurrentHashCode() { return this.currentHashCode; } } - - - - } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/sort/CombiningUnilateralSortMerger.java b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/sort/CombiningUnilateralSortMerger.java index 4f15abf246896..5d4c881f8c3f1 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/sort/CombiningUnilateralSortMerger.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/sort/CombiningUnilateralSortMerger.java @@ -243,10 +243,10 @@ else if (element == endMarker()) { } disposeSortBuffers(true); - // set lazy iterator + // set result iterator MutableObjectIterator resIter = iterators.isEmpty() ? EmptyMutableObjectIterator.get() : iterators.size() == 1 ? iterators.get(0) : - new MergeIterator(iterators, this.serializer, this.comparator); + new MergeIterator(iterators, this.comparator); setResultIterator(resIter); return; diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/sort/MergeIterator.java b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/sort/MergeIterator.java index f3dc50eb457cc..759e0e957e482 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/sort/MergeIterator.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/sort/MergeIterator.java @@ -16,7 +16,6 @@ * limitations under the License. */ - package org.apache.flink.runtime.operators.sort; import java.io.IOException; @@ -41,73 +40,49 @@ public class MergeIterator implements MutableObjectIterator { private final PartialOrderPriorityQueue> heap; // heap over the head elements of the stream - private final TypeSerializer serializer; - /** * @param iterators - * @param serializer * @param comparator * @throws IOException */ - public MergeIterator(List> iterators, - TypeSerializer serializer, TypeComparator comparator) - throws IOException - { + public MergeIterator(List> iterators, TypeComparator comparator) throws IOException { this.heap = new PartialOrderPriorityQueue>(new HeadStreamComparator(), iterators.size()); - this.serializer = serializer; for (MutableObjectIterator iterator : iterators) { - this.heap.add(new HeadStream(iterator, serializer, comparator.duplicate())); + this.heap.add(new HeadStream(iterator, comparator.duplicate())); } } /** * Gets the next smallest element, with respect to the definition of order implied by - * the {@link TypeSerializer} provided to this iterator. + * the {@link TypeSerializer} provided to this iterator. This method does in fact not + * reuse the given element (which would here imply potentially expensive copying), + * but always returns a new element. * - * @param reuse The object into which the result is put. The contents of the reuse object - * is only valid after this method, if the method returned true. Otherwise - * the contents is undefined. - * @return True, if the iterator had another element, false otherwise. + * @param reuse Ignored. + * @return The next smallest element, or null, if the iterator is exhausted. * * @see org.apache.flink.util.MutableObjectIterator#next(java.lang.Object) */ @Override - public E next(E reuse) throws IOException - { - if (this.heap.size() > 0) { - // get the smallest element - final HeadStream top = this.heap.peek(); - reuse = this.serializer.copy(top.getHead(), reuse); - - // read an element - if (!top.nextHead()) { - this.heap.poll(); - } else { - this.heap.adjustTop(); - } - return reuse; - } - else { - return null; - } + public E next(E reuse) throws IOException { + return next(); } /** * Gets the next smallest element, with respect to the definition of order implied by * the {@link TypeSerializer} provided to this iterator. * - * @return True, if the iterator had another element, false otherwise. + * @return The next element if the iterator has another element, null otherwise. * - * @see org.apache.flink.util.MutableObjectIterator#next(java.lang.Object) + * @see org.apache.flink.util.MutableObjectIterator#next() */ @Override - public E next() throws IOException - { + public E next() throws IOException { if (this.heap.size() > 0) { // get the smallest element final HeadStream top = this.heap.peek(); - E result = this.serializer.copy(top.getHead()); + E result = top.getHead(); // read an element if (!top.nextHead()) { @@ -126,20 +101,17 @@ public E next() throws IOException // Internal Classes that wrap the sorted input streams // ============================================================================================ - private static final class HeadStream - { + private static final class HeadStream { + private final MutableObjectIterator iterator; private final TypeComparator comparator; private E head; - public HeadStream(MutableObjectIterator iterator, TypeSerializer serializer, TypeComparator comparator) - throws IOException - { + public HeadStream(MutableObjectIterator iterator, TypeComparator comparator) throws IOException { this.iterator = iterator; this.comparator = comparator; - this.head = serializer.createInstance(); if (!nextHead()) { throw new IllegalStateException(); @@ -150,9 +122,8 @@ public E getHead() { return this.head; } - public boolean nextHead() throws IOException - { - if ((this.head = this.iterator.next(this.head)) != null) { + public boolean nextHead() throws IOException { + if ((this.head = this.iterator.next()) != null) { this.comparator.setReference(this.head); return true; } @@ -164,11 +135,10 @@ public boolean nextHead() throws IOException // -------------------------------------------------------------------------------------------- - private static final class HeadStreamComparator implements Comparator> - { + private static final class HeadStreamComparator implements Comparator> { + @Override - public int compare(HeadStream o1, HeadStream o2) - { + public int compare(HeadStream o1, HeadStream o2) { return o2.comparator.compareToReference(o1.comparator); } } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/sort/NonReusingMergeMatchIterator.java b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/sort/NonReusingMergeMatchIterator.java index 70b6f9a0e788b..c89b5c5dbe8c9 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/sort/NonReusingMergeMatchIterator.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/sort/NonReusingMergeMatchIterator.java @@ -40,7 +40,6 @@ import java.util.Iterator; import java.util.List; - /** * An implementation of the {@link org.apache.flink.runtime.operators.util.JoinTaskIterator} that realizes the * matching through a sort-merge join strategy. @@ -64,7 +63,7 @@ public class NonReusingMergeMatchIterator implements JoinTaskIterator private final TypeSerializer serializer2; - private final NonReusingBlockResettableIterator blockIt; // for N:M cross products with same key + private final NonReusingBlockResettableIterator blockIt; // for N:M cross products with same key private final List memoryForSpillingIterator; @@ -97,8 +96,8 @@ public NonReusingMergeMatchIterator( this.memoryManager = memoryManager; this.ioManager = ioManager; - this.iterator1 = new NonReusingKeyGroupedIterator(input1, this.serializer1, comparator1.duplicate()); - this.iterator2 = new NonReusingKeyGroupedIterator(input2, this.serializer2, comparator2.duplicate()); + this.iterator1 = new NonReusingKeyGroupedIterator(input1, comparator1.duplicate()); + this.iterator2 = new NonReusingKeyGroupedIterator(input2, comparator2.duplicate()); final int numPagesForSpiller = numMemoryPages > 20 ? 2 : 1; this.blockIt = new NonReusingBlockResettableIterator(this.memoryManager, this.serializer2, diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/sort/UnilateralSortMerger.java b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/sort/UnilateralSortMerger.java index 459ef82ba1532..dabf9bd69b7d8 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/sort/UnilateralSortMerger.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/sort/UnilateralSortMerger.java @@ -1250,13 +1250,13 @@ else if (element == EOF_MARKER) { // set lazy iterator setResultIterator(iterators.isEmpty() ? EmptyMutableObjectIterator.get() : iterators.size() == 1 ? iterators.get(0) : - new MergeIterator(iterators, this.serializer, this.comparator)); + new MergeIterator(iterators, this.comparator)); return; } // ------------------- Spilling Phase ------------------------ - final FileIOChannel.Enumerator enumerator = this.ioManager.createChannelEnumerator(); + final FileIOChannel.Enumerator enumerator = this.ioManager.createChannelEnumerator(); List channelIDs = new ArrayList(); @@ -1430,7 +1430,7 @@ protected final MergeIterator getMergingIterator(final List(inView, null, this.serializer)); } - return new MergeIterator(iterators, this.serializer, this.comparator); + return new MergeIterator(iterators, this.comparator); } /** diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/util/ReaderIterator.java b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/util/ReaderIterator.java index 606c50ce4511a..85e36a453f4b7 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/util/ReaderIterator.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/util/ReaderIterator.java @@ -16,7 +16,6 @@ * limitations under the License. */ - package org.apache.flink.runtime.operators.util; import java.io.IOException; @@ -28,13 +27,16 @@ import org.apache.flink.runtime.plugable.ReusingDeserializationDelegate; import org.apache.flink.util.MutableObjectIterator; - /** - * A {@link MutableObjectIterator} that wraps a Nephele Reader producing records of a certain type. + * A {@link MutableObjectIterator} that wraps a reader from an input channel and + * produces the reader's records. + * + * The reader supports reading objects with possible reuse of mutable types, and + * without reuse of mutable types. */ public final class ReaderIterator implements MutableObjectIterator { - private final MutableReader reader; // the source + private final MutableReader> reader; // the source private final ReusingDeserializationDelegate reusingDelegate; private final NonReusingDeserializationDelegate nonReusingDelegate; @@ -51,7 +53,6 @@ public ReaderIterator(MutableReader> reader, TypeSeri } @Override - @SuppressWarnings("unchecked") public T next(T target) throws IOException { this.reusingDelegate.setInstance(target); try { @@ -60,7 +61,6 @@ public T next(T target) throws IOException { } else { return null; } - } catch (InterruptedException e) { throw new IOException("Reader interrupted.", e); @@ -68,7 +68,6 @@ public T next(T target) throws IOException { } @Override - @SuppressWarnings("unchecked") public T next() throws IOException { try { if (this.reader.next(this.nonReusingDelegate)) { @@ -76,7 +75,6 @@ public T next() throws IOException { } else { return null; } - } catch (InterruptedException e) { throw new IOException("Reader interrupted.", e); diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/plugable/DeserializationDelegate.java b/flink-runtime/src/main/java/org/apache/flink/runtime/plugable/DeserializationDelegate.java index 9ca5954dee21e..fe941a42df98c 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/plugable/DeserializationDelegate.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/plugable/DeserializationDelegate.java @@ -20,6 +20,7 @@ import org.apache.flink.core.io.IOReadableWritable; public interface DeserializationDelegate extends IOReadableWritable { + void setInstance(T instance); T getInstance(); diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/util/EmptyMutableObjectIterator.java b/flink-runtime/src/main/java/org/apache/flink/runtime/util/EmptyMutableObjectIterator.java index 12ae5c1883c67..ae3f814952d23 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/util/EmptyMutableObjectIterator.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/util/EmptyMutableObjectIterator.java @@ -16,15 +16,12 @@ * limitations under the License. */ - package org.apache.flink.runtime.util; import org.apache.flink.util.MutableObjectIterator; - /** * An empty mutable object iterator that never returns anything. - * */ public final class EmptyMutableObjectIterator implements MutableObjectIterator { @@ -64,5 +61,4 @@ public E next(E target) { public E next() { return null; } - } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/util/KeyGroupedMutableObjectIterator.java b/flink-runtime/src/main/java/org/apache/flink/runtime/util/KeyGroupedMutableObjectIterator.java deleted file mode 100644 index c139aca8a3282..0000000000000 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/util/KeyGroupedMutableObjectIterator.java +++ /dev/null @@ -1,208 +0,0 @@ -/* - * 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.flink.runtime.util; - -import java.io.IOException; - -import org.apache.flink.api.common.typeutils.TypeComparator; -import org.apache.flink.api.common.typeutils.TypeSerializer; -import org.apache.flink.util.MutableObjectIterator; - -/** - * The KeyValueIterator returns a key and all values that belong to the key (share the same key). - * A sub-iterator over all values with the same key is provided. - */ -public final class KeyGroupedMutableObjectIterator { - - private final MutableObjectIterator iterator; - - private final TypeSerializer serializer; - - private final TypeComparator comparator; - - private E next; - - private ValuesIterator valuesIterator; - - private boolean nextIsFresh; - - /** - * Initializes the KeyGroupedIterator. It requires an iterator which returns its result - * sorted by the key fields. - * - * @param iterator An iterator over records, which are sorted by the key fields, in any order. - * @param serializer The serializer for the data type iterated over. - * @param comparator The comparator for the data type iterated over. - */ - public KeyGroupedMutableObjectIterator(MutableObjectIterator iterator, - TypeSerializer serializer, TypeComparator comparator) - { - if (iterator == null || serializer == null || comparator == null) { - throw new NullPointerException(); - } - - this.iterator = iterator; - this.serializer = serializer; - this.comparator = comparator; - } - - /** - * Moves the iterator to the next key. This method may skip any values that have not yet been returned by the - * iterator created by the {@link #getValues()} method. Hence, if called multiple times it "removes" pairs. - * - * @return true if the input iterator has an other group of key-value pairs that share the same key. - */ - public boolean nextKey() throws IOException - { - // first element - if (this.next == null) { - this.next = this.serializer.createInstance(); - if ((this.next = this.iterator.next(this.next)) != null) { - this.comparator.setReference(this.next); - this.nextIsFresh = false; - this.valuesIterator = new ValuesIterator(); - this.valuesIterator.nextIsUnconsumed = true; - return true; - } else { - // empty input, set everything null - this.valuesIterator = null; - return false; - } - } - - // Whole value-iterator was read and a new key is available. - if (this.nextIsFresh) { - this.nextIsFresh = false; - this.comparator.setReference(this.next); - this.valuesIterator.nextIsUnconsumed = true; - return true; - } - - // try to move to next key. - // Required if user code / reduce() method did not read the whole value iterator. - while (true) { - if ((this.next = this.iterator.next(this.next)) != null) { - if (!this.comparator.equalToReference(this.next)) { - // the keys do not match, so we have a new group. store the current keys - this.comparator.setReference(this.next); - this.nextIsFresh = false; - this.valuesIterator.nextIsUnconsumed = true; - return true; - } - } - else { - this.valuesIterator = null; - return false; - } - } - } - - /** - * Returns an iterator over all values that belong to the current key. The iterator is initially null - * (before the first call to {@link #nextKey()} and after all keys are consumed. In general, this method returns - * always a non-null value, if a previous call to {@link #nextKey()} return true. - * - * @return Iterator over all values that belong to the current key. - */ - public MutableObjectIterator getValues() { - return this.valuesIterator; - } - - // -------------------------------------------------------------------------------------------- - - private final class ValuesIterator implements MutableObjectIterator - { - private final TypeSerializer serializer = KeyGroupedMutableObjectIterator.this.serializer; - private final TypeComparator comparator = KeyGroupedMutableObjectIterator.this.comparator; - - private boolean nextIsUnconsumed = false; - - @Override - public E next(E target) - { - if (KeyGroupedMutableObjectIterator.this.next == null || KeyGroupedMutableObjectIterator.this.nextIsFresh) { - return null; - } - if (this.nextIsUnconsumed) { - return this.serializer.copy(KeyGroupedMutableObjectIterator.this.next, target); - } - - try { - if ((target = KeyGroupedMutableObjectIterator.this.iterator.next(target)) != null) { - // check whether the keys are equal - if (!this.comparator.equalToReference(target)) { - // moved to the next key, no more values here - KeyGroupedMutableObjectIterator.this.next = - this.serializer.copy(target, KeyGroupedMutableObjectIterator.this.next); - KeyGroupedMutableObjectIterator.this.nextIsFresh = true; - return null; - } - // same key, next value is in "next" - return target; - } - else { - // backing iterator is consumed - KeyGroupedMutableObjectIterator.this.next = null; - return null; - } - } - catch (IOException ioex) { - throw new RuntimeException("An error occurred while reading the next record: " + - ioex.getMessage(), ioex); - } - } - - @Override - public E next() - { - if (KeyGroupedMutableObjectIterator.this.next == null || KeyGroupedMutableObjectIterator.this.nextIsFresh) { - return null; - } - if (this.nextIsUnconsumed) { - return this.serializer.copy(KeyGroupedMutableObjectIterator.this.next); - } - - E result = null; - try { - if ((result = KeyGroupedMutableObjectIterator.this.iterator.next()) != null) { - // check whether the keys are equal - if (!this.comparator.equalToReference(result)) { - // moved to the next key, no more values here - KeyGroupedMutableObjectIterator.this.next = - this.serializer.copy(result); - KeyGroupedMutableObjectIterator.this.nextIsFresh = true; - return null; - } - // same key, next value is in "next" - return result; - } - else { - // backing iterator is consumed - KeyGroupedMutableObjectIterator.this.next = null; - return null; - } - } - catch (IOException ioex) { - throw new RuntimeException("An error occurred while reading the next record: " + - ioex.getMessage(), ioex); - } - } - - } -} diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/util/NonReusingKeyGroupedIterator.java b/flink-runtime/src/main/java/org/apache/flink/runtime/util/NonReusingKeyGroupedIterator.java index 43d9bde097d2f..3f28cfcfd876d 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/util/NonReusingKeyGroupedIterator.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/util/NonReusingKeyGroupedIterator.java @@ -23,18 +23,15 @@ import java.util.NoSuchElementException; import org.apache.flink.api.common.typeutils.TypeComparator; -import org.apache.flink.api.common.typeutils.TypeSerializer; import org.apache.flink.util.MutableObjectIterator; import org.apache.flink.util.TraversableOnceException; + /** - * The KeyValueIterator returns a key and all values that belong to the key (share the same key). - * + * The key grouped iterator returns a key and all values that share the same key. */ public final class NonReusingKeyGroupedIterator { private final MutableObjectIterator iterator; - - private final TypeSerializer serializer; private final TypeComparator comparator; @@ -51,18 +48,14 @@ public final class NonReusingKeyGroupedIterator { * sorted by the key fields. * * @param iterator An iterator over records, which are sorted by the key fields, in any order. - * @param serializer The serializer for the data type iterated over. * @param comparator The comparator for the data type iterated over. */ - public NonReusingKeyGroupedIterator(MutableObjectIterator iterator, TypeSerializer - serializer, TypeComparator comparator) - { - if (iterator == null || serializer == null || comparator == null) { + public NonReusingKeyGroupedIterator(MutableObjectIterator iterator, TypeComparator comparator) { + if (iterator == null || comparator == null) { throw new NullPointerException(); } this.iterator = iterator; - this.serializer = serializer; this.comparator = comparator; } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/util/NonReusingMutableToRegularIteratorWrapper.java b/flink-runtime/src/main/java/org/apache/flink/runtime/util/NonReusingMutableToRegularIteratorWrapper.java index 0db1670ea2c89..c564c93cc6fc0 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/util/NonReusingMutableToRegularIteratorWrapper.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/util/NonReusingMutableToRegularIteratorWrapper.java @@ -16,7 +16,6 @@ * limitations under the License. */ - package org.apache.flink.runtime.util; import org.apache.flink.api.common.typeutils.TypeSerializer; @@ -41,13 +40,9 @@ public class NonReusingMutableToRegularIteratorWrapper implements Iterator private boolean iteratorAvailable = true; - private TypeSerializer serializer; - - public NonReusingMutableToRegularIteratorWrapper(MutableObjectIterator source, - TypeSerializer serializer) { + public NonReusingMutableToRegularIteratorWrapper(MutableObjectIterator source, TypeSerializer serializer) { this.source = source; this.current = null; - this.serializer = serializer; } @Override diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/util/RegularToMutableObjectIterator.java b/flink-runtime/src/main/java/org/apache/flink/runtime/util/RegularToMutableObjectIterator.java index 8eb17c43726be..ffcddc15cbf4f 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/util/RegularToMutableObjectIterator.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/util/RegularToMutableObjectIterator.java @@ -16,7 +16,6 @@ * limitations under the License. */ - package org.apache.flink.runtime.util; import java.util.Iterator; @@ -24,7 +23,6 @@ import org.apache.flink.api.common.typeutils.TypeSerializer; import org.apache.flink.util.MutableObjectIterator; - public class RegularToMutableObjectIterator implements MutableObjectIterator { private final Iterator iterator; @@ -51,7 +49,7 @@ public T next(T reuse) { @Override public T next() { if (this.iterator.hasNext()) { - return this.serializer.copy(this.iterator.next()); + return this.iterator.next(); } else { return null; } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/instance/InstanceManagerTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/instance/InstanceManagerTest.java index 8a89503a58fb4..dff3dd37d6708 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/instance/InstanceManagerTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/instance/InstanceManagerTest.java @@ -84,9 +84,9 @@ public void testInstanceRegistering() { final JavaTestKit probe2 = new JavaTestKit(system); final JavaTestKit probe3 = new JavaTestKit(system); - InstanceID i1 = cm.registerTaskManager(probe1.getRef(), ici1, hardwareDescription, 1); - InstanceID i2 = cm.registerTaskManager(probe2.getRef(), ici2, hardwareDescription, 2); - InstanceID i3 = cm.registerTaskManager(probe3.getRef(), ici3, hardwareDescription, 5); + cm.registerTaskManager(probe1.getRef(), ici1, hardwareDescription, 1); + cm.registerTaskManager(probe2.getRef(), ici2, hardwareDescription, 2); + cm.registerTaskManager(probe3.getRef(), ici3, hardwareDescription, 5); assertEquals(3, cm.getNumberOfRegisteredTaskManagers()); assertEquals(8, cm.getTotalNumberOfSlots()); diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/hash/HashTableITCase.java b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/hash/HashTableITCase.java index 27ece69d9880c..c6e3062e98f70 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/hash/HashTableITCase.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/hash/HashTableITCase.java @@ -1406,15 +1406,14 @@ public void testInMemoryReOpen() throws IOException /** * An iterator that returns the Key/Value pairs with identical value a given number of times. */ - public static final class ConstantsKeyValuePairsIterator implements MutableObjectIterator - { + public static final class ConstantsKeyValuePairsIterator implements MutableObjectIterator { + private final IntValue key; private final IntValue value; private int numLeft; - public ConstantsKeyValuePairsIterator(int key, int value, int count) - { + public ConstantsKeyValuePairsIterator(int key, int value, int count) { this.key = new IntValue(key); this.value = new IntValue(value); this.numLeft = count; @@ -1436,16 +1435,7 @@ public Record next(Record reuse) { @Override public Record next() { - if (this.numLeft > 0) { - this.numLeft--; - Record result = new Record(2); - result.setField(0, this.key); - result.setField(1, this.value); - return result; - } - else { - return null; - } + return next(new Record(2)); } } @@ -1454,15 +1444,14 @@ public Record next() { /** * An iterator that returns the Key/Value pairs with identical value a given number of times. */ - private static final class ConstantsIntPairsIterator implements MutableObjectIterator - { + private static final class ConstantsIntPairsIterator implements MutableObjectIterator { + private final int key; private final int value; private int numLeft; - public ConstantsIntPairsIterator(int key, int value, int count) - { + public ConstantsIntPairsIterator(int key, int value, int count) { this.key = key; this.value = value; this.numLeft = count; @@ -1483,17 +1472,7 @@ public IntPair next(IntPair reuse) { @Override public IntPair next() { - if (this.numLeft > 0) { - this.numLeft--; - - IntPair result = new IntPair(); - result.setKey(this.key); - result.setValue(this.value); - return result; - } - else { - return null; - } + return next(new IntPair()); } } @@ -1522,7 +1501,6 @@ public boolean equalToReference(Record candidate) { } } - @Override public int compareToReference(Record candidate) { try { @@ -1531,7 +1509,6 @@ public int compareToReference(Record candidate) { } catch (NullPointerException npex) { throw new NullKeyFieldException(); } - } } } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/sort/MassiveStringSortingITCase.java b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/sort/MassiveStringSortingITCase.java index 8db9934d4cf53..55d01d2dbe1b4 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/sort/MassiveStringSortingITCase.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/sort/MassiveStringSortingITCase.java @@ -304,16 +304,7 @@ public Tuple2 next(Tuple2 reuse) throws IOEx @Override public Tuple2 next() throws IOException { - String line = reader.readLine(); - if (line == null) { - return null; - } - - String[] parts = line.split(" "); - Tuple2 result = new Tuple2(); - result.f0 = parts[0]; - result.f1 = parts; - return result; + return next(new Tuple2()); } } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/sort/MergeIteratorTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/sort/MergeIteratorTest.java index c2214560e0aa9..07330ee08e931 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/sort/MergeIteratorTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/sort/MergeIteratorTest.java @@ -16,7 +16,6 @@ * limitations under the License. */ - package org.apache.flink.runtime.operators.sort; import java.util.ArrayList; @@ -24,9 +23,7 @@ import java.util.List; import org.apache.flink.api.common.typeutils.TypeComparator; -import org.apache.flink.api.common.typeutils.TypeSerializer; import org.apache.flink.api.common.typeutils.record.RecordComparator; -import org.apache.flink.api.common.typeutils.record.RecordSerializer; import org.apache.flink.runtime.operators.sort.MergeIterator; import org.apache.flink.runtime.operators.testutils.TestData; import org.apache.flink.runtime.operators.testutils.TestData.Key; @@ -37,10 +34,7 @@ import org.junit.Before; import org.junit.Test; - -public class MergeIteratorTest -{ - private TypeSerializer serializer; +public class MergeIteratorTest { private TypeComparator comparator; @@ -48,23 +42,21 @@ public class MergeIteratorTest @SuppressWarnings("unchecked") @Before public void setup() { - this.serializer = RecordSerializer.get(); this.comparator = new RecordComparator(new int[] {0}, new Class[] { TestData.Key.class}); } - private MutableObjectIterator newIterator(final int[] keys, final String[] values) - { - return new MutableObjectIterator() - { + private MutableObjectIterator newIterator(final int[] keys, final String[] values) { + + return new MutableObjectIterator() { + private Key key = new Key(); private Value value = new Value(); private int current = 0; @Override - public Record next(Record reuse) - { + public Record next(Record reuse) { if (current < keys.length) { key.setKey(keys[current]); value.setValue(values[current]); @@ -79,15 +71,10 @@ public Record next(Record reuse) } @Override - public Record next() - { + public Record next() { if (current < keys.length) { - key.setKey(keys[current]); - value.setValue(values[current]); + Record result = new Record(new Key(keys[current]), new Value(values[current])); current++; - Record result = new Record(2); - result.setField(0, key); - result.setField(1, value); return result; } else { @@ -111,7 +98,7 @@ public void testMergeOfTwoStreams() throws Exception Comparator comparator = new TestData.KeyComparator(); // merge iterator - MutableObjectIterator iterator = new MergeIterator(iterators, this.serializer, this.comparator); + MutableObjectIterator iterator = new MergeIterator(iterators, this.comparator); // check expected order Record rec1 = new Record(); @@ -157,7 +144,7 @@ public void testMergeOfTenStreams() throws Exception Comparator comparator = new TestData.KeyComparator(); // merge iterator - MutableObjectIterator iterator = new MergeIterator(iterators, this.serializer, this.comparator); + MutableObjectIterator iterator = new MergeIterator(iterators, this.comparator); int elementsFound = 1; // check expected order @@ -201,7 +188,7 @@ public void testInvalidMerge() throws Exception Comparator comparator = new TestData.KeyComparator(); // merge iterator - MutableObjectIterator iterator = new MergeIterator(iterators, this.serializer, this.comparator); + MutableObjectIterator iterator = new MergeIterator(iterators, this.comparator); boolean violationFound = false; diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/sort/NonReusingSortMergeMatchIteratorITCase.java b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/sort/NonReusingSortMergeMatchIteratorITCase.java index e873416b2f99f..757b2e78890de 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/sort/NonReusingSortMergeMatchIteratorITCase.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/sort/NonReusingSortMergeMatchIteratorITCase.java @@ -192,8 +192,8 @@ public void testMergeWithHighNumberOfCommonKeys() inList2.add(gen2Iter); inList2.add(const2Iter); - MutableObjectIterator input1 = new MergeIterator(inList1, serializer1, comparator1.duplicate()); - MutableObjectIterator input2 = new MergeIterator(inList2, serializer2, comparator2.duplicate()); + MutableObjectIterator input1 = new MergeIterator(inList1, comparator1.duplicate()); + MutableObjectIterator input2 = new MergeIterator(inList2, comparator2.duplicate()); // collect expected data final Map> expectedMatchesMap = matchValues( @@ -218,8 +218,8 @@ public void testMergeWithHighNumberOfCommonKeys() inList2.add(gen2Iter); inList2.add(const2Iter); - input1 = new MergeIterator(inList1, serializer1, comparator1.duplicate()); - input2 = new MergeIterator(inList2, serializer2, comparator2.duplicate()); + input1 = new MergeIterator(inList1, comparator1.duplicate()); + input2 = new MergeIterator(inList2, comparator2.duplicate()); final JoinFunction matcher = new MatchRemovingMatcher(expectedMatchesMap); diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/sort/ReusingSortMergeMatchIteratorITCase.java b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/sort/ReusingSortMergeMatchIteratorITCase.java index dd7248d65ef39..474fa3cf63997 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/sort/ReusingSortMergeMatchIteratorITCase.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/sort/ReusingSortMergeMatchIteratorITCase.java @@ -192,8 +192,8 @@ public void testMergeWithHighNumberOfCommonKeys() inList2.add(gen2Iter); inList2.add(const2Iter); - MutableObjectIterator input1 = new MergeIterator(inList1, serializer1, comparator1.duplicate()); - MutableObjectIterator input2 = new MergeIterator(inList2, serializer2, comparator2.duplicate()); + MutableObjectIterator input1 = new MergeIterator(inList1, comparator1.duplicate()); + MutableObjectIterator input2 = new MergeIterator(inList2, comparator2.duplicate()); // collect expected data final Map> expectedMatchesMap = matchValues( @@ -218,8 +218,8 @@ public void testMergeWithHighNumberOfCommonKeys() inList2.add(gen2Iter); inList2.add(const2Iter); - input1 = new MergeIterator(inList1, serializer1, comparator1.duplicate()); - input2 = new MergeIterator(inList2, serializer2, comparator2.duplicate()); + input1 = new MergeIterator(inList1, comparator1.duplicate()); + input2 = new MergeIterator(inList2, comparator2.duplicate()); final JoinFunction matcher = new MatchRemovingMatcher(expectedMatchesMap); diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/testutils/DriverTestBase.java b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/testutils/DriverTestBase.java index 0e169ec7a3be1..9b9609b6ece44 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/testutils/DriverTestBase.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/testutils/DriverTestBase.java @@ -170,7 +170,7 @@ public void setNumFileHandlesForSort(int numFileHandles) { this.numFileHandles = numFileHandles; } - @SuppressWarnings({"unchecked","rawtypes"}) + @SuppressWarnings("rawtypes") public void testDriver(PactDriver driver, Class stubClass) throws Exception { testDriverInternal(driver, stubClass); } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/testutils/MockEnvironment.java b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/testutils/MockEnvironment.java index 0629ea08e8a2c..9a999fd242041 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/testutils/MockEnvironment.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/testutils/MockEnvironment.java @@ -55,7 +55,6 @@ import org.apache.flink.runtime.memorymanager.DefaultMemoryManager; import org.apache.flink.runtime.memorymanager.MemoryManager; import org.apache.flink.runtime.plugable.DeserializationDelegate; -import org.apache.flink.runtime.plugable.NonReusingDeserializationDelegate; import org.apache.flink.types.Record; import org.apache.flink.util.MutableObjectIterator; diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/testutils/MutableObjectIteratorWrapper.java b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/testutils/MutableObjectIteratorWrapper.java index 88f16fc9aac6f..8c079215b1645 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/testutils/MutableObjectIteratorWrapper.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/testutils/MutableObjectIteratorWrapper.java @@ -16,7 +16,6 @@ * limitations under the License. */ - package org.apache.flink.runtime.operators.testutils; import java.io.IOException; @@ -25,16 +24,14 @@ import org.apache.flink.types.Record; import org.apache.flink.util.MutableObjectIterator; - /** * */ -public class MutableObjectIteratorWrapper implements MutableObjectIterator -{ +public class MutableObjectIteratorWrapper implements MutableObjectIterator { + private final Iterator source; - public MutableObjectIteratorWrapper(Iterator source) - { + public MutableObjectIteratorWrapper(Iterator source) { this.source = source; } @@ -42,8 +39,7 @@ public MutableObjectIteratorWrapper(Iterator source) @Override public Record next(Record reuse) throws IOException { if (this.source.hasNext()) { - this.source.next().copyTo(reuse); - return reuse; + return this.source.next(); } else { return null; @@ -52,6 +48,7 @@ public Record next(Record reuse) throws IOException { @Override public Record next() throws IOException { + // copy to be on the safe side if (this.source.hasNext()) { Record result = new Record(); this.source.next().copyTo(result); @@ -61,6 +58,4 @@ public Record next() throws IOException { return null; } } - - } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/testutils/RandomIntPairGenerator.java b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/testutils/RandomIntPairGenerator.java index 48a512ce55141..19bd396728bd1 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/testutils/RandomIntPairGenerator.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/testutils/RandomIntPairGenerator.java @@ -60,10 +60,7 @@ public IntPair next(IntPair reuse) { @Override public IntPair next() { if (this.count++ < this.numRecords) { - IntPair result = new IntPair(); - result.setKey(this.rnd.nextInt()); - result.setValue(this.rnd.nextInt()); - return result; + return new IntPair(this.rnd.nextInt(), this.rnd.nextInt()); } else { return null; } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/testutils/TestData.java b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/testutils/TestData.java index 5fe1303a89ca8..400e7986d4c52 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/testutils/TestData.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/testutils/TestData.java @@ -16,7 +16,6 @@ * limitations under the License. */ - package org.apache.flink.runtime.operators.testutils; import java.util.Comparator; @@ -31,11 +30,11 @@ * Test data utilities classes. */ public final class TestData { + /** * Private constructor (container class should not be instantiated) */ - private TestData() { - } + private TestData() {} /** * Key comparator. @@ -74,6 +73,7 @@ public void setKey(int key) { * Value implementation. */ public static class Value extends StringValue { + private static final long serialVersionUID = 1L; public Value() { @@ -84,13 +84,8 @@ public Value(String v) { super(v); } - /* - * (non-Javadoc) - * @see java.lang.Object#equals(java.lang.Object) - */ @Override - public boolean equals(final Object obj) - { + public boolean equals(final Object obj) { if (this == obj) { return true; } @@ -119,7 +114,8 @@ public boolean equals(final Object obj) /** * Pair generator. */ - public static class Generator implements MutableObjectIterator{ + public static class Generator implements MutableObjectIterator { + public enum KeyMode { SORTED, RANDOM }; @@ -152,8 +148,7 @@ public Generator(long seed, int keyMax, int valueLength) { this(seed, keyMax, valueLength, KeyMode.RANDOM, ValueMode.FIX_LENGTH); } - public Generator(long seed, int keyMax, int valueLength, KeyMode keyMode, ValueMode valueMode) - { + public Generator(long seed, int keyMax, int valueLength, KeyMode keyMode, ValueMode valueMode) { this(seed, keyMax, valueLength, keyMode, valueMode, null); } @@ -182,14 +177,7 @@ public Record next(Record reuse) { } public Record next() { - this.key.setKey(keyMode == KeyMode.SORTED ? ++counter : Math.abs(random.nextInt() % keyMax) + 1); - if (this.valueMode != ValueMode.CONSTANT) { - this.value.setValue(randomString()); - } - Record result = new Record(2); - result.setField(0, this.key); - result.setField(1, this.value); - return result; + return next(new Record(2)); } public boolean next(org.apache.flink.types.Value[] target) { @@ -294,8 +282,8 @@ public void reset() { // -------------------------------------------------------------------------------------------- - public static class ConstantValueIterator implements MutableObjectIterator - { + public static class ConstantValueIterator implements MutableObjectIterator { + private final Key key; private final Value value; @@ -307,10 +295,9 @@ public static class ConstantValueIterator implements MutableObjectIterator= numVals) { - return null; - } - - result.setKey(keyCnt++); - result.setValue(valCnt); - - if(keyCnt == numKeys) { - keyCnt = 0; - valCnt++; - } - } else { - if(keyCnt >= numKeys) { - return null; - } - - result.setKey(keyCnt); - result.setValue(valCnt++); - - if(valCnt == numVals) { - valCnt = 0; - keyCnt++; - } - } - - return result; + return next(new IntPair()); } - } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/testutils/UniformRecordGenerator.java b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/testutils/UniformRecordGenerator.java index b628f059e14ae..e27bdfa9dc0b6 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/testutils/UniformRecordGenerator.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/testutils/UniformRecordGenerator.java @@ -16,7 +16,6 @@ * limitations under the License. */ - package org.apache.flink.runtime.operators.testutils; import org.apache.flink.types.IntValue; @@ -25,8 +24,8 @@ public class UniformRecordGenerator implements MutableObjectIterator { - private final IntValue key = new IntValue(); - private final IntValue value = new IntValue(); + private IntValue key = new IntValue(); + private IntValue value = new IntValue(); int numKeys; int numVals; @@ -84,35 +83,8 @@ public Record next(Record reuse) { @Override public Record next() { - if(!repeatKey) { - if(valCnt >= numVals+startVal) { - return null; - } - - key.setValue(keyCnt++); - value.setValue(valCnt); - - if(keyCnt == numKeys+startKey) { - keyCnt = startKey; - valCnt++; - } - } else { - if(keyCnt >= numKeys+startKey) { - return null; - } - key.setValue(keyCnt); - value.setValue(valCnt++); - - if(valCnt == numVals+startVal) { - valCnt = startVal; - keyCnt++; - } - } - - Record result = new Record(2); - result.setField(0, this.key); - result.setField(1, this.value); - result.updateBinaryRepresenation(); - return result; + key = new IntValue(); + value = new IntValue(); + return next(new Record(2)); } } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/testutils/UniformStringPairGenerator.java b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/testutils/UniformStringPairGenerator.java index 45a44facb4440..3607421efb2ab 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/testutils/UniformStringPairGenerator.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/testutils/UniformStringPairGenerator.java @@ -25,12 +25,12 @@ public class UniformStringPairGenerator implements MutableObjectIterator { - final int numKeys; - final int numVals; + private final int numKeys; + private final int numVals; - int keyCnt = 0; - int valCnt = 0; - boolean repeatKey; + private int keyCnt = 0; + private int valCnt = 0; + private boolean repeatKey; public UniformStringPairGenerator(int numKeys, int numVals, boolean repeatKey) { this.numKeys = numKeys; @@ -71,35 +71,6 @@ public StringPair next(StringPair target) throws IOException { @Override public StringPair next() throws IOException { - StringPair result = new StringPair(); - if(!repeatKey) { - if(valCnt >= numVals) { - return null; - } - - result.setKey(Integer.toString(keyCnt++)); - result.setValue(Integer.toBinaryString(valCnt)); - - if(keyCnt == numKeys) { - keyCnt = 0; - valCnt++; - } - } else { - if(keyCnt >= numKeys) { - return null; - } - - result.setKey(Integer.toString(keyCnt)); - result.setValue(Integer.toBinaryString(valCnt++)); - - if(valCnt == numVals) { - valCnt = 0; - keyCnt++; - } - } - - return result; + return next(new StringPair()); } - - } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/testutils/types/StringPair.java b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/testutils/types/StringPair.java index b832a46dc5515..93cd062807ce8 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/testutils/types/StringPair.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/testutils/types/StringPair.java @@ -24,8 +24,7 @@ public class StringPair { private String value; - public StringPair() - {} + public StringPair() {} public StringPair(String key, String value) { this.key = key; diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/util/NonReusingKeyGroupedIteratorTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/util/NonReusingKeyGroupedIteratorTest.java index 9f651b173c167..b71fdb0d88983 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/util/NonReusingKeyGroupedIteratorTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/util/NonReusingKeyGroupedIteratorTest.java @@ -24,7 +24,6 @@ import java.util.NoSuchElementException; import org.apache.flink.api.common.typeutils.record.RecordComparator; -import org.apache.flink.api.common.typeutils.record.RecordSerializer; import org.apache.flink.types.IntValue; import org.apache.flink.types.Record; import org.apache.flink.types.StringValue; @@ -94,11 +93,10 @@ public Record next() throws IOException { } }; - final RecordSerializer serializer = RecordSerializer.get(); @SuppressWarnings("unchecked") final RecordComparator comparator = new RecordComparator(new int[] {0}, new Class[] {IntValue.class}); - this.psi = new NonReusingKeyGroupedIterator(this.sourceIter, serializer, comparator); + this.psi = new NonReusingKeyGroupedIterator(this.sourceIter, comparator); } @Test