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



##########
File path: 
flink-runtime/src/main/java/org/apache/flink/runtime/operators/sort/StageRunner.java
##########
@@ -0,0 +1,73 @@
+/*
+ * 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;
+
+/**
+ * An interface for different stages of the sorting process. Different stages 
can communicate via
+ * the {@link StageMessageDispatcher}.
+ */
+public interface StageRunner extends AutoCloseable {
+       /**
+        * Starts the stage.
+        */
+       void start();
+
+       /**
+        * A marker interface for sending messages to different stages.
+        */
+       enum SortStage {
+               READ,
+               SPILL,

Review comment:
       I think it might be better to exchange the order of `SORT` and `SPILL` 
since sorting happens before spilling.

##########
File path: 
flink-runtime/src/main/java/org/apache/flink/runtime/operators/sort/CircularQueues.java
##########
@@ -0,0 +1,87 @@
+/*
+ * 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 org.apache.flink.util.WrappingRuntimeException;
+
+import java.util.concurrent.BlockingQueue;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.LinkedBlockingQueue;
+
+/**
+ * Collection of queues that are used for the communication between the 
threads.
+ */
+final class CircularQueues<E> implements StageRunner.StageMessageDispatcher<E> 
{
+       final BlockingQueue<CircularElement<E>> empty;
+       final BlockingQueue<CircularElement<E>> sort;
+       final BlockingQueue<CircularElement<E>> spill;
+       /**
+        * The iterator to be returned by the sort-merger. This variable is 
null, while receiving and merging is still in
+        * progress and it will be set once we have &lt; merge factor sorted 
sub-streams that will then be streamed sorted.
+        */
+       private final CompletableFuture<MutableObjectIterator<E>> 
iteratorFuture = new CompletableFuture<>();
+
+       public CircularQueues() {
+               this.empty = new LinkedBlockingQueue<>();
+               this.sort = new LinkedBlockingQueue<>();
+               this.spill = new LinkedBlockingQueue<>();
+       }
+
+       private BlockingQueue<CircularElement<E>> 
getQueue(StageRunner.SortStage stage) {
+               switch (stage) {
+                       case READ:
+                               return empty;
+                       case SPILL:

Review comment:
       Similar to the one comment in `SortStage` I would prefer to move the 
`SORT` case before the `SPILL` case.

##########
File path: 
flink-runtime/src/main/java/org/apache/flink/runtime/operators/sort/SorterInputGateway.java
##########
@@ -0,0 +1,132 @@
+/*
+ * 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.runtime.operators.sort.StageRunner.SortStage;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+
+/**
+ * A gateway for writing records into the sort/merge process.
+ */
+final class SorterInputGateway<E> {
+       /** Logging. */
+       private static final Logger LOG = 
LoggerFactory.getLogger(SorterInputGateway.class);
+       private final LargeRecordHandler<E> largeRecords;
+       /** The object into which the thread reads the data from the input. */
+       private final StageRunner.StageMessageDispatcher<E> dispatcher;
+       private long bytesUntilSpilling;
+       private CircularElement<E> currentBuffer;
+
+       /**
+        * Creates a new gateway for pushing records into the sorter.
+        *
+        * @param dispatcher The queues used to pass buffers between the 
threads.
+        */
+       SorterInputGateway(
+                       StageRunner.StageMessageDispatcher<E> dispatcher,
+                       LargeRecordHandler<E> largeRecordsHandler,
+                       long startSpillingBytes) {
+
+               // members
+               this.bytesUntilSpilling = startSpillingBytes;
+               this.largeRecords = largeRecordsHandler;
+               this.dispatcher = dispatcher;
+
+               if (bytesUntilSpilling < 1) {
+                       this.dispatcher.send(SortStage.SORT, 
CircularElement.spillingMarker());
+               }
+       }
+
+       /**
+        * Writes the given record for sorting.
+        */
+       public void writeRecord(E record) throws IOException {
+
+               if (currentBuffer == null) {
+                       this.currentBuffer = 
this.dispatcher.take(SortStage.READ);

Review comment:
       One concern about infinite waiting for next buffer would be that if the 
spilling thread throws exception, the current thread (I think it should be the 
task main thread) would not get notified and would be keep waiting here. In 
this case the task would not be able to failover.

##########
File path: 
flink-runtime/src/main/java/org/apache/flink/runtime/operators/sort/ExternalSorterBuilder.java
##########
@@ -0,0 +1,419 @@
+/*
+ * 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.GroupCombineFunction;
+import org.apache.flink.api.common.typeutils.TypeComparator;
+import org.apache.flink.api.common.typeutils.TypeSerializer;
+import org.apache.flink.configuration.AlgorithmOptions;
+import org.apache.flink.configuration.ConfigConstants;
+import org.apache.flink.configuration.Configuration;
+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.memory.MemoryAllocationException;
+import org.apache.flink.runtime.memory.MemoryManager;
+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.ArrayList;
+import java.util.Iterator;
+import java.util.List;
+
+import static org.apache.flink.util.Preconditions.checkNotNull;
+
+/**
+ * A builder for an {@link ExternalSorter}. It can construct either a 
pull-based sorter if provided with an input
+ * iterator via {@link #build(MutableObjectIterator)} or a push-based one via 
{@link #build()}.
+ */
+public final class ExternalSorterBuilder<T> {
+
+       /** Logging. */
+       private static final Logger LOG = 
LoggerFactory.getLogger(ExternalSorter.class);
+
+       /** Fix length records with a length below this threshold will be 
in-place sorted, if possible. */
+       private static final int THRESHOLD_FOR_IN_PLACE_SORTING = 32;
+
+       /** The minimal number of buffers to use by the writers. */
+       private static final int MIN_NUM_WRITE_BUFFERS = 2;
+
+       /** The maximal number of buffers to use by the writers. */
+       private static final int MAX_NUM_WRITE_BUFFERS = 4;
+
+       /** The minimum number of segments that are required for the sort to 
operate. */
+       private static final int MIN_NUM_SORT_MEM_SEGMENTS = 10;
+
+       private final MemoryManager memoryManager;
+       private final AbstractInvokable parentTask;
+       private final TypeSerializer<T> serializer;
+       private final TypeComparator<T> comparator;
+       private InMemorySorterFactory<T> inMemorySorterFactory;
+       private int maxNumFileHandles = 
AlgorithmOptions.SPILLING_MAX_FAN.defaultValue();
+       private boolean objectReuseEnabled = false;
+       private boolean handleLargeRecords = 
ConfigConstants.DEFAULT_USE_LARGE_RECORD_HANDLER;
+       private double memoryFraction = 1.0;
+       private int numSortBuffers = -1;
+       private double startSpillingFraction = 
AlgorithmOptions.SORT_SPILLING_THRESHOLD.defaultValue();
+       private IOManager ioManager;
+       private boolean noSpillingMemory = true;
+       private GroupCombineFunction<T, T> combineFunction;
+       private Configuration udfConfig;
+       private List<MemorySegment> memorySegments = null;
+
+       ExternalSorterBuilder(
+                       MemoryManager memoryManager,
+                       AbstractInvokable parentTask,
+                       TypeSerializer<T> serializer,
+                       TypeComparator<T> comparator) {
+               this.memoryManager = memoryManager;
+               this.parentTask = parentTask;
+               this.serializer = serializer;
+               this.comparator = comparator;
+               this.inMemorySorterFactory = new DefaultInMemorySorterFactory<>(
+                       serializer,
+                       comparator,
+                       THRESHOLD_FOR_IN_PLACE_SORTING);
+       }
+
+       public ExternalSorterBuilder<T> maxNumFileHandles(int 
maxNumFileHandles) {
+               if (maxNumFileHandles < 2) {
+                       throw new IllegalArgumentException("Merger cannot work 
with less than two file handles.");
+               }
+               this.maxNumFileHandles = maxNumFileHandles;
+               return this;
+       }
+
+       public ExternalSorterBuilder<T> objectReuse(boolean enabled) {
+               this.objectReuseEnabled = enabled;
+               return this;
+       }
+
+       public ExternalSorterBuilder<T> largeRecords(boolean enabled) {
+               this.handleLargeRecords = enabled;
+               return this;
+       }
+
+       public ExternalSorterBuilder<T> enableSpilling(IOManager ioManager) {
+               this.noSpillingMemory = false;
+               this.ioManager = checkNotNull(ioManager);
+               return this;
+       }
+
+       public ExternalSorterBuilder<T> enableSpilling(IOManager ioManager, 
double startSpillingFraction) {
+               this.startSpillingFraction = startSpillingFraction;
+               return enableSpilling(ioManager);
+       }
+
+       public ExternalSorterBuilder<T> memoryFraction(double fraction) {
+               this.memoryFraction = fraction;
+               return this;
+       }
+
+       public ExternalSorterBuilder<T> memory(List<MemorySegment> 
memorySegments) {
+               this.memorySegments = checkNotNull(memorySegments);
+               return this;
+       }
+
+       public ExternalSorterBuilder<T> sortBuffers(int numSortBuffers) {
+               this.numSortBuffers = numSortBuffers;
+               return this;
+       }
+
+       public ExternalSorterBuilder<T> withCombiner(GroupCombineFunction<T, T> 
combineFunction, Configuration udfConfig) {
+               this.combineFunction = checkNotNull(combineFunction);
+               this.udfConfig = checkNotNull(udfConfig);
+               return this;
+       }
+
+       public ExternalSorterBuilder<T> withCombiner(GroupCombineFunction<T, T> 
combineFunction) {
+               this.combineFunction = checkNotNull(combineFunction);
+               this.udfConfig = new Configuration();
+               return this;
+       }
+
+       public ExternalSorterBuilder<T> sorterFactory(InMemorySorterFactory<T> 
sorterFactory) {
+               this.inMemorySorterFactory = checkNotNull(sorterFactory);
+               return this;
+       }
+
+       /**
+        * Creates a pull-based {@link Sorter}. The {@link 
Sorter#getIterator()} will return when all the records from
+        * the given input are consumed. Will spawn three threads: read, sort, 
spill.
+        */
+       public ExternalSorter<T> build(MutableObjectIterator<T> input) throws 
MemoryAllocationException {
+               return doBuild((exceptionHandler, dispatcher, 
largeRecordHandler, startSpillingBytes) ->
+                       new ReadingThread<>(
+                               exceptionHandler,
+                               input,
+                               dispatcher,
+                               largeRecordHandler,
+                               serializer.createInstance(),
+                               startSpillingBytes
+                       )
+               );
+       }
+
+       /**
+        * Creates a push-based {@link PushSorter}. The {@link 
PushSorter#getIterator()} will return when the
+        * {@link PushSorter#finishReading()} is called. Will spawn two 
threads: sort, spill.
+        */
+       public PushSorter<T> build() throws MemoryAllocationException {
+               PushFactory<T> pushFactory = new PushFactory<>();
+               ExternalSorter<T> tExternalSorter = doBuild(pushFactory);
+
+               return new PushSorter<T>() {
+                       private final SorterInputGateway<T> recordProducer = 
pushFactory.sorterInputGateway;
+
+                       @Override
+                       public void writeRecord(T record) throws IOException {
+                               recordProducer.writeRecord(record);
+                       }
+
+                       @Override
+                       public void finishReading() {
+                               recordProducer.finishReading();
+                       }
+
+                       @Override
+                       public MutableObjectIterator<T> getIterator() throws 
InterruptedException {
+                               return tExternalSorter.getIterator();
+                       }
+
+                       @Override
+                       public void close() throws IOException {
+                               tExternalSorter.close();
+                       }
+               };
+       }
+
+       @FunctionalInterface
+       private interface ReadingStageFactory<E> {
+               @Nullable StageRunner createReadingStage(
+                       ExceptionHandler<IOException> exceptionHandler,
+                       StageRunner.StageMessageDispatcher<E> dispatcher,
+                       LargeRecordHandler<E> largeRecordHandler,
+                       long startSpillingBytes);
+       }
+
+       private static final class PushFactory<E> implements 
ReadingStageFactory<E> {
+               private SorterInputGateway<E> sorterInputGateway;
+
+               @Override
+               public StageRunner createReadingStage(
+                               ExceptionHandler<IOException> exceptionHandler,
+                               StageRunner.StageMessageDispatcher<E> 
dispatcher,
+                               LargeRecordHandler<E> largeRecordHandler,
+                               long startSpillingBytes) {
+                       sorterInputGateway = new SorterInputGateway<>(
+                               dispatcher,
+                               largeRecordHandler,
+                               startSpillingBytes
+                       );
+                       return null;
+               }
+       }
+
+       private ExternalSorter<T> doBuild(ReadingStageFactory<T> 
readingStageFactory) throws MemoryAllocationException {
+
+               final List<MemorySegment> memory;
+               if (this.memorySegments != null) {
+                       memory = this.memorySegments;
+               } else {
+                       memory = memoryManager.allocatePages(
+                               parentTask,
+                               
memoryManager.computeNumberOfPages(memoryFraction));
+               }
+
+               // adjust the memory quotas to the page size
+               final int numPagesTotal = memory.size();
+
+               if (numPagesTotal < MIN_NUM_WRITE_BUFFERS + 
MIN_NUM_SORT_MEM_SEGMENTS) {
+                       throw new IllegalArgumentException("Too little memory 
provided to sorter to perform task. " +
+                               "Required are at least " + 
(MIN_NUM_WRITE_BUFFERS + MIN_NUM_SORT_MEM_SEGMENTS) +
+                               " pages. Current page size is " + 
memoryManager.getPageSize() + " bytes.");
+               }
+
+               // determine how many buffers to use for writing
+               final int numWriteBuffers;
+               final int numLargeRecordBuffers;
+
+               if (noSpillingMemory && !handleLargeRecords) {
+                       numWriteBuffers = 0;
+                       numLargeRecordBuffers = 0;
+               } else {
+                       int numConsumers = (noSpillingMemory ? 0 : 1) + 
(handleLargeRecords ? 2 : 0);
+
+                       // determine how many buffers we have when we do a full 
mere with maximal fan-in

Review comment:
       Typo `mere` -> `merge`




----------------------------------------------------------------
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