[FLINK-7903] [tests] Add flip6 build profile

The flip6 build profile only runs the Flip-6 related test cases. Moreover,
all Flip-6 related test cases are excluded when not running the flip6 build
profile. This should reduce testing time when adding more and more Flip-6
test cases.

Include flink-test-utils-junit in all submodules to make the Category marker interfaces Flip6 and OldAndFlip6 available

This closes #4889.
上级 f16335d4
......@@ -73,8 +73,6 @@ under the License.
<dependency>
<groupId>org.apache.flink</groupId>
<artifactId>flink-test-utils-junit</artifactId>
<version>${project.version}</version>
<scope>test</scope>
</dependency>
<dependency>
......
......@@ -58,12 +58,14 @@ import org.apache.flink.runtime.rest.messages.job.savepoints.SavepointTriggerHea
import org.apache.flink.runtime.rest.messages.job.savepoints.SavepointTriggerResponseBody;
import org.apache.flink.runtime.rpc.RpcUtils;
import org.apache.flink.runtime.webmonitor.retriever.GatewayRetriever;
import org.apache.flink.testutils.category.Flip6;
import org.apache.flink.util.TestLogger;
import org.apache.flink.shaded.netty4.io.netty.channel.ChannelInboundHandler;
import org.junit.Assert;
import org.junit.Test;
import org.junit.experimental.categories.Category;
import javax.annotation.Nonnull;
......@@ -86,6 +88,7 @@ import static org.mockito.Mockito.when;
* <p>These tests verify that the client uses the appropriate headers for each
* request, properly constructs the request bodies/parameters and processes the responses correctly.
*/
@Category(Flip6.class)
public class RestClusterClientTest extends TestLogger {
private static final String restAddress = "http://localhost:1234";
......
......@@ -72,6 +72,11 @@ under the License.
<artifactId>jsr305</artifactId>
<scope>provided</scope>
</dependency>
<dependency>
<groupId>org.apache.flink</groupId>
<artifactId>flink-test-utils-junit</artifactId>
</dependency>
</dependencies>
<!-- See main pom.xml for explanation of profiles -->
......
......@@ -62,13 +62,6 @@ under the License.
<!-- test dependencies -->
<dependency>
<groupId>org.apache.flink</groupId>
<artifactId>flink-test-utils-junit</artifactId>
<version>${project.version}</version>
<scope>test</scope>
</dependency>
<dependency>
<groupId>org.apache.flink</groupId>
<artifactId>flink-core</artifactId>
......
......@@ -43,4 +43,11 @@ under the License.
<module>flink-connector-wikiedits</module>
<module>flink-statebackend-rocksdb</module>
</modules>
<dependencies>
<dependency>
<groupId>org.apache.flink</groupId>
<artifactId>flink-test-utils-junit</artifactId>
</dependency>
</dependencies>
</project>
......@@ -91,8 +91,6 @@ under the License.
<dependency>
<groupId>org.apache.flink</groupId>
<artifactId>flink-test-utils-junit</artifactId>
<version>${project.version}</version>
<scope>test</scope>
</dependency>
<dependency>
......
......@@ -69,6 +69,11 @@ under the License.
<scope>compile</scope>
</dependency>
<dependency>
<groupId>org.apache.flink</groupId>
<artifactId>flink-test-utils-junit</artifactId>
</dependency>
</dependencies>
<build>
......
......@@ -49,15 +49,6 @@ under the License.
<optional>true</optional>
</dependency>
<!-- test dependencies -->
<dependency>
<groupId>org.apache.flink</groupId>
<artifactId>flink-test-utils-junit</artifactId>
<version>${project.version}</version>
<scope>test</scope>
</dependency>
<!-- for the behavior test suite -->
<dependency>
<groupId>org.apache.flink</groupId>
......
......@@ -65,13 +65,6 @@ under the License.
<optional>true</optional>
</dependency>
<!-- test dependencies -->
<dependency>
<groupId>org.apache.flink</groupId>
<artifactId>flink-test-utils-junit</artifactId>
<version>${project.version}</version>
</dependency>
</dependencies>
</project>
......@@ -175,15 +175,6 @@ under the License.
<scope>provided</scope>
</dependency>
<!-- test dependencies -->
<dependency>
<groupId>org.apache.flink</groupId>
<artifactId>flink-test-utils-junit</artifactId>
<version>${project.version}</version>
<scope>test</scope>
</dependency>
<!-- for the behavior test suite -->
<dependency>
<groupId>org.apache.flink</groupId>
......
......@@ -198,15 +198,6 @@ under the License.
<scope>provided</scope>
</dependency>
<!-- test dependencies -->
<dependency>
<groupId>org.apache.flink</groupId>
<artifactId>flink-test-utils-junit</artifactId>
<version>${project.version}</version>
<scope>test</scope>
</dependency>
<!-- for the behavior test suite -->
<dependency>
<groupId>org.apache.flink</groupId>
......
......@@ -62,4 +62,11 @@ under the License.
</plugins>
</build>
<dependencies>
<dependency>
<groupId>org.apache.flink</groupId>
<artifactId>flink-test-utils-junit</artifactId>
</dependency>
</dependencies>
</project>
......@@ -65,6 +65,11 @@ under the License.
<scope>test</scope>
</dependency>
<dependency>
<groupId>org.apache.flink</groupId>
<artifactId>flink-test-utils-junit</artifactId>
</dependency>
<dependency>
<groupId>org.apache.flink</groupId>
<artifactId>flink-test-utils_${scala.binary.version}</artifactId>
......
......@@ -63,8 +63,6 @@ under the License.
<dependency>
<groupId>org.apache.flink</groupId>
<artifactId>flink-test-utils-junit</artifactId>
<version>${project.version}</version>
<scope>test</scope>
</dependency>
</dependencies>
......
......@@ -59,6 +59,11 @@ under the License.
<artifactId>jsr305</artifactId>
<scope>provided</scope>
</dependency>
<dependency>
<groupId>org.apache.flink</groupId>
<artifactId>flink-test-utils-junit</artifactId>
</dependency>
</dependencies>
<build>
......
......@@ -132,6 +132,11 @@ under the License.
<type>test-jar</type>
</dependency>
<dependency>
<groupId>org.apache.flink</groupId>
<artifactId>flink-test-utils-junit</artifactId>
</dependency>
<dependency>
<groupId>org.apache.flink</groupId>
<artifactId>flink-test-utils_${scala.binary.version}</artifactId>
......
......@@ -60,6 +60,10 @@ under the License.
<artifactId>jsr305</artifactId>
<scope>provided</scope>
</dependency>
<dependency>
<groupId>org.apache.flink</groupId>
<artifactId>flink-test-utils-junit</artifactId>
</dependency>
</dependencies>
</project>
......@@ -71,8 +71,6 @@ under the License.
<dependency>
<groupId>org.apache.flink</groupId>
<artifactId>flink-test-utils-junit</artifactId>
<version>${project.version}</version>
<scope>test</scope>
</dependency>
</dependencies>
......
......@@ -51,5 +51,10 @@ under the License.
<artifactId>jsr305</artifactId>
<scope>provided</scope>
</dependency>
<dependency>
<groupId>org.apache.flink</groupId>
<artifactId>flink-test-utils-junit</artifactId>
</dependency>
</dependencies>
</project>
......@@ -82,8 +82,6 @@ under the License.
<dependency>
<groupId>org.apache.flink</groupId>
<artifactId>flink-test-utils-junit</artifactId>
<version>${project.version}</version>
<scope>test</scope>
</dependency>
<dependency>
......
......@@ -202,8 +202,6 @@ under the License.
<dependency>
<groupId>org.apache.flink</groupId>
<artifactId>flink-test-utils-junit</artifactId>
<version>${project.version}</version>
<scope>test</scope>
</dependency>
<dependency>
......
......@@ -50,6 +50,7 @@ import org.apache.flink.runtime.rpc.TestingRpcService;
import org.apache.flink.runtime.testtasks.NoOpInvokable;
import org.apache.flink.runtime.testutils.InMemorySubmittedJobGraphStore;
import org.apache.flink.runtime.util.TestingFatalErrorHandler;
import org.apache.flink.testutils.category.Flip6;
import org.apache.flink.util.TestLogger;
import org.junit.After;
......@@ -58,6 +59,7 @@ import org.junit.Before;
import org.junit.BeforeClass;
import org.junit.Rule;
import org.junit.Test;
import org.junit.experimental.categories.Category;
import org.junit.rules.TemporaryFolder;
import org.junit.rules.TestName;
import org.mockito.Mockito;
......@@ -81,6 +83,7 @@ import static org.mockito.Mockito.verify;
/**
* Test for the {@link Dispatcher} component.
*/
@Category(Flip6.class)
public class DispatcherTest extends TestLogger {
private static RpcService rpcService;
......
......@@ -23,9 +23,11 @@ import org.apache.flink.runtime.concurrent.Executors;
import org.apache.flink.runtime.concurrent.ScheduledExecutor;
import org.apache.flink.runtime.concurrent.ScheduledExecutorServiceAdapter;
import org.apache.flink.runtime.util.DirectExecutorService;
import org.apache.flink.testutils.category.Flip6;
import org.apache.flink.util.TestLogger;
import org.junit.Test;
import org.junit.experimental.categories.Category;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
......@@ -54,6 +56,7 @@ import static org.mockito.Mockito.when;
/**
* Tests for the {@link HeartbeatManager}.
*/
@Category(Flip6.class)
public class HeartbeatManagerTest extends TestLogger {
private static final Logger LOG = LoggerFactory.getLogger(HeartbeatManagerTest.class);
......
......@@ -37,11 +37,13 @@ import org.apache.flink.runtime.metrics.MetricRegistryImpl;
import org.apache.flink.runtime.metrics.MetricRegistryConfiguration;
import org.apache.flink.runtime.rpc.FatalErrorHandler;
import org.apache.flink.runtime.rpc.RpcService;
import org.apache.flink.testutils.category.Flip6;
import org.apache.flink.util.TestLogger;
import org.junit.After;
import org.junit.Before;
import org.junit.Ignore;
import org.junit.Test;
import org.junit.experimental.categories.Category;
import org.junit.runner.RunWith;
import org.powermock.api.mockito.PowerMockito;
import org.powermock.core.classloader.annotations.PrepareForTest;
......@@ -60,6 +62,7 @@ import static org.mockito.Mockito.when;
@RunWith(PowerMockRunner.class)
@PrepareForTest(JobManagerRunner.class)
@Category(Flip6.class)
public class JobManagerRunnerMockTest extends TestLogger {
private JobManagerRunner runner;
......
......@@ -43,8 +43,10 @@ import org.apache.flink.runtime.rpc.TestingRpcService;
import org.apache.flink.runtime.taskexecutor.TaskExecutorGateway;
import org.apache.flink.runtime.taskmanager.TaskManagerLocation;
import org.apache.flink.runtime.util.TestingFatalErrorHandler;
import org.apache.flink.testutils.category.Flip6;
import org.apache.flink.util.TestLogger;
import org.junit.Test;
import org.junit.experimental.categories.Category;
import org.junit.runner.RunWith;
import org.mockito.ArgumentCaptor;
import org.powermock.core.classloader.annotations.PrepareForTest;
......@@ -61,6 +63,7 @@ import static org.mockito.Mockito.*;
@RunWith(PowerMockRunner.class)
@PrepareForTest(BlobLibraryCacheManager.class)
@Category(Flip6.class)
public class JobMasterTest extends TestLogger {
private final Time testingTimeout = Time.seconds(10L);
......
......@@ -25,15 +25,18 @@ import org.apache.flink.runtime.jobgraph.JobGraph;
import org.apache.flink.runtime.jobgraph.JobVertex;
import org.apache.flink.runtime.jobgraph.ScheduleMode;
import org.apache.flink.runtime.testtasks.NoOpInvokable;
import org.apache.flink.testutils.category.Flip6;
import org.apache.flink.util.TestLogger;
import org.junit.Test;
import org.junit.experimental.categories.Category;
import java.io.IOException;
/**
* Integration test cases for the {@link MiniCluster}.
*/
@Category(Flip6.class)
public class MiniClusterITCase extends TestLogger {
// ------------------------------------------------------------------------
......
......@@ -24,8 +24,10 @@ import org.apache.flink.runtime.concurrent.ScheduledExecutor;
import org.apache.flink.runtime.highavailability.TestingHighAvailabilityServices;
import org.apache.flink.runtime.jobmaster.JobMasterId;
import org.apache.flink.runtime.leaderelection.TestingLeaderRetrievalService;
import org.apache.flink.testutils.category.Flip6;
import org.apache.flink.util.TestLogger;
import org.junit.Test;
import org.junit.experimental.categories.Category;
import org.mockito.ArgumentCaptor;
import org.mockito.invocation.InvocationOnMock;
import org.mockito.stubbing.Answer;
......@@ -54,6 +56,7 @@ import static org.mockito.Mockito.mock;
import static org.mockito.Mockito.times;
import static org.mockito.Mockito.verify;
@Category(Flip6.class)
public class JobLeaderIdServiceTest extends TestLogger {
/**
......
......@@ -30,9 +30,11 @@ import org.apache.flink.runtime.rpc.RpcService;
import org.apache.flink.runtime.rpc.TestingRpcService;
import org.apache.flink.runtime.testingUtils.TestingUtils;
import org.apache.flink.runtime.util.TestingFatalErrorHandler;
import org.apache.flink.testutils.category.Flip6;
import org.apache.flink.util.TestLogger;
import org.junit.Assert;
import org.junit.Test;
import org.junit.experimental.categories.Category;
import java.util.UUID;
import java.util.concurrent.CompletableFuture;
......@@ -42,6 +44,7 @@ import static org.mockito.Mockito.mock;
/**
* resourceManager HA test, including grant leadership and revoke leadership
*/
@Category(Flip6.class)
public class ResourceManagerHATest extends TestLogger {
@Test
......
......@@ -41,11 +41,13 @@ import org.apache.flink.runtime.registration.RegistrationResponse;
import org.apache.flink.runtime.rpc.exceptions.FencingTokenException;
import org.apache.flink.runtime.testingUtils.TestingUtils;
import org.apache.flink.runtime.util.TestingFatalErrorHandler;
import org.apache.flink.testutils.category.Flip6;
import org.apache.flink.util.ExceptionUtils;
import org.apache.flink.util.TestLogger;
import org.junit.After;
import org.junit.Before;
import org.junit.Test;
import org.junit.experimental.categories.Category;
import java.util.UUID;
import java.util.concurrent.CompletableFuture;
......@@ -56,6 +58,7 @@ import static org.junit.Assert.assertTrue;
import static org.junit.Assert.fail;
import static org.mockito.Mockito.*;
@Category(Flip6.class)
public class ResourceManagerJobMasterTest extends TestLogger {
private TestingRpcService rpcService;
......
......@@ -38,11 +38,13 @@ import org.apache.flink.runtime.taskexecutor.TaskExecutorGateway;
import org.apache.flink.runtime.taskexecutor.TaskExecutorRegistrationSuccess;
import org.apache.flink.runtime.testingUtils.TestingUtils;
import org.apache.flink.runtime.util.TestingFatalErrorHandler;
import org.apache.flink.testutils.category.Flip6;
import org.apache.flink.util.ExceptionUtils;
import org.apache.flink.util.TestLogger;
import org.junit.After;
import org.junit.Before;
import org.junit.Test;
import org.junit.experimental.categories.Category;
import java.util.UUID;
import java.util.concurrent.CompletableFuture;
......@@ -57,6 +59,7 @@ import static org.junit.Assert.fail;
import static org.mockito.Mockito.mock;
import static org.mockito.Mockito.when;
@Category(Flip6.class)
public class ResourceManagerTaskExecutorTest extends TestLogger {
private final Time timeout = Time.seconds(10L);
......
......@@ -38,9 +38,11 @@ import org.apache.flink.runtime.taskexecutor.SlotStatus;
import org.apache.flink.runtime.taskexecutor.TaskExecutorGateway;
import org.apache.flink.runtime.taskexecutor.exceptions.SlotAllocationException;
import org.apache.flink.runtime.testingUtils.TestingUtils;
import org.apache.flink.testutils.category.Flip6;
import org.apache.flink.util.TestLogger;
import org.junit.Test;
import org.junit.experimental.categories.Category;
import org.mockito.ArgumentCaptor;
import java.util.Arrays;
......@@ -68,6 +70,7 @@ import static org.mockito.Mockito.times;
import static org.mockito.Mockito.verify;
import static org.mockito.Mockito.when;
@Category(Flip6.class)
public class SlotManagerTest extends TestLogger {
/**
......
......@@ -33,11 +33,13 @@ import org.apache.flink.runtime.taskexecutor.SlotReport;
import org.apache.flink.runtime.taskexecutor.SlotStatus;
import org.apache.flink.runtime.taskexecutor.TaskExecutorGateway;
import org.apache.flink.runtime.testingUtils.TestingUtils;
import org.apache.flink.testutils.category.Flip6;
import org.apache.flink.util.ExecutorUtils;
import org.apache.flink.util.TestLogger;
import org.junit.AfterClass;
import org.junit.Test;
import org.junit.experimental.categories.Category;
import org.mockito.Mockito;
import java.util.Collections;
......@@ -52,6 +54,7 @@ import static org.mockito.Mockito.mock;
import static org.mockito.Mockito.timeout;
import static org.mockito.Mockito.verify;
@Category(Flip6.class)
public class SlotProtocolTest extends TestLogger {
private static final long timeout = 10000L;
......
......@@ -38,6 +38,7 @@ import org.apache.flink.runtime.rpc.RpcUtils;
import org.apache.flink.runtime.testingUtils.TestingUtils;
import org.apache.flink.runtime.webmonitor.RestfulGateway;
import org.apache.flink.runtime.webmonitor.retriever.GatewayRetriever;
import org.apache.flink.testutils.category.Flip6;
import org.apache.flink.util.ExceptionUtils;
import org.apache.flink.util.Preconditions;
import org.apache.flink.util.TestLogger;
......@@ -51,6 +52,7 @@ import org.junit.After;
import org.junit.Assert;
import org.junit.Before;
import org.junit.Test;
import org.junit.experimental.categories.Category;
import javax.annotation.Nonnull;
......@@ -69,6 +71,7 @@ import static org.mockito.Mockito.when;
/**
* IT cases for {@link RestClient} and {@link RestServerEndpoint}.
*/
@Category(Flip6.class)
public class RestEndpointITCase extends TestLogger {
private static final JobID PATH_JOB_ID = new JobID();
......
......@@ -28,12 +28,14 @@ import org.apache.flink.runtime.rest.messages.EmptyMessageParameters;
import org.apache.flink.runtime.rest.messages.EmptyRequestBody;
import org.apache.flink.runtime.rpc.RpcUtils;
import org.apache.flink.runtime.webmonitor.retriever.GatewayRetriever;
import org.apache.flink.testutils.category.Flip6;
import org.apache.flink.util.TestLogger;
import org.apache.flink.shaded.netty4.io.netty.handler.codec.http.HttpResponseStatus;
import org.junit.Assert;
import org.junit.Test;
import org.junit.experimental.categories.Category;
import java.util.Collections;
import java.util.concurrent.CompletableFuture;
......@@ -46,6 +48,7 @@ import static org.mockito.Mockito.when;
/**
* Tests for the {@link BlobServerPortHandler}.
*/
@Category(Flip6.class)
public class BlobServerPortHandlerTest extends TestLogger {
private static final int PORT = 64;
......
......@@ -28,12 +28,14 @@ import org.apache.flink.runtime.rest.messages.EmptyMessageParameters;
import org.apache.flink.runtime.rest.messages.job.JobSubmitRequestBody;
import org.apache.flink.runtime.rpc.RpcUtils;
import org.apache.flink.runtime.webmonitor.retriever.GatewayRetriever;
import org.apache.flink.testutils.category.Flip6;
import org.apache.flink.util.TestLogger;
import org.apache.flink.shaded.netty4.io.netty.handler.codec.http.HttpResponseStatus;
import org.junit.Assert;
import org.junit.Test;
import org.junit.experimental.categories.Category;
import java.util.Collections;
import java.util.concurrent.CompletableFuture;
......@@ -45,6 +47,7 @@ import static org.mockito.Mockito.when;
/**
* Tests for the {@link JobSubmitHandler}.
*/
@Category(Flip6.class)
public class JobSubmitHandlerTest extends TestLogger {
@Test
......
......@@ -19,10 +19,12 @@
package org.apache.flink.runtime.rest.messages;
import org.apache.flink.api.common.JobID;
import org.apache.flink.testutils.category.Flip6;
import org.apache.flink.util.TestLogger;
import org.junit.Assert;
import org.junit.Test;
import org.junit.experimental.categories.Category;
import java.util.Collection;
import java.util.Collections;
......@@ -30,6 +32,7 @@ import java.util.Collections;
/**
* Tests for {@link MessageParameters}.
*/
@Category(Flip6.class)
public class MessageParametersTest extends TestLogger {
@Test
public void testResolveUrl() {
......
......@@ -19,16 +19,19 @@
package org.apache.flink.runtime.rest.messages;
import org.apache.flink.runtime.rest.util.RestMapperUtils;
import org.apache.flink.testutils.category.Flip6;
import org.apache.flink.util.TestLogger;
import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.ObjectMapper;
import org.junit.Assert;
import org.junit.Test;
import org.junit.experimental.categories.Category;
/**
* Test base for verifying that marshalling / unmarshalling REST {@link RequestBody}s work properly.
*/
@Category(Flip6.class)
public abstract class RestRequestMarshallingTestBase<R extends RequestBody> extends TestLogger {
/**
......
......@@ -19,16 +19,19 @@
package org.apache.flink.runtime.rest.messages;
import org.apache.flink.runtime.rest.util.RestMapperUtils;
import org.apache.flink.testutils.category.Flip6;
import org.apache.flink.util.TestLogger;
import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.ObjectMapper;
import org.junit.Assert;
import org.junit.Test;
import org.junit.experimental.categories.Category;
/**
* Test base for verifying that marshalling / unmarshalling REST {@link ResponseBody}s work properly.
*/
@Category(Flip6.class)
public abstract class RestResponseMarshallingTestBase<R extends ResponseBody> extends TestLogger {
/**
......
......@@ -26,11 +26,13 @@ import org.apache.flink.runtime.akka.AkkaUtils;
import org.apache.flink.runtime.messages.Acknowledge;
import org.apache.flink.runtime.rpc.akka.AkkaRpcService;
import org.apache.flink.runtime.rpc.exceptions.FencingTokenException;
import org.apache.flink.testutils.category.Flip6;
import org.apache.flink.util.ExceptionUtils;
import org.apache.flink.util.TestLogger;
import org.junit.AfterClass;
import org.junit.Test;
import org.junit.experimental.categories.Category;
import java.util.UUID;
import java.util.concurrent.CompletableFuture;
......@@ -43,6 +45,7 @@ import java.util.function.Function;
import static org.junit.Assert.*;
@Category(Flip6.class)
public class AsyncCallsTest extends TestLogger {
// ------------------------------------------------------------------------
......
......@@ -23,6 +23,7 @@ import org.apache.flink.core.testutils.OneShotLatch;
import org.apache.flink.runtime.messages.Acknowledge;
import org.apache.flink.runtime.rpc.exceptions.FencingTokenException;
import org.apache.flink.runtime.rpc.exceptions.RpcException;
import org.apache.flink.testutils.category.Flip6;
import org.apache.flink.util.ExceptionUtils;
import org.apache.flink.util.FlinkException;
import org.apache.flink.util.TestLogger;
......@@ -30,6 +31,7 @@ import org.apache.flink.util.TestLogger;
import org.junit.AfterClass;
import org.junit.BeforeClass;
import org.junit.Test;
import org.junit.experimental.categories.Category;
import java.util.UUID;
import java.util.concurrent.CompletableFuture;
......@@ -44,6 +46,7 @@ import static org.junit.Assert.assertNull;
import static org.junit.Assert.assertTrue;
import static org.junit.Assert.fail;
@Category(Flip6.class)
public class FencedRpcEndpointTest extends TestLogger {
private static final Time timeout = Time.seconds(10L);
......
......@@ -26,8 +26,10 @@ import org.apache.flink.runtime.akka.AkkaUtils;
import org.apache.flink.runtime.rpc.akka.AkkaRpcService;
import org.apache.flink.runtime.rpc.exceptions.RpcConnectionException;
import org.apache.flink.runtime.taskexecutor.TaskExecutorGateway;
import org.apache.flink.testutils.category.Flip6;
import org.junit.Test;
import org.junit.experimental.categories.Category;
import scala.Option;
import scala.Tuple2;
......@@ -43,6 +45,7 @@ import static org.junit.Assert.*;
* This test validates that the RPC service gives a good message when it cannot
* connect to an RpcEndpoint.
*/
@Category(Flip6.class)
public class RpcConnectionTest {
@Test
......
......@@ -21,12 +21,14 @@ package org.apache.flink.runtime.rpc;
import org.apache.flink.api.common.time.Time;
import org.apache.flink.runtime.akka.AkkaUtils;
import org.apache.flink.runtime.rpc.akka.AkkaRpcService;
import org.apache.flink.testutils.category.Flip6;
import org.apache.flink.util.TestLogger;
import akka.actor.ActorSystem;
import org.junit.AfterClass;
import org.junit.BeforeClass;
import org.junit.Test;
import org.junit.experimental.categories.Category;
import java.util.concurrent.CompletableFuture;
import java.util.concurrent.TimeUnit;
......@@ -39,6 +41,7 @@ import static org.junit.Assert.fail;
/**
* Tests for the RpcEndpoint and its self gateways.
*/
@Category(Flip6.class)
public class RpcEndpointTest extends TestLogger {
private static final Time TIMEOUT = Time.seconds(10L);
......
......@@ -27,12 +27,14 @@ import org.apache.flink.runtime.rpc.RpcGateway;
import org.apache.flink.runtime.rpc.RpcService;
import org.apache.flink.runtime.rpc.akka.exceptions.AkkaRpcException;
import org.apache.flink.runtime.rpc.exceptions.RpcConnectionException;
import org.apache.flink.testutils.category.Flip6;
import org.apache.flink.util.FlinkException;
import org.apache.flink.util.TestLogger;
import org.hamcrest.core.Is;
import org.junit.AfterClass;
import org.junit.Test;
import org.junit.experimental.categories.Category;
import java.util.concurrent.CompletableFuture;
import java.util.concurrent.ExecutionException;
......@@ -44,6 +46,7 @@ import static org.junit.Assert.assertThat;
import static org.junit.Assert.assertTrue;
import static org.junit.Assert.fail;
@Category(Flip6.class)
public class AkkaRpcActorTest extends TestLogger {
// ------------------------------------------------------------------------
......
......@@ -23,11 +23,13 @@ import org.apache.flink.core.testutils.OneShotLatch;
import org.apache.flink.runtime.akka.AkkaUtils;
import org.apache.flink.runtime.concurrent.FutureUtils;
import org.apache.flink.runtime.concurrent.ScheduledExecutor;
import org.apache.flink.testutils.category.Flip6;
import org.apache.flink.util.TestLogger;
import akka.actor.ActorSystem;
import org.junit.AfterClass;
import org.junit.Test;
import org.junit.experimental.categories.Category;
import java.util.concurrent.Callable;
import java.util.concurrent.CompletableFuture;
......@@ -42,6 +44,7 @@ import static org.junit.Assert.assertFalse;
import static org.junit.Assert.assertTrue;
import static org.junit.Assert.fail;
@Category(Flip6.class)
public class AkkaRpcServiceTest extends TestLogger {
// ------------------------------------------------------------------------
......
......@@ -25,11 +25,14 @@ import org.apache.flink.runtime.rpc.RpcEndpoint;
import org.apache.flink.runtime.rpc.RpcGateway;
import org.apache.flink.runtime.rpc.RpcService;
import org.apache.flink.testutils.category.Flip6;
import org.apache.flink.util.TestLogger;
import org.junit.Test;
import org.junit.experimental.categories.Category;
import static org.junit.Assert.assertTrue;
@Category(Flip6.class)
public class MainThreadValidationTest extends TestLogger {
@Test
......
......@@ -26,11 +26,13 @@ import org.apache.flink.runtime.akka.AkkaUtils;
import org.apache.flink.runtime.rpc.RpcEndpoint;
import org.apache.flink.runtime.rpc.RpcGateway;
import org.apache.flink.runtime.rpc.RpcService;
import org.apache.flink.testutils.category.Flip6;
import org.apache.flink.util.TestLogger;
import org.hamcrest.core.Is;
import org.junit.AfterClass;
import org.junit.BeforeClass;
import org.junit.Test;
import org.junit.experimental.categories.Category;
import java.io.IOException;
import java.util.concurrent.CompletableFuture;
......@@ -42,6 +44,7 @@ import static org.junit.Assert.fail;
/**
* Tests that akka rpc invocation messages are properly serialized and errors reported
*/
@Category(Flip6.class)
public class MessageSerializationTest extends TestLogger {
private static ActorSystem actorSystem1;
private static ActorSystem actorSystem2;
......
......@@ -57,10 +57,12 @@ import org.apache.flink.runtime.taskexecutor.slot.TimerService;
import org.apache.flink.runtime.taskmanager.TaskManagerLocation;
import org.apache.flink.runtime.testingUtils.TestingUtils;
import org.apache.flink.runtime.util.TestingFatalErrorHandler;
import org.apache.flink.testutils.category.Flip6;
import org.apache.flink.util.TestLogger;
import org.hamcrest.Matchers;
import org.junit.Test;
import org.junit.experimental.categories.Category;
import org.mockito.Mockito;
import java.net.InetAddress;
......@@ -81,6 +83,7 @@ import static org.mockito.Mockito.mock;
import static org.mockito.Mockito.verify;
import static org.mockito.Mockito.when;
@Category(Flip6.class)
public class TaskExecutorITCase extends TestLogger {
private final Time timeout = Time.seconds(10L);
......
......@@ -81,6 +81,7 @@ import org.apache.flink.runtime.taskmanager.TaskExecutionState;
import org.apache.flink.runtime.taskmanager.TaskManagerActions;
import org.apache.flink.runtime.taskmanager.TaskManagerLocation;
import org.apache.flink.runtime.util.TestingFatalErrorHandler;
import org.apache.flink.testutils.category.Flip6;
import org.apache.flink.util.ExceptionUtils;
import org.apache.flink.util.SerializedValue;
import org.apache.flink.util.TestLogger;
......@@ -90,6 +91,7 @@ import org.junit.Before;
import org.junit.Ignore;
import org.junit.Rule;
import org.junit.Test;
import org.junit.experimental.categories.Category;
import org.junit.rules.TestName;
import org.mockito.ArgumentCaptor;
import org.mockito.Matchers;
......@@ -126,6 +128,7 @@ import static org.mockito.Mockito.timeout;
import static org.mockito.Mockito.verify;
import static org.mockito.Mockito.when;
@Category(Flip6.class)
public class TaskExecutorTest extends TestLogger {
private final Time timeout = Time.milliseconds(10000L);
......
......@@ -20,14 +20,19 @@ package org.apache.flink.runtime.taskexecutor;
import org.apache.flink.configuration.Configuration;
import org.apache.flink.configuration.TaskManagerOptions;
import org.apache.flink.testutils.category.OldAndFlip6;
import org.apache.flink.util.TestLogger;
import org.junit.Test;
import org.junit.experimental.categories.Category;
import static org.junit.Assert.*;
/**
* Unit test for {@link TaskManagerServicesConfiguration}.
*/
public class TaskManagerServicesConfigurationTest {
@Category(OldAndFlip6.class)
public class TaskManagerServicesConfigurationTest extends TestLogger {
/**
* Verifies that {@link TaskManagerServicesConfiguration#hasNewNetworkBufConf(Configuration)}
* returns the correct result for old configurations via
......
......@@ -20,9 +20,11 @@ package org.apache.flink.runtime.taskexecutor;
import org.apache.flink.configuration.Configuration;
import org.apache.flink.configuration.TaskManagerOptions;
import org.apache.flink.testutils.category.OldAndFlip6;
import org.apache.flink.util.TestLogger;
import org.junit.Test;
import org.junit.experimental.categories.Category;
import java.util.Random;
......@@ -33,7 +35,8 @@ import static org.junit.Assert.fail;
/**
* Unit test for {@link TaskManagerServices}.
*/
public class TaskManagerServicesTest extends TestLogger{
@Category(OldAndFlip6.class)
public class TaskManagerServicesTest extends TestLogger {
/**
* Test for {@link TaskManagerServices#calculateNetworkBufferMemory(long, Configuration)} using old
......
......@@ -19,7 +19,11 @@
package org.apache.flink.runtime.taskexecutor.slot;
import org.apache.flink.runtime.clusterframework.types.AllocationID;
import org.apache.flink.testutils.category.Flip6;
import org.apache.flink.util.TestLogger;
import org.junit.Test;
import org.junit.experimental.categories.Category;
import org.mockito.internal.util.reflection.Whitebox;
import java.util.Map;
......@@ -35,7 +39,8 @@ import static org.mockito.Mockito.times;
import static org.mockito.Mockito.verify;
import static org.mockito.Mockito.when;
public class TimerServiceTest {
@Category(Flip6.class)
public class TimerServiceTest extends TestLogger {
/**
* Test all timeouts registered can be unregistered
* @throws Exception
......
......@@ -76,8 +76,6 @@ under the License.
<dependency>
<groupId>org.apache.flink</groupId>
<artifactId>flink-test-utils-junit</artifactId>
<version>${project.version}</version>
<scope>test</scope>
</dependency>
<dependency>
......
......@@ -78,8 +78,6 @@ under the License.
<dependency>
<groupId>org.apache.flink</groupId>
<artifactId>flink-test-utils-junit</artifactId>
<version>${project.version}</version>
<scope>test</scope>
</dependency>
<dependency>
......
......@@ -71,6 +71,11 @@ under the License.
<scope>test</scope>
</dependency>
<dependency>
<groupId>org.apache.flink</groupId>
<artifactId>flink-test-utils-junit</artifactId>
</dependency>
<dependency>
<groupId>org.apache.flink</groupId>
<artifactId>flink-test-utils_${scala.binary.version}</artifactId>
......
/*
* 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.testutils.category;
/**
* Category marker interface for Flip-6 tests.
*/
public interface Flip6 {
}
/*
* 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.testutils.category;
/**
* Category marker interface for old and Flip-6 tests.
*/
public interface OldAndFlip6 {
}
......@@ -131,6 +131,11 @@ under the License.
<scope>test</scope>
</dependency>
<dependency>
<groupId>org.apache.flink</groupId>
<artifactId>flink-test-utils-junit</artifactId>
</dependency>
<dependency>
<groupId>org.apache.flink</groupId>
<artifactId>flink-test-utils_${scala.binary.version}</artifactId>
......
......@@ -41,6 +41,11 @@ under the License.
<!-- test dependencies -->
<dependency>
<groupId>org.apache.flink</groupId>
<artifactId>flink-test-utils-junit</artifactId>
</dependency>
<dependency>
<groupId>org.apache.flink</groupId>
<artifactId>flink-shaded-jackson</artifactId>
......
......@@ -96,6 +96,11 @@ under the License.
<scope>test</scope>
</dependency>
<dependency>
<groupId>org.apache.flink</groupId>
<artifactId>flink-test-utils-junit</artifactId>
</dependency>
<dependency>
<groupId>org.apache.flink</groupId>
<artifactId>flink-runtime_${scala.binary.version}</artifactId>
......
......@@ -126,6 +126,9 @@ under the License.
<powermock.version>1.6.5</powermock.version>
<hamcrest.version>1.3</hamcrest.version>
<japicmp.skip>false</japicmp.skip>
<!-- run all groups except flip6 by default -->
<test.groups></test.groups>
<test.excludedGroups>org.apache.flink.testutils.category.Flip6</test.excludedGroups>
<!--
Keeping the MiniKDC version fixed instead of taking hadoop version dependency
to support testing Kafka, ZK etc., modules that does not have Hadoop dependency
......@@ -582,11 +585,28 @@ under the License.
<version>0.9.10</version>
<scope>test</scope>
</dependency>
<!-- Testing dependency which should be included by all projects because of the Category definitions -->
<dependency>
<groupId>org.apache.flink</groupId>
<artifactId>flink-test-utils-junit</artifactId>
<version>${project.version}</version>
<scope>test</scope>
</dependency>
</dependencies>
</dependencyManagement>
<profiles>
<profile>
<id>flip6</id>
<properties>
<test.groups>org.apache.flink.testutils.category.Flip6, org.apache.flink.testutils.category.OldAndFlip6</test.groups>
<!-- clear the excluded groups list -->
<test.excludedGroups></test.excludedGroups>
</properties>
</profile>
<profile>
<id>spotbugs</id>
<activation>
......@@ -1104,6 +1124,8 @@ under the License.
<!-- Do NOT use a version >=2.19.X, as test cases may get stuck before execution. See SUREFIRE-1255 -->
<version>2.18.1</version>
<configuration>
<groups>${test.groups}</groups>
<excludedGroups>${test.excludedGroups}</excludedGroups>
<forkCount>${flink.forkCount}</forkCount>
<reuseForks>${flink.reuseForks}</reuseForks>
<systemPropertyVariables>
......
Markdown is supported
0% .
You are about to add 0 people to the discussion. Proceed with caution.
先完成此消息的编辑!
想要评论请 注册