Skip to content
Open
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
Expand Up @@ -30,6 +30,7 @@
import org.apache.flink.api.common.typeutils.TypeSerializer;
import org.apache.flink.api.java.functions.KeySelector;
import org.apache.flink.api.java.tuple.Tuple2;
import org.apache.flink.api.java.tuple.Tuple3;
import org.apache.flink.api.java.typeutils.TypeExtractor;
import org.apache.flink.configuration.AlgorithmOptions;
import org.apache.flink.configuration.Configuration;
Expand All @@ -44,7 +45,6 @@
import org.apache.flink.streaming.api.operators.OperatorAttributes;
import org.apache.flink.streaming.api.operators.OperatorAttributesBuilder;
import org.apache.flink.streaming.api.operators.Output;
import org.apache.flink.streaming.api.operators.TimestampedCollector;
import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
import org.apache.flink.streaming.runtime.tasks.StreamTask;
import org.apache.flink.util.MutableObjectIterator;
Expand Down Expand Up @@ -86,10 +86,10 @@ public class SortPartitionOperator<INPUT> extends AbstractStreamOperator<INPUT>
private final int positionSortField;

/** The sorter to sort record if the record is not sorted by {@link KeySelector}. */
private PushSorter<INPUT> recordSorter = null;
private PushSorter<Tuple2<INPUT, Long>> recordSorter = null;

/** The sorter to sort record if the record is sorted by {@link KeySelector}. */
private PushSorter<Tuple2<?, INPUT>> recordSorterForKeySelector = null;
private PushSorter<Tuple3<?, INPUT, Long>> recordSorterForKeySelector = null;

public SortPartitionOperator(
TypeInformation<INPUT> inputType, int positionSortField, Order sortOrder) {
Expand Down Expand Up @@ -131,25 +131,27 @@ protected void setup(
super.setup(containingTask, config, output);
ExecutionConfig executionConfig = containingTask.getEnvironment().getExecutionConfig();
if (sortFieldSelector != null) {
TypeInformation<Tuple2<?, INPUT>> sortTypeInfo =
TypeInformation<Tuple3<?, INPUT, Long>> sortTypeInfo =
Types.TUPLE(
TypeExtractor.getKeySelectorTypes(sortFieldSelector, inputType),
inputType);
inputType,
Types.LONG);
recordSorterForKeySelector =
getSorter(
sortTypeInfo.createSerializer(executionConfig.getSerializerConfig()),
((CompositeType<Tuple2<?, INPUT>>) sortTypeInfo)
((CompositeType<Tuple3<?, INPUT, Long>>) sortTypeInfo)
.createComparator(
getSortFieldIndex(),
getSortOrderIndicator(),
0,
executionConfig),
containingTask);
} else {
TypeInformation<Tuple2<INPUT, Long>> sortTypeInfo = Types.TUPLE(inputType, Types.LONG);
recordSorter =
getSorter(
inputType.createSerializer(executionConfig.getSerializerConfig()),
((CompositeType<INPUT>) inputType)
sortTypeInfo.createSerializer(executionConfig.getSerializerConfig()),
((CompositeType<Tuple2<INPUT, Long>>) sortTypeInfo)
.createComparator(
getSortFieldIndex(),
getSortOrderIndicator(),
Expand All @@ -163,31 +165,33 @@ protected void setup(
public void processElement(StreamRecord<INPUT> element) throws Exception {
if (sortFieldSelector != null) {
recordSorterForKeySelector.writeRecord(
Tuple2.of(sortFieldSelector.getKey(element.getValue()), element.getValue()));
Tuple3.of(
sortFieldSelector.getKey(element.getValue()),
element.getValue(),
element.getTimestamp()));
} else {
recordSorter.writeRecord(element.getValue());
recordSorter.writeRecord(Tuple2.of(element.getValue(), element.getTimestamp()));
}
}

@Override
public void endInput() throws Exception {
TimestampedCollector<INPUT> outputCollector = new TimestampedCollector<>(output);
if (sortFieldSelector != null) {
recordSorterForKeySelector.finishReading();
MutableObjectIterator<Tuple2<?, INPUT>> dataIterator =
MutableObjectIterator<Tuple3<?, INPUT, Long>> dataIterator =
recordSorterForKeySelector.getIterator();
Tuple2<?, INPUT> record = dataIterator.next();
Tuple3<?, INPUT, Long> record = dataIterator.next();
while (record != null) {
outputCollector.collect(record.f1);
output.collect(new StreamRecord<>(record.f1, record.f2));
record = dataIterator.next();
}
recordSorterForKeySelector.close();
} else {
recordSorter.finishReading();
MutableObjectIterator<INPUT> dataIterator = recordSorter.getIterator();
INPUT record = dataIterator.next();
MutableObjectIterator<Tuple2<INPUT, Long>> dataIterator = recordSorter.getIterator();
Tuple2<INPUT, Long> record = dataIterator.next();
while (record != null) {
outputCollector.collect(record);
output.collect(new StreamRecord<>(record.f0, record.f1));
record = dataIterator.next();
}
recordSorter.close();
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -47,12 +47,13 @@ void testSortPartition() throws Exception {
testHarness1 = new OneInputStreamOperatorTestHarness<>(operator1);
Queue<Object> expectedOutput1 = new LinkedList<>();
testHarness1.setup();
testHarness1.processElement(new StreamRecord<>(Tuple2.of(3, "3")));
testHarness1.processElement(new StreamRecord<>(Tuple2.of(1, "1")));
long timestamp = 1L;
testHarness1.processElement(new StreamRecord<>(Tuple2.of(3, "3"), timestamp));
testHarness1.processElement(new StreamRecord<>(Tuple2.of(1, "1"), timestamp));
testHarness1.endInput();
testHarness1.close();
expectedOutput1.add(new StreamRecord<>(Tuple2.of(1, "1")));
expectedOutput1.add(new StreamRecord<>(Tuple2.of(3, "3")));
expectedOutput1.add(new StreamRecord<>(Tuple2.of(1, "1"), timestamp));
expectedOutput1.add(new StreamRecord<>(Tuple2.of(3, "3"), timestamp));
TestHarnessUtil.assertOutputEquals(
"The sort partition result is not correct.",
expectedOutput1,
Expand All @@ -63,12 +64,12 @@ void testSortPartition() throws Exception {
new OneInputStreamOperatorTestHarness<>(operator2);
Queue<Object> expectedOutput2 = new LinkedList<>();
testHarness2.setup();
testHarness2.processElement(new StreamRecord<>(new TestPojo("3", 3)));
testHarness2.processElement(new StreamRecord<>(new TestPojo("1", 1)));
testHarness2.processElement(new StreamRecord<>(new TestPojo("3", 3), timestamp));
testHarness2.processElement(new StreamRecord<>(new TestPojo("1", 1), timestamp));
testHarness2.endInput();
testHarness2.close();
expectedOutput2.add(new StreamRecord<>(new TestPojo("1", 1)));
expectedOutput2.add(new StreamRecord<>(new TestPojo("3", 3)));
expectedOutput2.add(new StreamRecord<>(new TestPojo("1", 1), timestamp));
expectedOutput2.add(new StreamRecord<>(new TestPojo("3", 3), timestamp));
TestHarnessUtil.assertOutputEquals(
"The sort partition result is not correct.",
expectedOutput2,
Expand All @@ -79,12 +80,12 @@ void testSortPartition() throws Exception {
new OneInputStreamOperatorTestHarness<>(operator3);
Queue<Object> expectedOutput3 = new LinkedList<>();
testHarness3.setup();
testHarness3.processElement(new StreamRecord<>(new TestPojo("3", 3)));
testHarness3.processElement(new StreamRecord<>(new TestPojo("1", 1)));
testHarness3.processElement(new StreamRecord<>(new TestPojo("3", 3), timestamp));
testHarness3.processElement(new StreamRecord<>(new TestPojo("1", 1), timestamp));
testHarness3.endInput();
testHarness3.close();
expectedOutput3.add(new StreamRecord<>(new TestPojo("1", 1)));
expectedOutput3.add(new StreamRecord<>(new TestPojo("3", 3)));
expectedOutput3.add(new StreamRecord<>(new TestPojo("1", 1), timestamp));
expectedOutput3.add(new StreamRecord<>(new TestPojo("3", 3), timestamp));
TestHarnessUtil.assertOutputEquals(
"The sort partition result is not correct.",
expectedOutput3,
Expand Down