kl0u commented on a change in pull request #13357: URL: https://github.com/apache/flink/pull/13357#discussion_r493267562
########## File path: flink-runtime/src/main/java/org/apache/flink/runtime/operators/sort/SortingThread.java ########## @@ -0,0 +1,85 @@ +/* + * 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.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import javax.annotation.Nullable; + +import java.io.IOException; + +import static org.apache.flink.runtime.operators.sort.CircularElement.EOF_MARKER; +import static org.apache.flink.runtime.operators.sort.CircularElement.SPILLING_MARKER; + +/** + * The thread that sorts filled buffers. + */ +class SortingThread<E> extends ThreadBase<E> { + + /** Logging. */ + private static final Logger LOG = LoggerFactory.getLogger(SortingThread.class); + + private final IndexedSorter sorter; + + /** + * Creates a new sorting thread. + * + * @param exceptionHandler The exception handler to call for all exceptions. + * @param dispatcher The queues used to pass buffers between the threads. + */ + public SortingThread( + @Nullable ExceptionHandler<IOException> exceptionHandler, + StageMessageDispatcher<E> dispatcher) { + super(exceptionHandler, "SortMerger sorting thread", dispatcher); + + // members + this.sorter = new QuickSort(); + } + + /** + * Entry point of the thread. + */ + public void go() throws IOException, InterruptedException { + boolean alive = true; + + // loop as long as the thread is marked alive + while (isRunning() && alive) { + CircularElement<E> element = this.dispatcher.take(SortStage.SORT); Review comment: This could become `final`. ########## File path: flink-runtime/src/main/java/org/apache/flink/runtime/operators/sort/SortingThread.java ########## @@ -0,0 +1,85 @@ +/* + * 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.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import javax.annotation.Nullable; + +import java.io.IOException; + +import static org.apache.flink.runtime.operators.sort.CircularElement.EOF_MARKER; +import static org.apache.flink.runtime.operators.sort.CircularElement.SPILLING_MARKER; + +/** + * The thread that sorts filled buffers. + */ +class SortingThread<E> extends ThreadBase<E> { + + /** Logging. */ + private static final Logger LOG = LoggerFactory.getLogger(SortingThread.class); + + private final IndexedSorter sorter; + + /** + * Creates a new sorting thread. + * + * @param exceptionHandler The exception handler to call for all exceptions. + * @param dispatcher The queues used to pass buffers between the threads. + */ + public SortingThread( + @Nullable ExceptionHandler<IOException> exceptionHandler, + StageMessageDispatcher<E> dispatcher) { + super(exceptionHandler, "SortMerger sorting thread", dispatcher); + + // members + this.sorter = new QuickSort(); + } + + /** + * Entry point of the thread. + */ + public void go() throws IOException, InterruptedException { Review comment: It seems you can remove the `IOException` now. ########## File path: flink-runtime/src/main/java/org/apache/flink/runtime/operators/sort/SpillingThread.java ########## @@ -0,0 +1,575 @@ +/* + * 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.TypeSerializer; +import org.apache.flink.core.memory.MemorySegment; +import org.apache.flink.runtime.io.disk.ChannelReaderInputViewIterator; +import org.apache.flink.runtime.io.disk.iomanager.BlockChannelReader; +import org.apache.flink.runtime.io.disk.iomanager.BlockChannelWriter; +import org.apache.flink.runtime.io.disk.iomanager.ChannelReaderInputView; +import org.apache.flink.runtime.io.disk.iomanager.ChannelWriterOutputView; +import org.apache.flink.runtime.io.disk.iomanager.FileIOChannel; +import org.apache.flink.runtime.io.disk.iomanager.IOManager; +import org.apache.flink.runtime.memory.MemoryManager; +import org.apache.flink.runtime.util.EmptyMutableObjectIterator; +import org.apache.flink.util.MutableObjectIterator; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import javax.annotation.Nullable; + +import java.io.IOException; +import java.util.ArrayDeque; +import java.util.ArrayList; +import java.util.Collections; +import java.util.Iterator; +import java.util.List; +import java.util.Queue; + +import static org.apache.flink.runtime.operators.sort.CircularElement.EOF_MARKER; +import static org.apache.flink.runtime.operators.sort.CircularElement.SPILLING_MARKER; +import static org.apache.flink.util.Preconditions.checkNotNull; + +/** + * The thread that handles the spilling of intermediate results and sets up the merging. It also merges the + * channels until sufficiently few channels remain to perform the final streamed merge. + */ +final class SpillingThread<E> extends ThreadBase<E> { + /** + * An interface for injecting custom behaviour for spilling and merging phases. + */ + interface SpillingBehaviour<E> { + default void open() {} + + default void close() {} + + /** + * A method that allows adjusting the spilling phase. We can inject e.g. combining the elements while spilling. + */ + void spillBuffer( + CircularElement<E> element, + ChannelWriterOutputView output, + LargeRecordHandler<E> largeRecordHandler) throws IOException; + + /** + * A method that allows adjusting the merging phase. We can inject e.g. combining the spilled elements. + */ + void mergeRecords(MergeIterator<E> mergeIterator, ChannelWriterOutputView output) throws IOException; + } + + /** Logging. */ + private static final Logger LOG = LoggerFactory.getLogger(SpillingThread.class); + + private final MemoryManager memManager; // memory manager to release memory + + private final IOManager ioManager; // I/O manager to create channels + + private final TypeSerializer<E> serializer; // The serializer for the data type + + private final TypeComparator<E> comparator; // The comparator that establishes the order relation. + + private final List<MemorySegment> writeMemory; // memory segments for writing + + private final List<MemorySegment> mergeReadMemory; // memory segments for sorting/reading + + private final int maxFanIn; + + private final SpillChannelManager spillChannelManager; + + private final LargeRecordHandler<E> largeRecordHandler; + + private final SpillingBehaviour<E> spillingBehaviour; + + private boolean spillingBehaviourOpened = false; Review comment: I think this should become `volatile` because it is called in `close()` and in `go()` which run on/ are called by different threads. ########## File path: flink-runtime/src/main/java/org/apache/flink/runtime/operators/sort/ThreadBase.java ########## @@ -0,0 +1,137 @@ +/* + * 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 javax.annotation.Nullable; + +import java.io.IOException; + +import static org.apache.flink.util.Preconditions.checkNotNull; + +/** + * Base class for all working threads in this sort-merger. The specific threads for reading, sorting, spilling, + * merging, etc... extend this subclass. + * <p> + * The threads are designed to terminate themselves when the task they are set up to do is completed. Further more, + * they terminate immediately when the <code>shutdown()</code> method is called. + */ +abstract class ThreadBase<E> extends Thread implements Thread.UncaughtExceptionHandler, StageRunner { + + /** + * The queue of empty buffer that can be used for reading; + */ + protected final StageMessageDispatcher<E> dispatcher; + + /** + * The exception handler for any problems. + */ + private final ExceptionHandler<IOException> exceptionHandler; + + /** + * The flag marking this thread as alive. + */ + private volatile boolean alive; + + /** + * Creates a new thread. + * + * @param exceptionHandler The exception handler to call for all exceptions. + * @param name The name of the thread. + * @param queues The queues used to pass buffers between the threads. + */ + protected ThreadBase( + @Nullable ExceptionHandler<IOException> exceptionHandler, + String name, + StageMessageDispatcher<E> queues) { + // thread setup + super(checkNotNull(name)); + this.setDaemon(true); + + // exception handling + this.exceptionHandler = exceptionHandler; + this.setUncaughtExceptionHandler(this); + + this.dispatcher = checkNotNull(queues); + this.alive = true; + } + + /** + * Implements exception handling and delegates to go(). + */ + public void run() { + try { + go(); + } + catch (Throwable t) { + internalHandleException(new IOException("Thread '" + getName() + "' terminated due to an exception: " + + t.getMessage(), t)); + } + } + + /** + * Equivalent to the run() method. + * + * @throws IOException Exceptions that prohibit correct completion of the work may be thrown by the thread. + */ + protected abstract void go() throws IOException, InterruptedException; + + /** + * Checks whether this thread is still alive. + * + * @return true, if the thread is alive, false otherwise. + */ + public boolean isRunning() { Review comment: This can become `protected`. ########## File path: flink-runtime/src/main/java/org/apache/flink/runtime/operators/sort/SpillingThread.java ########## @@ -0,0 +1,575 @@ +/* + * 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.TypeSerializer; +import org.apache.flink.core.memory.MemorySegment; +import org.apache.flink.runtime.io.disk.ChannelReaderInputViewIterator; +import org.apache.flink.runtime.io.disk.iomanager.BlockChannelReader; +import org.apache.flink.runtime.io.disk.iomanager.BlockChannelWriter; +import org.apache.flink.runtime.io.disk.iomanager.ChannelReaderInputView; +import org.apache.flink.runtime.io.disk.iomanager.ChannelWriterOutputView; +import org.apache.flink.runtime.io.disk.iomanager.FileIOChannel; +import org.apache.flink.runtime.io.disk.iomanager.IOManager; +import org.apache.flink.runtime.memory.MemoryManager; +import org.apache.flink.runtime.util.EmptyMutableObjectIterator; +import org.apache.flink.util.MutableObjectIterator; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import javax.annotation.Nullable; + +import java.io.IOException; +import java.util.ArrayDeque; +import java.util.ArrayList; +import java.util.Collections; +import java.util.Iterator; +import java.util.List; +import java.util.Queue; + +import static org.apache.flink.runtime.operators.sort.CircularElement.EOF_MARKER; +import static org.apache.flink.runtime.operators.sort.CircularElement.SPILLING_MARKER; +import static org.apache.flink.util.Preconditions.checkNotNull; + +/** + * The thread that handles the spilling of intermediate results and sets up the merging. It also merges the + * channels until sufficiently few channels remain to perform the final streamed merge. + */ +final class SpillingThread<E> extends ThreadBase<E> { + /** + * An interface for injecting custom behaviour for spilling and merging phases. + */ + interface SpillingBehaviour<E> { + default void open() {} + + default void close() {} + + /** + * A method that allows adjusting the spilling phase. We can inject e.g. combining the elements while spilling. + */ + void spillBuffer( + CircularElement<E> element, + ChannelWriterOutputView output, + LargeRecordHandler<E> largeRecordHandler) throws IOException; + + /** + * A method that allows adjusting the merging phase. We can inject e.g. combining the spilled elements. + */ + void mergeRecords(MergeIterator<E> mergeIterator, ChannelWriterOutputView output) throws IOException; + } + + /** Logging. */ + private static final Logger LOG = LoggerFactory.getLogger(SpillingThread.class); + + private final MemoryManager memManager; // memory manager to release memory + + private final IOManager ioManager; // I/O manager to create channels + + private final TypeSerializer<E> serializer; // The serializer for the data type + + private final TypeComparator<E> comparator; // The comparator that establishes the order relation. + + private final List<MemorySegment> writeMemory; // memory segments for writing + + private final List<MemorySegment> mergeReadMemory; // memory segments for sorting/reading + + private final int maxFanIn; + + private final SpillChannelManager spillChannelManager; + + private final LargeRecordHandler<E> largeRecordHandler; + + private final SpillingBehaviour<E> spillingBehaviour; + + private boolean spillingBehaviourOpened = false; + + private final int minNumWriteBuffers; + + private final int maxNumWriteBuffers; + + SpillingThread( + @Nullable ExceptionHandler<IOException> exceptionHandler, + StageMessageDispatcher<E> dispatcher, + MemoryManager memManager, + IOManager ioManager, + TypeSerializer<E> serializer, + TypeComparator<E> comparator, + List<MemorySegment> sortReadMemory, + List<MemorySegment> writeMemory, + int maxNumFileHandles, + SpillChannelManager spillingChannelManager, + @Nullable LargeRecordHandler<E> largeRecordHandler, + SpillingBehaviour<E> spillingBehaviour, + int minNumWriteBuffers, + int maxNumWriteBuffers) { + super(exceptionHandler, "SortMerger spilling thread", dispatcher); + this.memManager = checkNotNull(memManager); + this.ioManager = checkNotNull(ioManager); + this.serializer = checkNotNull(serializer); + this.comparator = checkNotNull(comparator); + this.mergeReadMemory = checkNotNull(sortReadMemory); + this.writeMemory = checkNotNull(writeMemory); + this.maxFanIn = maxNumFileHandles; + this.spillChannelManager = checkNotNull(spillingChannelManager); + this.largeRecordHandler = largeRecordHandler; + this.spillingBehaviour = checkNotNull(spillingBehaviour); + this.minNumWriteBuffers = minNumWriteBuffers; + this.maxNumWriteBuffers = maxNumWriteBuffers; + } + + /** + * Entry point of the thread. + */ + public void go() throws IOException, InterruptedException { + + final Queue<CircularElement<E>> cache = new ArrayDeque<>(); + boolean cacheOnly = readCache(cache); + + // check whether the thread was canceled + if (!isRunning()) { + return; + } + + MutableObjectIterator<E> largeRecords = null; + + // check if we can stay in memory with the large record handler + if (cacheOnly && largeRecordHandler != null && largeRecordHandler.hasData()) { + List<MemorySegment> memoryForLargeRecordSorting = new ArrayList<>(); + + CircularElement<E> circElement; + while ((circElement = this.dispatcher.poll(SortStage.READ)) != null) { + circElement.getBuffer().dispose(); + memoryForLargeRecordSorting.addAll(circElement.getMemory()); + } + + if (memoryForLargeRecordSorting.isEmpty()) { + cacheOnly = false; + LOG.debug("Going to disk-based merge because of large records."); + } else { + LOG.debug("Sorting large records, to add them to in-memory merge."); + largeRecords = largeRecordHandler.finishWriteAndSortKeys(memoryForLargeRecordSorting); + } + } + + // ------------------- In-Memory Merge ------------------------ + if (cacheOnly) { + mergeInMemory(cache, largeRecords); + return; + } + + // ------------------- Spilling Phase ------------------------ + List<ChannelWithBlockCount> channelIDs = startSpilling(cache); + + // ------------------- Merging Phase ------------------------ + + mergeOnDisk(channelIDs); + } + + @Override + public void close() throws InterruptedException { + super.close(); + if (spillingBehaviourOpened) { + this.spillingBehaviour.close(); + this.spillingBehaviourOpened = false; + } + } + + private boolean readCache(Queue<CircularElement<E>> cache) throws InterruptedException { + CircularElement<E> element; Review comment: This can go in the `while()` loop now and become `final`. ########## File path: flink-runtime/src/main/java/org/apache/flink/runtime/operators/sort/SpillingThread.java ########## @@ -0,0 +1,575 @@ +/* + * 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.TypeSerializer; +import org.apache.flink.core.memory.MemorySegment; +import org.apache.flink.runtime.io.disk.ChannelReaderInputViewIterator; +import org.apache.flink.runtime.io.disk.iomanager.BlockChannelReader; +import org.apache.flink.runtime.io.disk.iomanager.BlockChannelWriter; +import org.apache.flink.runtime.io.disk.iomanager.ChannelReaderInputView; +import org.apache.flink.runtime.io.disk.iomanager.ChannelWriterOutputView; +import org.apache.flink.runtime.io.disk.iomanager.FileIOChannel; +import org.apache.flink.runtime.io.disk.iomanager.IOManager; +import org.apache.flink.runtime.memory.MemoryManager; +import org.apache.flink.runtime.util.EmptyMutableObjectIterator; +import org.apache.flink.util.MutableObjectIterator; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import javax.annotation.Nullable; + +import java.io.IOException; +import java.util.ArrayDeque; +import java.util.ArrayList; +import java.util.Collections; +import java.util.Iterator; +import java.util.List; +import java.util.Queue; + +import static org.apache.flink.runtime.operators.sort.CircularElement.EOF_MARKER; +import static org.apache.flink.runtime.operators.sort.CircularElement.SPILLING_MARKER; +import static org.apache.flink.util.Preconditions.checkNotNull; + +/** + * The thread that handles the spilling of intermediate results and sets up the merging. It also merges the + * channels until sufficiently few channels remain to perform the final streamed merge. + */ +final class SpillingThread<E> extends ThreadBase<E> { + /** + * An interface for injecting custom behaviour for spilling and merging phases. + */ + interface SpillingBehaviour<E> { + default void open() {} + + default void close() {} + + /** + * A method that allows adjusting the spilling phase. We can inject e.g. combining the elements while spilling. + */ + void spillBuffer( + CircularElement<E> element, + ChannelWriterOutputView output, + LargeRecordHandler<E> largeRecordHandler) throws IOException; + + /** + * A method that allows adjusting the merging phase. We can inject e.g. combining the spilled elements. + */ + void mergeRecords(MergeIterator<E> mergeIterator, ChannelWriterOutputView output) throws IOException; + } + + /** Logging. */ + private static final Logger LOG = LoggerFactory.getLogger(SpillingThread.class); + + private final MemoryManager memManager; // memory manager to release memory + + private final IOManager ioManager; // I/O manager to create channels + + private final TypeSerializer<E> serializer; // The serializer for the data type + + private final TypeComparator<E> comparator; // The comparator that establishes the order relation. + + private final List<MemorySegment> writeMemory; // memory segments for writing + + private final List<MemorySegment> mergeReadMemory; // memory segments for sorting/reading + + private final int maxFanIn; + + private final SpillChannelManager spillChannelManager; + + private final LargeRecordHandler<E> largeRecordHandler; + + private final SpillingBehaviour<E> spillingBehaviour; + + private boolean spillingBehaviourOpened = false; + + private final int minNumWriteBuffers; + + private final int maxNumWriteBuffers; + + SpillingThread( + @Nullable ExceptionHandler<IOException> exceptionHandler, + StageMessageDispatcher<E> dispatcher, + MemoryManager memManager, + IOManager ioManager, + TypeSerializer<E> serializer, + TypeComparator<E> comparator, + List<MemorySegment> sortReadMemory, + List<MemorySegment> writeMemory, + int maxNumFileHandles, + SpillChannelManager spillingChannelManager, + @Nullable LargeRecordHandler<E> largeRecordHandler, + SpillingBehaviour<E> spillingBehaviour, + int minNumWriteBuffers, + int maxNumWriteBuffers) { + super(exceptionHandler, "SortMerger spilling thread", dispatcher); + this.memManager = checkNotNull(memManager); + this.ioManager = checkNotNull(ioManager); + this.serializer = checkNotNull(serializer); + this.comparator = checkNotNull(comparator); + this.mergeReadMemory = checkNotNull(sortReadMemory); + this.writeMemory = checkNotNull(writeMemory); + this.maxFanIn = maxNumFileHandles; + this.spillChannelManager = checkNotNull(spillingChannelManager); + this.largeRecordHandler = largeRecordHandler; + this.spillingBehaviour = checkNotNull(spillingBehaviour); + this.minNumWriteBuffers = minNumWriteBuffers; + this.maxNumWriteBuffers = maxNumWriteBuffers; + } + + /** + * Entry point of the thread. + */ + public void go() throws IOException, InterruptedException { + + final Queue<CircularElement<E>> cache = new ArrayDeque<>(); + boolean cacheOnly = readCache(cache); + + // check whether the thread was canceled + if (!isRunning()) { + return; + } + + MutableObjectIterator<E> largeRecords = null; + + // check if we can stay in memory with the large record handler + if (cacheOnly && largeRecordHandler != null && largeRecordHandler.hasData()) { + List<MemorySegment> memoryForLargeRecordSorting = new ArrayList<>(); + + CircularElement<E> circElement; + while ((circElement = this.dispatcher.poll(SortStage.READ)) != null) { + circElement.getBuffer().dispose(); + memoryForLargeRecordSorting.addAll(circElement.getMemory()); + } + + if (memoryForLargeRecordSorting.isEmpty()) { + cacheOnly = false; + LOG.debug("Going to disk-based merge because of large records."); + } else { + LOG.debug("Sorting large records, to add them to in-memory merge."); + largeRecords = largeRecordHandler.finishWriteAndSortKeys(memoryForLargeRecordSorting); + } + } + + // ------------------- In-Memory Merge ------------------------ + if (cacheOnly) { + mergeInMemory(cache, largeRecords); + return; + } + + // ------------------- Spilling Phase ------------------------ + List<ChannelWithBlockCount> channelIDs = startSpilling(cache); + + // ------------------- Merging Phase ------------------------ + + mergeOnDisk(channelIDs); + } + + @Override + public void close() throws InterruptedException { + super.close(); + if (spillingBehaviourOpened) { + this.spillingBehaviour.close(); + this.spillingBehaviourOpened = false; + } + } + + private boolean readCache(Queue<CircularElement<E>> cache) throws InterruptedException { + CircularElement<E> element; + + // ------------------- In-Memory Cache ------------------------ + // fill cache + while (isRunning()) { + // take next element from queue + element = this.dispatcher.take(SortStage.SPILL); + + if (element == SPILLING_MARKER) { + return false; + } else if (element == EOF_MARKER) { + return true; + } + cache.add(element); + } + return false; + } + + private void mergeOnDisk(List<ChannelWithBlockCount> channelIDs) throws IOException { + // make sure we have enough memory to merge and for large record handling + List<MemorySegment> mergeReadMemory; + MutableObjectIterator<E> largeRecords = null; + + if (largeRecordHandler != null && largeRecordHandler.hasData()) { + + List<MemorySegment> longRecMem; + if (channelIDs.isEmpty()) { + // only long records + longRecMem = this.mergeReadMemory; + mergeReadMemory = Collections.emptyList(); + } + else { + int maxMergedStreams = Math.min(this.maxFanIn, channelIDs.size()); + + int pagesPerStream = Math.max( + minNumWriteBuffers, + Math.min(maxNumWriteBuffers, this.mergeReadMemory.size() / 2 / maxMergedStreams)); + + int totalMergeReadMemory = maxMergedStreams * pagesPerStream; + + // grab the merge memory + mergeReadMemory = new ArrayList<>(totalMergeReadMemory); + for (int i = 0; i < totalMergeReadMemory; i++) { + mergeReadMemory.add(this.mergeReadMemory.get(i)); + } + + // the remainder of the memory goes to the long record sorter + longRecMem = new ArrayList<>(); + for (int i = totalMergeReadMemory; i < this.mergeReadMemory.size(); i++) { + longRecMem.add(this.mergeReadMemory.get(i)); + } + } + + LOG.debug("Sorting keys for large records."); + largeRecords = largeRecordHandler.finishWriteAndSortKeys(longRecMem); + } + else { + mergeReadMemory = this.mergeReadMemory; + } + + // merge channels until sufficient file handles are available + while (isRunning() && channelIDs.size() > this.maxFanIn) { + channelIDs = mergeChannelList(channelIDs, mergeReadMemory, this.writeMemory); + } + + // from here on, we won't write again + this.memManager.release(this.writeMemory); + this.writeMemory.clear(); + + // check if we have spilled some data at all + if (channelIDs.isEmpty()) { + if (largeRecords == null) { + this.dispatcher.sendResult(EmptyMutableObjectIterator.get()); + } else { + this.dispatcher.sendResult(largeRecords); + } + } + else { + LOG.debug("Beginning final merge."); + + // allocate the memory for the final merging step + List<List<MemorySegment>> readBuffers = new ArrayList<>(channelIDs.size()); + + // allocate the read memory and register it to be released + getSegmentsForReaders(readBuffers, mergeReadMemory, channelIDs.size()); + + // get the readers and register them to be released + this.dispatcher.sendResult(getMergingIterator(channelIDs, readBuffers, + new ArrayList<>(channelIDs.size()), largeRecords)); + } + + // done + LOG.debug("Spilling and merging thread done."); + } + + private void mergeInMemory( + Queue<CircularElement<E>> cache, + MutableObjectIterator<E> largeRecords) throws IOException { + // operates on in-memory buffers only + LOG.debug("Initiating in memory merge."); + + List<MutableObjectIterator<E>> iterators = new ArrayList<>(cache.size() + 1); + + // iterate buffers and collect a set of iterators + for (CircularElement<E> cached : cache) { + // note: the yielded iterator only operates on the buffer heap (and disregards the stack) + iterators.add(cached.getBuffer().getIterator()); + } + + if (largeRecords != null) { + iterators.add(largeRecords); + } + + // release the remaining sort-buffers + LOG.debug("Releasing unused sort-buffer memory."); + disposeSortBuffers(true); + + // set lazy iterator + this.dispatcher.sendResult(iterators.isEmpty() ? EmptyMutableObjectIterator.get() : + iterators.size() == 1 ? iterators.get(0) : + new MergeIterator<>(iterators, this.comparator)); + } + + private List<ChannelWithBlockCount> startSpilling(Queue<CircularElement<E>> cache) throws IOException, InterruptedException { + final FileIOChannel.Enumerator enumerator = this.ioManager.createChannelEnumerator(); + List<ChannelWithBlockCount> channelIDs = new ArrayList<>(); + + // loop as long as the thread is marked alive and we do not see the final element + CircularElement<E> element; Review comment: This can go in the `while()` loop now and become `final`. ########## File path: flink-runtime/src/main/java/org/apache/flink/runtime/operators/sort/SpillingThread.java ########## @@ -0,0 +1,575 @@ +/* + * 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.TypeSerializer; +import org.apache.flink.core.memory.MemorySegment; +import org.apache.flink.runtime.io.disk.ChannelReaderInputViewIterator; +import org.apache.flink.runtime.io.disk.iomanager.BlockChannelReader; +import org.apache.flink.runtime.io.disk.iomanager.BlockChannelWriter; +import org.apache.flink.runtime.io.disk.iomanager.ChannelReaderInputView; +import org.apache.flink.runtime.io.disk.iomanager.ChannelWriterOutputView; +import org.apache.flink.runtime.io.disk.iomanager.FileIOChannel; +import org.apache.flink.runtime.io.disk.iomanager.IOManager; +import org.apache.flink.runtime.memory.MemoryManager; +import org.apache.flink.runtime.util.EmptyMutableObjectIterator; +import org.apache.flink.util.MutableObjectIterator; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import javax.annotation.Nullable; + +import java.io.IOException; +import java.util.ArrayDeque; +import java.util.ArrayList; +import java.util.Collections; +import java.util.Iterator; +import java.util.List; +import java.util.Queue; + +import static org.apache.flink.runtime.operators.sort.CircularElement.EOF_MARKER; +import static org.apache.flink.runtime.operators.sort.CircularElement.SPILLING_MARKER; +import static org.apache.flink.util.Preconditions.checkNotNull; + +/** + * The thread that handles the spilling of intermediate results and sets up the merging. It also merges the + * channels until sufficiently few channels remain to perform the final streamed merge. + */ +final class SpillingThread<E> extends ThreadBase<E> { + /** + * An interface for injecting custom behaviour for spilling and merging phases. + */ + interface SpillingBehaviour<E> { + default void open() {} + + default void close() {} + + /** + * A method that allows adjusting the spilling phase. We can inject e.g. combining the elements while spilling. + */ + void spillBuffer( + CircularElement<E> element, + ChannelWriterOutputView output, + LargeRecordHandler<E> largeRecordHandler) throws IOException; + + /** + * A method that allows adjusting the merging phase. We can inject e.g. combining the spilled elements. + */ + void mergeRecords(MergeIterator<E> mergeIterator, ChannelWriterOutputView output) throws IOException; + } + + /** Logging. */ + private static final Logger LOG = LoggerFactory.getLogger(SpillingThread.class); + + private final MemoryManager memManager; // memory manager to release memory + + private final IOManager ioManager; // I/O manager to create channels + + private final TypeSerializer<E> serializer; // The serializer for the data type + + private final TypeComparator<E> comparator; // The comparator that establishes the order relation. + + private final List<MemorySegment> writeMemory; // memory segments for writing + + private final List<MemorySegment> mergeReadMemory; // memory segments for sorting/reading + + private final int maxFanIn; + + private final SpillChannelManager spillChannelManager; + + private final LargeRecordHandler<E> largeRecordHandler; + + private final SpillingBehaviour<E> spillingBehaviour; + + private boolean spillingBehaviourOpened = false; + + private final int minNumWriteBuffers; + + private final int maxNumWriteBuffers; + + SpillingThread( + @Nullable ExceptionHandler<IOException> exceptionHandler, + StageMessageDispatcher<E> dispatcher, + MemoryManager memManager, + IOManager ioManager, + TypeSerializer<E> serializer, + TypeComparator<E> comparator, + List<MemorySegment> sortReadMemory, + List<MemorySegment> writeMemory, + int maxNumFileHandles, + SpillChannelManager spillingChannelManager, + @Nullable LargeRecordHandler<E> largeRecordHandler, + SpillingBehaviour<E> spillingBehaviour, + int minNumWriteBuffers, + int maxNumWriteBuffers) { + super(exceptionHandler, "SortMerger spilling thread", dispatcher); + this.memManager = checkNotNull(memManager); + this.ioManager = checkNotNull(ioManager); + this.serializer = checkNotNull(serializer); + this.comparator = checkNotNull(comparator); + this.mergeReadMemory = checkNotNull(sortReadMemory); + this.writeMemory = checkNotNull(writeMemory); + this.maxFanIn = maxNumFileHandles; + this.spillChannelManager = checkNotNull(spillingChannelManager); + this.largeRecordHandler = largeRecordHandler; + this.spillingBehaviour = checkNotNull(spillingBehaviour); + this.minNumWriteBuffers = minNumWriteBuffers; + this.maxNumWriteBuffers = maxNumWriteBuffers; + } + + /** + * Entry point of the thread. + */ Review comment: Probably add the `@Override`. ########## File path: flink-runtime/src/main/java/org/apache/flink/runtime/operators/sort/SortingThread.java ########## @@ -0,0 +1,85 @@ +/* + * 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.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import javax.annotation.Nullable; + +import java.io.IOException; + +import static org.apache.flink.runtime.operators.sort.CircularElement.EOF_MARKER; +import static org.apache.flink.runtime.operators.sort.CircularElement.SPILLING_MARKER; + +/** + * The thread that sorts filled buffers. + */ +class SortingThread<E> extends ThreadBase<E> { + + /** Logging. */ + private static final Logger LOG = LoggerFactory.getLogger(SortingThread.class); + + private final IndexedSorter sorter; + + /** + * Creates a new sorting thread. + * + * @param exceptionHandler The exception handler to call for all exceptions. + * @param dispatcher The queues used to pass buffers between the threads. + */ + public SortingThread( + @Nullable ExceptionHandler<IOException> exceptionHandler, + StageMessageDispatcher<E> dispatcher) { + super(exceptionHandler, "SortMerger sorting thread", dispatcher); + + // members + this.sorter = new QuickSort(); + } + + /** + * Entry point of the thread. + */ Review comment: I would also add the `@Override` annotation. ########## File path: flink-runtime/src/main/java/org/apache/flink/runtime/operators/sort/ReadingThread.java ########## @@ -0,0 +1,75 @@ +/* + * 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.util.MutableObjectIterator; + +import javax.annotation.Nullable; + +import java.io.IOException; + +import static org.apache.flink.util.Preconditions.checkNotNull; + +/** + * The thread that consumes the input data and puts it into a buffer that will be sorted. + */ +final class ReadingThread<E> extends ThreadBase<E> { + + /** The input channels to read from. */ + private final MutableObjectIterator<E> reader; + + /** The object into which the thread reads the data from the input. */ + private final E readTarget; + + private final SorterInputGateway<E> sorterGateway; + + /** + * Creates a new reading thread. + * + * @param exceptionHandler The exception handler to call for all exceptions. + * @param reader The reader to pull the data from. + * @param dispatcher The queues used to pass buffers between the threads. + */ + ReadingThread( + @Nullable ExceptionHandler<IOException> exceptionHandler, + MutableObjectIterator<E> reader, + StageMessageDispatcher<E> dispatcher, + @Nullable LargeRecordHandler<E> largeRecordsHandler, + @Nullable E readTarget, + long startSpillingBytes) { + super(exceptionHandler, "SortMerger Reading Thread", dispatcher); + + // members + this.sorterGateway = new SorterInputGateway<>(dispatcher, largeRecordsHandler, startSpillingBytes); + this.reader = checkNotNull(reader); + this.readTarget = readTarget; + } + Review comment: Probably add `@Override`. ---------------------------------------------------------------- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. For queries about this service, please contact Infrastructure at: [email protected]
