dawidwys commented on a change in pull request #13357:
URL: https://github.com/apache/flink/pull/13357#discussion_r492174885



##########
File path: 
flink-runtime/src/main/java/org/apache/flink/runtime/operators/sort/SpillingThread.java
##########
@@ -0,0 +1,601 @@
+/*
+ * 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 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;
+
+/**
+ * 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.
+        * @param <E>
+        */
+       interface SpillingBehaviour<E> {
+               default void open() {}
+
+               default void close() {}
+
+               /**
+                * An 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;
+
+               /**
+                * An 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;
+
+       /**
+        * Creates the spilling thread.
+        * @param exceptionHandler The exception handler to call for all 
exceptions.
+        * @param dispatcher The queues used to pass buffers between the 
threads.
+        * @param memManager The memory manager used to allocate buffers for 
the readers and writers.
+        * @param ioManager The I/I manager used to instantiate readers and 
writers from.
+        * @param serializer
+        * @param comparator
+        * @param sortReadMemory
+        * @param writeMemory
+        * @param maxNumFileHandles
+        * @param spillingChannelManager
+        * @param largeRecordHandler
+        * @param minNumWriteBuffers
+        * @param maxNumWriteBuffers
+        */
+       SpillingThread(
+                       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,
+                       LargeRecordHandler<E> largeRecordHandler,
+                       SpillingBehaviour<E> spillingBehaviour,
+                       int minNumWriteBuffers,
+                       int maxNumWriteBuffers) {
+               super(exceptionHandler, "SortMerger spilling thread", 
dispatcher);
+               this.memManager = memManager;
+               this.ioManager = ioManager;
+               this.serializer = serializer;
+               this.comparator = comparator;
+               this.mergeReadMemory = sortReadMemory;
+               this.writeMemory = writeMemory;
+               this.maxFanIn = maxNumFileHandles;
+               this.spillChannelManager = spillingChannelManager;
+               this.largeRecordHandler = largeRecordHandler;
+               this.spillingBehaviour = spillingBehaviour;
+               this.minNumWriteBuffers = minNumWriteBuffers;
+               this.maxNumWriteBuffers = maxNumWriteBuffers;
+       }
+
+       /**
+        * Entry point of the thread.
+        */
+       public void go() throws IOException {
+
+               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 
IOException {
+               CircularElement<E> element;
+
+               // ------------------- In-Memory Cache ------------------------
+               // fill cache
+               while (isRunning()) {
+                       // take next element from queue
+                       try {
+                               element = this.dispatcher.take(SortStage.SPILL);

Review comment:
       I though about it for a while after your comment. Even though we are 
slightly changing the behaviour I think it is actually fine. There are two 
situation where the thread can be interrupted.
   
   1. When it is being closed via `ThreadBase#shutdown`. In this scenario both 
in the old and in the new code we are exiting the thread. In the new code we 
are throwing an exception, but it will be discarded in the 
`ThreadBase#internalExceptionHandler`.
   2. If the thread is interrupted from outside. First of all I cannot think of 
exact scenario when this can happen. Nevertheless, I am pretty sure it should 
not happen during a successful processing and actually I doubt it is safe to 
continue on processing.




----------------------------------------------------------------
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:
us...@infra.apache.org


Reply via email to