Skip to content

Commit

Permalink
[FLINK-35094] Apply fix for Opensearch2
Browse files Browse the repository at this point in the history
  • Loading branch information
snuyanzin committed Apr 16, 2024
1 parent 1e97abe commit 68d109c
Show file tree
Hide file tree
Showing 2 changed files with 72 additions and 1 deletion.
Original file line number Diff line number Diff line change
@@ -0,0 +1,32 @@
/*
* 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;

import org.apache.flink.streaming.tests.OpensearchSinkE2ECase;

/**
* This is a copy of {@link CheckpointingMode} from flink-core module introduced in Flink 1.20. We
* need it here to make {@link OpensearchSinkE2ECase} 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 @@ -85,7 +85,8 @@ public OpensearchSinkE2ECase() throws Exception {}
.toUri()
.toURL()));

@Override
/** Could be removed together with dropping support of Flink 1.19. */
@Deprecated
protected void checkResultWithSemantic(
ExternalSystemDataReader<ComparableTuple2<Integer, String>> reader,
List<ComparableTuple2<Integer, String>> testData,
Expand All @@ -109,8 +110,46 @@ protected void checkResultWithSemantic(
READER_RETRY_ATTEMPTS);
}

protected void checkResultWithSemantic(
ExternalSystemDataReader<ComparableTuple2<Integer, String>> reader,
List<ComparableTuple2<Integer, String>> testData,
org.apache.flink.core.execution.CheckpointingMode semantic)
throws Exception {
waitUntilCondition(
() -> {
try {
List<ComparableTuple2<Integer, String>> 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 static <T extends Comparable<T>> List<T> sort(List<T> list) {
Collections.sort(list);
return list;
}

/** 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);
}
}
}

0 comments on commit 68d109c

Please sign in to comment.