Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

[FLINK-35094] SinkTestSuiteBase.testScaleDown is hanging for 1.20-SNAPSHOT #98

Merged
merged 1 commit into from
Apr 15, 2024
Merged
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
@@ -0,0 +1,30 @@
/*
* 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.core.execution;

/**
* This is a copy of {@link CheckpointingMode} from flink-core module introduced in Flink 1.20. We
* need it here to make E2E Sink tests compatible with earlier releases. Could be removed together
* with dropping support of Flink 1.19.
*/
public enum CheckpointingMode {
EXACTLY_ONCE,
AT_LEAST_ONCE;

private CheckpointingMode() {}
}
Original file line number Diff line number Diff line change
Expand Up @@ -72,7 +72,8 @@ public abstract class ElasticsearchSinkE2ECaseBase<T extends Comparable<T>>
.bindWithFlinkContainer(flink.getFlinkContainers().getJobManager())
.build();

@Override
/** Could be removed together with dropping support of Flink 1.19. */
@Deprecated
protected void checkResultWithSemantic(
ExternalSystemDataReader<T> reader, List<T> testData, CheckpointingMode semantic)
throws Exception {
Expand All @@ -93,9 +94,46 @@ protected void checkResultWithSemantic(
READER_RETRY_ATTEMPTS);
}

protected void checkResultWithSemantic(
ExternalSystemDataReader<T> reader,
List<T> testData,
org.apache.flink.core.execution.CheckpointingMode semantic)
throws Exception {
waitUntilCondition(
() -> {
try {
List<T> result = reader.poll(Duration.ofMillis(READER_TIMEOUT));
assertThat(sort(result).iterator())
.matchesRecordsFromSource(
Collections.singletonList(sort(testData)),
convertFromCheckpointingMode(semantic));
return true;
} catch (Throwable t) {
LOG.warn("Polled results not as expected", t);
return false;
}
},
5000,
READER_RETRY_ATTEMPTS);
}

private List<T> sort(List<T> list) {
return list.stream().sorted().collect(Collectors.toList());
}

/** Could be removed together with dropping support of Flink 1.19. */
@Deprecated
private static org.apache.flink.streaming.api.CheckpointingMode convertFromCheckpointingMode(
org.apache.flink.core.execution.CheckpointingMode semantic) {
switch (semantic) {
case EXACTLY_ONCE:
return org.apache.flink.streaming.api.CheckpointingMode.EXACTLY_ONCE;
case AT_LEAST_ONCE:
return org.apache.flink.streaming.api.CheckpointingMode.AT_LEAST_ONCE;
default:
throw new IllegalArgumentException("Unsupported semantic: " + semantic);
}
}

abstract String getElasticsearchContainerName();
}