WencongLiu commented on code in PR #24398: URL: https://github.com/apache/flink/pull/24398#discussion_r1510733720
########## flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/sort/AbstractSortPartitionOperator.java: ########## @@ -0,0 +1,229 @@ +/* + * 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.streaming.api.operators.sort; + +import org.apache.flink.annotation.Internal; +import org.apache.flink.api.common.InvalidProgramException; +import org.apache.flink.api.common.operators.Keys; +import org.apache.flink.api.common.operators.Order; +import org.apache.flink.api.common.typeinfo.AtomicType; +import org.apache.flink.api.common.typeinfo.TypeInformation; +import org.apache.flink.api.common.typeinfo.Types; +import org.apache.flink.api.common.typeutils.CompositeType; +import org.apache.flink.api.common.typeutils.TypeComparator; +import org.apache.flink.api.common.typeutils.TypeSerializer; +import org.apache.flink.api.java.functions.KeySelector; +import org.apache.flink.api.java.typeutils.TypeExtractor; +import org.apache.flink.configuration.AlgorithmOptions; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.core.memory.ManagedMemoryUseCase; +import org.apache.flink.runtime.memory.MemoryAllocationException; +import org.apache.flink.runtime.operators.sort.ExternalSorter; +import org.apache.flink.runtime.operators.sort.PushSorter; +import org.apache.flink.streaming.api.operators.AbstractStreamOperator; +import org.apache.flink.streaming.api.operators.BoundedOneInput; +import org.apache.flink.streaming.api.operators.OneInputStreamOperator; +import org.apache.flink.streaming.api.operators.OperatorAttributes; +import org.apache.flink.streaming.api.operators.OperatorAttributesBuilder; +import org.apache.flink.streaming.runtime.tasks.StreamTask; + +/** + * The {@link AbstractSortPartitionOperator} is the base class of sort partition operator, which + * provides shared construction methods and utility functions. + * + * @param <INPUT_TYPE> The type of input record. + * @param <SORT_TYPE> The type used to sort the records, which may be different from the INPUT_TYPE. + * For example, if the input record is sorted according to the selected key by {@link + * KeySelector}, the selected key should also be written to {@link ExternalSorter} with the + * input record to avid repeated key selections. In this case, the type used to sort the records + * will be a tuple containing both the selected key and record. + */ +@Internal +@SuppressWarnings("unchecked") +public abstract class AbstractSortPartitionOperator<INPUT_TYPE, SORT_TYPE> + extends AbstractStreamOperator<INPUT_TYPE> + implements OneInputStreamOperator<INPUT_TYPE, INPUT_TYPE>, BoundedOneInput { + + /** The default manage memory weight of sort partition operator. */ + public static final int DEFAULT_MANAGE_MEMORY_WEIGHT = 128; + + /** The type information of input records. */ + protected final TypeInformation<INPUT_TYPE> inputType; + + /** The type information used to sort the records. */ + protected final TypeInformation<SORT_TYPE> sortType; Review Comment: I've refactored the implementations of SortPartition API on DataStream and KeyedStream. PTAL. -- 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. To unsubscribe, e-mail: issues-unsubscr...@flink.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org