Skip to content

Commit

Permalink
[FLINK-32857][JUnit5 Migration] Migrate the webmonitor and zookeeper …
Browse files Browse the repository at this point in the history
…packages of flink-runtime module to junit5 (apache#23196)
  • Loading branch information
1996fanrui authored Aug 23, 2023
1 parent 4aebc43 commit 1ae8e40
Show file tree
Hide file tree
Showing 9 changed files with 494 additions and 435 deletions.
Original file line number Diff line number Diff line change
@@ -0,0 +1,70 @@
/*
* 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.rpc;

import java.util.function.Supplier;

/** {@code TestingRpcGateway} is a generic test implementation of {@link RpcGateway}. */
public class TestingRpcGateway implements RpcGateway {

private final Supplier<String> addressSupplier;
private final Supplier<String> hostnameSupplier;

private TestingRpcGateway(Supplier<String> addressSupplier, Supplier<String> hostnameSupplier) {
this.addressSupplier = addressSupplier;
this.hostnameSupplier = hostnameSupplier;
}

@Override
public String getAddress() {
return addressSupplier.get();
}

@Override
public String getHostname() {
return hostnameSupplier.get();
}

public static Builder newBuilder() {
return new Builder();
}

/** {@code Builder} for {@code TestingRpcGateway}. */
public static class Builder {

private Supplier<String> addressSupplier = () -> "address";
private Supplier<String> hostnameSupplier = () -> "hostname";

private Builder() {}

public Builder setAddressSupplier(Supplier<String> addressSupplier) {
this.addressSupplier = addressSupplier;
return this;
}

public Builder setHostnameSupplier(Supplier<String> hostnameSupplier) {
this.hostnameSupplier = hostnameSupplier;
return this;
}

public TestingRpcGateway build() {
return new TestingRpcGateway(addressSupplier, hostnameSupplier);
}
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -28,9 +28,8 @@
import org.apache.flink.runtime.rest.handler.legacy.metrics.VoidMetricFetcher;
import org.apache.flink.runtime.util.TestingFatalErrorHandler;
import org.apache.flink.util.ExecutorUtils;
import org.apache.flink.util.TestLogger;

import org.junit.Test;
import org.junit.jupiter.api.Test;

import java.util.UUID;
import java.util.concurrent.CompletableFuture;
Expand All @@ -39,10 +38,10 @@
import java.util.concurrent.TimeUnit;

/** Tests for the {@link WebMonitorEndpoint}. */
public class WebMonitorEndpointTest extends TestLogger {
class WebMonitorEndpointTest {

@Test
public void cleansUpExpiredExecutionGraphs() throws Exception {
void cleansUpExpiredExecutionGraphs() throws Exception {
final Configuration configuration = new Configuration();
configuration.setString(RestOptions.ADDRESS, "localhost");
configuration.setLong(WebOptions.REFRESH_INTERVAL, 5L);
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -18,32 +18,33 @@

package org.apache.flink.runtime.webmonitor.history;

import org.junit.Assert;
import org.junit.Test;
import org.junit.jupiter.api.Test;

import static org.assertj.core.api.Assertions.assertThat;

/** Tests for the {@link ArchivedJson}. */
public class ArchivedJsonTest {
class ArchivedJsonTest {

@Test
public void testEquals() {
void testEquals() {
ArchivedJson original = new ArchivedJson("path", "json");
ArchivedJson twin = new ArchivedJson("path", "json");
ArchivedJson identicalPath = new ArchivedJson("path", "hello");
ArchivedJson identicalJson = new ArchivedJson("hello", "json");

Assert.assertEquals(original, original);
Assert.assertEquals(original, twin);
Assert.assertNotEquals(original, identicalPath);
Assert.assertNotEquals(original, identicalJson);
assertThat(original).isEqualTo(original);
assertThat(twin).isEqualTo(original);
assertThat(identicalPath).isNotEqualTo(original);
assertThat(identicalJson).isNotEqualTo(original);
}

@Test
public void testHashCode() {
void testHashCode() {
ArchivedJson original = new ArchivedJson("path", "json");
ArchivedJson twin = new ArchivedJson("path", "json");

Assert.assertEquals(original, original);
Assert.assertEquals(original, twin);
Assert.assertEquals(original.hashCode(), twin.hashCode());
assertThat(original).isEqualTo(original);
assertThat(twin).isEqualTo(original);
assertThat(twin).hasSameHashCodeAs(original);
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -21,54 +21,52 @@
import org.apache.flink.configuration.Configuration;
import org.apache.flink.configuration.HistoryServerOptions;
import org.apache.flink.configuration.SecurityOptions;
import org.apache.flink.util.TestLogger;

import org.junit.Test;
import org.junit.jupiter.api.Test;

import javax.annotation.Nonnull;

import java.net.MalformedURLException;
import java.net.URL;
import java.util.Optional;

import static org.hamcrest.Matchers.is;
import static org.junit.Assert.assertThat;
import static org.assertj.core.api.Assertions.assertThat;

/** Tests for the {@link HistoryServerUtils}. */
public class HistoryServerUtilsTest extends TestLogger {
class HistoryServerUtilsTest {

private static final String HOSTNAME = "foobar";
private static final int PORT = 1234;

@Test
public void testIsSSLEnabledDefault() {
void testIsSSLEnabledDefault() {
final Configuration configuration = new Configuration();

assertThat(HistoryServerUtils.isSSLEnabled(configuration), is(false));
assertThat(HistoryServerUtils.isSSLEnabled(configuration)).isFalse();
}

@Test
public void testIsSSLEnabledWithoutRestSSL() {
void testIsSSLEnabledWithoutRestSSL() {
final Configuration configuration = new Configuration();
configuration.setBoolean(HistoryServerOptions.HISTORY_SERVER_WEB_SSL_ENABLED, true);

assertThat(HistoryServerUtils.isSSLEnabled(configuration), is(false));
assertThat(HistoryServerUtils.isSSLEnabled(configuration)).isFalse();
}

@Test
public void testIsSSLEnabledOnlyRestSSL() {
void testIsSSLEnabledOnlyRestSSL() {
final Configuration configuration = new Configuration();
configuration.setBoolean(SecurityOptions.SSL_REST_ENABLED, true);

assertThat(HistoryServerUtils.isSSLEnabled(configuration), is(false));
assertThat(HistoryServerUtils.isSSLEnabled(configuration)).isFalse();
}

@Test
public void testIsSSLEnabled() {
void testIsSSLEnabled() {
final Configuration configuration = new Configuration();
enableSSL(configuration);

assertThat(HistoryServerUtils.isSSLEnabled(configuration), is(true));
assertThat(HistoryServerUtils.isSSLEnabled(configuration)).isTrue();
}

private void enableSSL(Configuration configuration) {
Expand All @@ -77,31 +75,31 @@ private void enableSSL(Configuration configuration) {
}

@Test
public void testGetHistoryServerURL() throws MalformedURLException {
void testGetHistoryServerURL() throws MalformedURLException {
final Configuration configuration = createDefaultConfiguration();

final Optional<URL> historyServerURL =
HistoryServerUtils.getHistoryServerURL(configuration);

assertThat(historyServerURL.get(), is(new URL("http", HOSTNAME, PORT, "")));
assertThat(historyServerURL).isPresent().hasValue(new URL("http", HOSTNAME, PORT, ""));
}

@Test
public void testGetHistoryServerURLWithSSL() throws MalformedURLException {
void testGetHistoryServerURLWithSSL() throws MalformedURLException {
final Configuration configuration = createDefaultConfiguration();
enableSSL(configuration);

final Optional<URL> historyServerURL =
HistoryServerUtils.getHistoryServerURL(configuration);

assertThat(historyServerURL.get(), is(new URL("https", HOSTNAME, PORT, "")));
assertThat(historyServerURL).isPresent().hasValue(new URL("https", HOSTNAME, PORT, ""));
}

@Test
public void testGetHistoryServerURLWithoutHS() {
void testGetHistoryServerURLWithoutHS() {
final Configuration configuration = new Configuration();

assertThat(HistoryServerUtils.getHistoryServerURL(configuration).isPresent(), is(false));
assertThat(HistoryServerUtils.getHistoryServerURL(configuration)).isNotPresent();
}

@Nonnull
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -19,33 +19,31 @@
package org.apache.flink.runtime.webmonitor.retriever;

import org.apache.flink.api.java.tuple.Tuple2;
import org.apache.flink.core.testutils.FlinkAssertions;
import org.apache.flink.runtime.leaderretrieval.SettableLeaderRetrievalService;
import org.apache.flink.runtime.rpc.RpcGateway;
import org.apache.flink.runtime.rpc.TestingRpcGateway;
import org.apache.flink.util.FlinkException;
import org.apache.flink.util.TestLogger;
import org.apache.flink.util.concurrent.FutureUtils;

import org.junit.Test;
import org.junit.jupiter.api.Test;

import java.util.UUID;
import java.util.concurrent.CompletableFuture;
import java.util.concurrent.ExecutionException;

import static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertFalse;
import static org.junit.Assert.fail;
import static org.mockito.Mockito.mock;
import static org.assertj.core.api.Assertions.assertThat;

/** Test cases for the {@link LeaderGatewayRetriever}. */
public class LeaderGatewayRetrieverTest extends TestLogger {
class LeaderGatewayRetrieverTest {

/** Tests that the gateway retrieval is retried in case of a failure. */
@Test
public void testGatewayRetrievalFailures() throws Exception {
void testGatewayRetrievalFailures() throws Exception {
final String address = "localhost";
final UUID leaderId = UUID.randomUUID();

RpcGateway rpcGateway = mock(RpcGateway.class);
RpcGateway rpcGateway = TestingRpcGateway.newBuilder().build();

TestingLeaderGatewayRetriever leaderGatewayRetriever =
new TestingLeaderGatewayRetriever(rpcGateway);
Expand All @@ -59,20 +57,15 @@ public void testGatewayRetrievalFailures() throws Exception {
// this triggers the first gateway retrieval attempt
settableLeaderRetrievalService.notifyListener(address, leaderId);

// check that the first future has been failed
try {
gatewayFuture.get();

fail("The first future should have been failed.");
} catch (ExecutionException ignored) {
// that's what we expect
}
FlinkAssertions.assertThatFuture(gatewayFuture)
.as("The first future should have been failed.")
.eventuallyFailsWith(ExecutionException.class);

// the second attempt should fail as well
assertFalse((leaderGatewayRetriever.getNow().isPresent()));
assertThat((leaderGatewayRetriever.getNow())).isNotPresent();

// the third attempt should succeed
assertEquals(rpcGateway, leaderGatewayRetriever.getNow().get());
assertThat(leaderGatewayRetriever.getNow()).hasValue(rpcGateway);
}

private static class TestingLeaderGatewayRetriever extends LeaderGatewayRetriever<RpcGateway> {
Expand Down
Loading

0 comments on commit 1ae8e40

Please sign in to comment.