XComp commented on code in PR #19780:
URL: https://github.com/apache/flink/pull/19780#discussion_r983718553


##########
flink-core/src/test/java/org/apache/flink/api/common/typeutils/TypeSerializerUpgradeTestBase.java:
##########
@@ -24,50 +24,52 @@
 import org.apache.flink.core.memory.DataInputView;
 import org.apache.flink.core.memory.DataOutputSerializer;
 import org.apache.flink.core.memory.DataOutputView;
-import org.apache.flink.util.TestLogger;
+import org.apache.flink.util.TestLoggerExtension;
 
+import org.assertj.core.api.HamcrestCondition;
 import org.hamcrest.Matcher;
-import org.junit.Ignore;
-import org.junit.Test;
+import org.junit.jupiter.api.Disabled;
+import org.junit.jupiter.api.TestInstance;
+import org.junit.jupiter.api.extension.ExtendWith;
+import org.junit.jupiter.params.ParameterizedTest;
+import org.junit.jupiter.params.provider.MethodSource;
 
 import java.io.IOException;
 import java.nio.file.Files;
 import java.nio.file.Path;
 import java.nio.file.Paths;
+import java.util.Collection;
 import java.util.Collections;
 import java.util.List;
 import java.util.Set;
 
 import static org.apache.flink.util.Preconditions.checkNotNull;
+import static org.assertj.core.api.AssertionsForInterfaceTypes.assertThat;
+import static org.assertj.core.api.Assumptions.assumeThat;
 import static org.hamcrest.CoreMatchers.not;
-import static org.junit.Assert.assertThat;
-import static org.junit.Assume.assumeThat;
 
 /**
  * A test base for testing {@link TypeSerializer} upgrades.
  *
- * <p>You can run {@link #generateTestSetupFiles()} on a Flink branch to 
(re-)generate the test data
- * files.
+ * <p>You can run {@link
+ * 
#generateTestSetupFiles(org.apache.flink.api.common.typeutils.TypeSerializerUpgradeTestBase.TestSpecification)}
+ * on a Flink branch to (re-)generate the test data files.
  */
-public abstract class TypeSerializerUpgradeTestBase<PreviousElementT, 
UpgradedElementT>
-        extends TestLogger {
+@ExtendWith(TestLoggerExtension.class)

Review Comment:
   Using the `META-INF/services/org.junit.jupiter.api.extension.Extension` is 
the better approach here.



##########
flink-core/src/test/java/org/apache/flink/api/java/typeutils/runtime/CopyableSerializerUpgradeTest.java:
##########
@@ -168,8 +159,8 @@ public Matcher<SimpleCopyable> testDataMatcher() {
     }
 
     @Test
-    public void testF() {
+    void testF() {

Review Comment:
   ```suggestion
       void testSimpleCopyableEqualsImplementation() {
   ```
   Can we make this test method more descriptive? (this should be a separate 
hotfix commit)



##########
flink-scala/src/test/scala/org/apache/flink/api/scala/typeutils/TraversableSerializerUpgradeTest.scala:
##########
@@ -23,58 +23,21 @@ import org.apache.flink.api.common.typeinfo.TypeInformation
 import org.apache.flink.api.common.typeutils.{TypeSerializer, 
TypeSerializerMatchers, TypeSerializerSchemaCompatibility, 
TypeSerializerUpgradeTestBase}
 import 
org.apache.flink.api.common.typeutils.TypeSerializerUpgradeTestBase.TestSpecification
 import org.apache.flink.api.scala.createTypeInformation
+import org.apache.flink.api.scala.typeutils.TraversableSerializerUpgradeTest._
+import 
org.apache.flink.api.scala.typeutils.TraversableSerializerUpgradeTest.Types.Pojo
 
 import org.hamcrest.Matcher
 import org.hamcrest.Matchers.is
-import org.junit.runner.RunWith
-import org.junit.runners.Parameterized
 
 import java.util
 import java.util.function.Supplier
 
 import scala.collection.{mutable, BitSet, LinearSeq}
 
 /** A [[TypeSerializerUpgradeTestBase]] for [[TraversableSerializer]]. */
-@RunWith(classOf[Parameterized])
-class TraversableSerializerUpgradeTest(
-    testSpecification: TypeSerializerUpgradeTestBase.TestSpecification[
-      TraversableOnce[_],
-      TraversableOnce[_]])
-  extends TypeSerializerUpgradeTestBase[TraversableOnce[_], 
TraversableOnce[_]](testSpecification)
-
-object TraversableSerializerUpgradeTest {
-
-  object Types {
-
-    class Pojo(var name: String, var count: Int) {
-      def this() = this("", -1)
-
-      override def equals(other: Any): Boolean = {
-        other match {
-          case oP: Pojo => name == oP.name && count == oP.count
-          case _ => false
-        }
-      }
-    }
-
-    val seqTypeInfo = implicitly[TypeInformation[Seq[Int]]]
-    val indexedSeqTypeInfo =
-      implicitly[TypeInformation[IndexedSeq[Int]]]
-    val linearSeqTypeInfo = implicitly[TypeInformation[LinearSeq[Int]]]
-    val mapTypeInfo = implicitly[TypeInformation[Map[String, Int]]]
-    val setTypeInfo = implicitly[TypeInformation[Set[Int]]]
-    val bitsetTypeInfo = implicitly[TypeInformation[BitSet]]
-    val mutableListTypeInfo =
-      implicitly[TypeInformation[mutable.MutableList[Int]]]
-    val seqTupleTypeInfo = implicitly[TypeInformation[Seq[(Int, String)]]]
-    val seqPojoTypeInfo = implicitly[TypeInformation[Seq[Pojo]]]
-  }
-
-  import Types._
-
-  @Parameterized.Parameters(name = "Test Specification = {0}")
-  def testSpecifications: util.Collection[TestSpecification[_, _]] = {
-
+class TraversableSerializerUpgradeTest
+  extends TypeSerializerUpgradeTestBase[TraversableOnce[_], 
TraversableOnce[_]] {
+  override def testData(): util.Collection[TestSpecification[_, _]] = {

Review Comment:
   ```suggestion
   
     override def testData(): util.Collection[TestSpecification[_, _]] = {
   ```
   nit



##########
flink-core/src/test/java/org/apache/flink/api/common/typeutils/TypeSerializerUpgradeTestBase.java:
##########
@@ -24,50 +24,52 @@
 import org.apache.flink.core.memory.DataInputView;
 import org.apache.flink.core.memory.DataOutputSerializer;
 import org.apache.flink.core.memory.DataOutputView;
-import org.apache.flink.util.TestLogger;
+import org.apache.flink.util.TestLoggerExtension;
 
+import org.assertj.core.api.HamcrestCondition;
 import org.hamcrest.Matcher;
-import org.junit.Ignore;
-import org.junit.Test;
+import org.junit.jupiter.api.Disabled;
+import org.junit.jupiter.api.TestInstance;
+import org.junit.jupiter.api.extension.ExtendWith;
+import org.junit.jupiter.params.ParameterizedTest;
+import org.junit.jupiter.params.provider.MethodSource;
 
 import java.io.IOException;
 import java.nio.file.Files;
 import java.nio.file.Path;
 import java.nio.file.Paths;
+import java.util.Collection;
 import java.util.Collections;
 import java.util.List;
 import java.util.Set;
 
 import static org.apache.flink.util.Preconditions.checkNotNull;
+import static org.assertj.core.api.AssertionsForInterfaceTypes.assertThat;
+import static org.assertj.core.api.Assumptions.assumeThat;
 import static org.hamcrest.CoreMatchers.not;
-import static org.junit.Assert.assertThat;
-import static org.junit.Assume.assumeThat;
 
 /**
  * A test base for testing {@link TypeSerializer} upgrades.
  *
- * <p>You can run {@link #generateTestSetupFiles()} on a Flink branch to 
(re-)generate the test data
- * files.
+ * <p>You can run {@link
+ * 
#generateTestSetupFiles(org.apache.flink.api.common.typeutils.TypeSerializerUpgradeTestBase.TestSpecification)}
+ * on a Flink branch to (re-)generate the test data files.
  */
-public abstract class TypeSerializerUpgradeTestBase<PreviousElementT, 
UpgradedElementT>
-        extends TestLogger {
+@ExtendWith(TestLoggerExtension.class)
+@TestInstance(TestInstance.Lifecycle.PER_CLASS)
+public abstract class TypeSerializerUpgradeTestBase<PreviousElementT, 
UpgradedElementT> {
 
     public static final FlinkVersion CURRENT_VERSION = FlinkVersion.v1_15;

Review Comment:
   This is the wrong base version. I created FLINK-29485 to cover this issue.



##########
flink-core/src/test/java/org/apache/flink/api/common/typeutils/TypeSerializerUpgradeTestBase.java:
##########
@@ -24,50 +24,52 @@
 import org.apache.flink.core.memory.DataInputView;
 import org.apache.flink.core.memory.DataOutputSerializer;
 import org.apache.flink.core.memory.DataOutputView;
-import org.apache.flink.util.TestLogger;
+import org.apache.flink.util.TestLoggerExtension;
 
+import org.assertj.core.api.HamcrestCondition;
 import org.hamcrest.Matcher;
-import org.junit.Ignore;
-import org.junit.Test;
+import org.junit.jupiter.api.Disabled;
+import org.junit.jupiter.api.TestInstance;
+import org.junit.jupiter.api.extension.ExtendWith;
+import org.junit.jupiter.params.ParameterizedTest;
+import org.junit.jupiter.params.provider.MethodSource;
 
 import java.io.IOException;
 import java.nio.file.Files;
 import java.nio.file.Path;
 import java.nio.file.Paths;
+import java.util.Collection;
 import java.util.Collections;
 import java.util.List;
 import java.util.Set;
 
 import static org.apache.flink.util.Preconditions.checkNotNull;
+import static org.assertj.core.api.AssertionsForInterfaceTypes.assertThat;
+import static org.assertj.core.api.Assumptions.assumeThat;
 import static org.hamcrest.CoreMatchers.not;
-import static org.junit.Assert.assertThat;
-import static org.junit.Assume.assumeThat;
 
 /**
  * A test base for testing {@link TypeSerializer} upgrades.
  *
- * <p>You can run {@link #generateTestSetupFiles()} on a Flink branch to 
(re-)generate the test data
- * files.
+ * <p>You can run {@link
+ * 
#generateTestSetupFiles(org.apache.flink.api.common.typeutils.TypeSerializerUpgradeTestBase.TestSpecification)}

Review Comment:
   ```suggestion
    * #generateTestSetupFiles(TestSpecification)}
   ```
   The unqualified reference should be sufficient enough here.



##########
flink-core/src/test/java/org/apache/flink/api/common/typeutils/TypeSerializerUpgradeTestBase.java:
##########
@@ -24,50 +24,52 @@
 import org.apache.flink.core.memory.DataInputView;
 import org.apache.flink.core.memory.DataOutputSerializer;
 import org.apache.flink.core.memory.DataOutputView;
-import org.apache.flink.util.TestLogger;
+import org.apache.flink.util.TestLoggerExtension;
 
+import org.assertj.core.api.HamcrestCondition;
 import org.hamcrest.Matcher;
-import org.junit.Ignore;
-import org.junit.Test;
+import org.junit.jupiter.api.Disabled;
+import org.junit.jupiter.api.TestInstance;
+import org.junit.jupiter.api.extension.ExtendWith;
+import org.junit.jupiter.params.ParameterizedTest;
+import org.junit.jupiter.params.provider.MethodSource;
 
 import java.io.IOException;
 import java.nio.file.Files;
 import java.nio.file.Path;
 import java.nio.file.Paths;
+import java.util.Collection;
 import java.util.Collections;
 import java.util.List;
 import java.util.Set;
 
 import static org.apache.flink.util.Preconditions.checkNotNull;
+import static org.assertj.core.api.AssertionsForInterfaceTypes.assertThat;
+import static org.assertj.core.api.Assumptions.assumeThat;
 import static org.hamcrest.CoreMatchers.not;
-import static org.junit.Assert.assertThat;
-import static org.junit.Assume.assumeThat;
 
 /**
  * A test base for testing {@link TypeSerializer} upgrades.
  *
- * <p>You can run {@link #generateTestSetupFiles()} on a Flink branch to 
(re-)generate the test data
- * files.
+ * <p>You can run {@link
+ * 
#generateTestSetupFiles(org.apache.flink.api.common.typeutils.TypeSerializerUpgradeTestBase.TestSpecification)}
+ * on a Flink branch to (re-)generate the test data files.
  */
-public abstract class TypeSerializerUpgradeTestBase<PreviousElementT, 
UpgradedElementT>
-        extends TestLogger {
+@ExtendWith(TestLoggerExtension.class)
+@TestInstance(TestInstance.Lifecycle.PER_CLASS)
+public abstract class TypeSerializerUpgradeTestBase<PreviousElementT, 
UpgradedElementT> {
 
     public static final FlinkVersion CURRENT_VERSION = FlinkVersion.v1_15;
 
     public static final Set<FlinkVersion> MIGRATION_VERSIONS =
             FlinkVersion.rangeOf(FlinkVersion.v1_11, CURRENT_VERSION);
 
-    private final TestSpecification<PreviousElementT, UpgradedElementT> 
testSpecification;
-
-    protected TypeSerializerUpgradeTestBase(
-            TestSpecification<PreviousElementT, UpgradedElementT> 
testSpecification) {
-        this.testSpecification = checkNotNull(testSpecification);
-    }
-
     // 
------------------------------------------------------------------------------
     //  APIs
     // 
------------------------------------------------------------------------------
 
+    public abstract Collection<TestSpecification<?, ?>> testData() throws 
Exception;

Review Comment:
   ```suggestion
       public abstract Collection<TestSpecification<?, ?>> 
createTestSpecifications() throws Exception;
   ```
   I feel like `testData` doesn't describe properly what this method does. 
Essentially, it's a factory method (i.e. `create`) of `TestSpecification` 
instances rather than actual test data. WDYT?



##########
flink-core/src/test/java/org/apache/flink/api/common/typeutils/TypeSerializerUpgradeTestBase.java:
##########
@@ -24,50 +24,52 @@
 import org.apache.flink.core.memory.DataInputView;
 import org.apache.flink.core.memory.DataOutputSerializer;
 import org.apache.flink.core.memory.DataOutputView;
-import org.apache.flink.util.TestLogger;
+import org.apache.flink.util.TestLoggerExtension;
 
+import org.assertj.core.api.HamcrestCondition;
 import org.hamcrest.Matcher;
-import org.junit.Ignore;
-import org.junit.Test;
+import org.junit.jupiter.api.Disabled;
+import org.junit.jupiter.api.TestInstance;
+import org.junit.jupiter.api.extension.ExtendWith;
+import org.junit.jupiter.params.ParameterizedTest;
+import org.junit.jupiter.params.provider.MethodSource;
 
 import java.io.IOException;
 import java.nio.file.Files;
 import java.nio.file.Path;
 import java.nio.file.Paths;
+import java.util.Collection;
 import java.util.Collections;
 import java.util.List;
 import java.util.Set;
 
 import static org.apache.flink.util.Preconditions.checkNotNull;
+import static org.assertj.core.api.AssertionsForInterfaceTypes.assertThat;
+import static org.assertj.core.api.Assumptions.assumeThat;
 import static org.hamcrest.CoreMatchers.not;
-import static org.junit.Assert.assertThat;
-import static org.junit.Assume.assumeThat;
 
 /**
  * A test base for testing {@link TypeSerializer} upgrades.
  *
- * <p>You can run {@link #generateTestSetupFiles()} on a Flink branch to 
(re-)generate the test data
- * files.
+ * <p>You can run {@link
+ * 
#generateTestSetupFiles(org.apache.flink.api.common.typeutils.TypeSerializerUpgradeTestBase.TestSpecification)}
+ * on a Flink branch to (re-)generate the test data files.
  */
-public abstract class TypeSerializerUpgradeTestBase<PreviousElementT, 
UpgradedElementT>
-        extends TestLogger {
+@ExtendWith(TestLoggerExtension.class)
+@TestInstance(TestInstance.Lifecycle.PER_CLASS)
+public abstract class TypeSerializerUpgradeTestBase<PreviousElementT, 
UpgradedElementT> {
 
     public static final FlinkVersion CURRENT_VERSION = FlinkVersion.v1_15;
 
     public static final Set<FlinkVersion> MIGRATION_VERSIONS =
             FlinkVersion.rangeOf(FlinkVersion.v1_11, CURRENT_VERSION);
 
-    private final TestSpecification<PreviousElementT, UpgradedElementT> 
testSpecification;
-
-    protected TypeSerializerUpgradeTestBase(
-            TestSpecification<PreviousElementT, UpgradedElementT> 
testSpecification) {
-        this.testSpecification = checkNotNull(testSpecification);
-    }
-
     // 
------------------------------------------------------------------------------
     //  APIs
     // 
------------------------------------------------------------------------------
 
+    public abstract Collection<TestSpecification<?, ?>> testData() throws 
Exception;

Review Comment:
   We should add proper JavaDoc here as well.



##########
flink-scala/src/test/scala/org/apache/flink/api/scala/typeutils/ScalaCaseClassSerializerUpgradeTest.scala:
##########
@@ -23,32 +23,17 @@ import 
org.apache.flink.api.common.typeutils.{TypeSerializer, TypeSerializerMatc
 import 
org.apache.flink.api.common.typeutils.TypeSerializerUpgradeTestBase.TestSpecification
 import org.apache.flink.api.scala.createTypeInformation
 import org.apache.flink.api.scala.types.CustomCaseClass
+import 
org.apache.flink.api.scala.typeutils.ScalaCaseClassSerializerUpgradeTest.{ScalaCaseClassSerializerSetup,
 ScalaCaseClassSerializerVerifier}
 
 import org.hamcrest.Matcher
 import org.hamcrest.Matchers.is
-import org.junit.runner.RunWith
-import org.junit.runners.Parameterized
 
 import java.util
 
 /** A [[TypeSerializerUpgradeTestBase]] for [[ScalaCaseClassSerializer]]. */
-@RunWith(classOf[Parameterized])
-class ScalaCaseClassSerializerUpgradeTest(
-    spec: TestSpecification[CustomCaseClass, CustomCaseClass]
-) extends TypeSerializerUpgradeTestBase[CustomCaseClass, 
CustomCaseClass](spec) {}
-
-object ScalaCaseClassSerializerUpgradeTest {
-
-  private val typeInfo = createTypeInformation[CustomCaseClass]
-
-  private val supplier =
-    new util.function.Supplier[TypeSerializer[CustomCaseClass]] {
-      override def get(): TypeSerializer[CustomCaseClass] =
-        typeInfo.createSerializer(new ExecutionConfig)
-    }
-
-  @Parameterized.Parameters(name = "Test Specification = {0}")
-  def testSpecifications(): util.Collection[TestSpecification[_, _]] = {
+class ScalaCaseClassSerializerUpgradeTest
+  extends TypeSerializerUpgradeTestBase[CustomCaseClass, CustomCaseClass] {
+  override def testData(): util.Collection[TestSpecification[_, _]] = {

Review Comment:
   ```suggestion
   
     override def testData(): util.Collection[TestSpecification[_, _]] = {
   ```
   nit: for readability purposes



##########
flink-core/src/test/java/org/apache/flink/api/java/typeutils/runtime/RowSerializerUpgradeTest.java:
##########
@@ -40,15 +38,9 @@
 import static org.hamcrest.Matchers.is;
 
 /** A {@link TypeSerializerUpgradeTestBase} for {@link RowSerializer}. */
-@RunWith(Parameterized.class)
 public class RowSerializerUpgradeTest extends 
TypeSerializerUpgradeTestBase<Row, Row> {

Review Comment:
   ```suggestion
   class RowSerializerUpgradeTest extends TypeSerializerUpgradeTestBase<Row, 
Row> {
   ```



##########
flink-core/src/test/java/org/apache/flink/api/java/typeutils/runtime/CopyableSerializerUpgradeTest.java:
##########
@@ -29,29 +29,20 @@
 import org.apache.flink.types.CopyableValue;
 
 import org.hamcrest.Matcher;
-import org.junit.Assert;
-import org.junit.Test;
-import org.junit.runner.RunWith;
-import org.junit.runners.Parameterized;
+import org.junit.jupiter.api.Test;
 
 import java.io.IOException;
 import java.util.ArrayList;
 import java.util.Collection;
 
+import static org.assertj.core.api.Assertions.assertThat;
 import static org.hamcrest.Matchers.is;
 
 /** A {@link TypeSerializerUpgradeTestBase} for {@link 
CopyableValueSerializer}. */
-@RunWith(Parameterized.class)
 public class CopyableSerializerUpgradeTest

Review Comment:
   ```suggestion
   class CopyableSerializerUpgradeTest
   ```



##########
flink-scala/src/test/scala/org/apache/flink/api/scala/typeutils/EnumValueSerializerUpgradeTest.scala:
##########
@@ -20,29 +20,17 @@ package org.apache.flink.api.scala.typeutils
 import org.apache.flink.FlinkVersion
 import org.apache.flink.api.common.typeutils.{TypeSerializer, 
TypeSerializerMatchers, TypeSerializerSchemaCompatibility, 
TypeSerializerUpgradeTestBase}
 import 
org.apache.flink.api.common.typeutils.TypeSerializerUpgradeTestBase.TestSpecification
+import 
org.apache.flink.api.scala.typeutils.EnumValueSerializerUpgradeTest.{EnumValueSerializerSetup,
 EnumValueSerializerVerifier}
 
 import org.hamcrest.Matcher
 import org.hamcrest.Matchers.is
-import org.junit.runner.RunWith
-import org.junit.runners.Parameterized
 
 import java.util
 
 /** A [[TypeSerializerUpgradeTestBase]] for [[EnumValueSerializer]]. */
-@RunWith(classOf[Parameterized])
-class EnumValueSerializerUpgradeTest(spec: TestSpecification[Letters.Value, 
Letters.Value])
-  extends TypeSerializerUpgradeTestBase[Letters.Value, Letters.Value](spec) {}
-
-object EnumValueSerializerUpgradeTest {
-
-  private val supplier =
-    new util.function.Supplier[EnumValueSerializer[Letters.type]] {
-      override def get(): EnumValueSerializer[Letters.type] =
-        new EnumValueSerializer(Letters)
-    }
-
-  @Parameterized.Parameters(name = "Test Specification = {0}")
-  def testSpecifications(): util.Collection[TestSpecification[_, _]] = {
+class EnumValueSerializerUpgradeTest
+  extends TypeSerializerUpgradeTestBase[Letters.Value, Letters.Value] {
+  override def testData(): util.Collection[TestSpecification[_, _]] = {

Review Comment:
   ```suggestion
   
     override def testData(): util.Collection[TestSpecification[_, _]] = {
   ```
   nit: for readability purposes



##########
flink-streaming-java/src/test/java/org/apache/flink/streaming/api/functions/sink/TwoPhaseCommitSinkStateSerializerUpgradeTest.java:
##########
@@ -39,22 +37,12 @@
 /**
  * A {@link TypeSerializerUpgradeTestBase} for {@link 
TwoPhaseCommitSinkFunction.StateSerializer}.
  */
-@RunWith(Parameterized.class)
 public class TwoPhaseCommitSinkStateSerializerUpgradeTest

Review Comment:
   ```suggestion
   class TwoPhaseCommitSinkStateSerializerUpgradeTest
   ```



##########
flink-core/src/test/java/org/apache/flink/api/java/typeutils/runtime/CopyableSerializerUpgradeTest.java:
##########
@@ -168,8 +159,8 @@ public Matcher<SimpleCopyable> testDataMatcher() {
     }
 
     @Test
-    public void testF() {
+    void testF() {

Review Comment:
   I'm not even sure whether we need that here. But it doesn't hurt... :shrug: 



##########
flink-core/src/test/java/org/apache/flink/api/common/typeutils/TypeSerializerUpgradeTestBase.java:
##########
@@ -46,28 +50,25 @@
 /**
  * A test base for testing {@link TypeSerializer} upgrades.
  *
- * <p>You can run {@link #generateTestSetupFiles()} on a Flink branch to 
(re-)generate the test data
- * files.
+ * <p>You can run {@link
+ * 
#generateTestSetupFiles(org.apache.flink.api.common.typeutils.TypeSerializerUpgradeTestBase.TestSpecification)}
+ * on a Flink branch to (re-)generate the test data files.
  */
-public abstract class TypeSerializerUpgradeTestBase<PreviousElementT, 
UpgradedElementT>
-        extends TestLogger {
+@ExtendWith(TestLoggerExtension.class)
+@TestInstance(TestInstance.Lifecycle.PER_CLASS)

Review Comment:
   That changes the grouping of tests executing all versions per test method in 
a group rather than having all test methods grouped by Flink base version. But 
that shouldn't be that big of a deal. I tend to prefer the new version even.



##########
flink-streaming-java/src/test/java/org/apache/flink/streaming/api/operators/TimerSerializerUpgradeTest.java:
##########
@@ -26,31 +26,19 @@
 import org.apache.flink.api.common.typeutils.base.StringSerializer;
 
 import org.hamcrest.Matcher;
-import org.junit.runner.RunWith;
-import org.junit.runners.Parameterized;
 
 import java.util.ArrayList;
 import java.util.Collection;
 
 import static org.hamcrest.Matchers.is;
 
 /** Migration test for {@link TimerSerializer}. */
-@RunWith(Parameterized.class)
-public class TimerSerializerUpgradeTest
+class TimerSerializerUpgradeTest
         extends TypeSerializerUpgradeTestBase<
                 TimerHeapInternalTimer<String, Integer>, 
TimerHeapInternalTimer<String, Integer>> {
 
-    public TimerSerializerUpgradeTest(
-            TestSpecification<
-                            TimerHeapInternalTimer<String, Integer>,
-                            TimerHeapInternalTimer<String, Integer>>
-                    testSpecification) {
-        super(testSpecification);
-    }
-
     @SuppressWarnings("unchecked")

Review Comment:
   there's also `stringIntTimerSerializerSupplier` below that is unused and 
could be removed in a separate hotfix commit. It looks like it was never used 
:thinking: please verify



##########
flink-table/flink-table-runtime/src/test/java/org/apache/flink/table/runtime/typeutils/LinkedListSerializerUpgradeTest.java:
##########
@@ -37,17 +35,10 @@
 import static org.hamcrest.Matchers.is;
 
 /** A {@link TypeSerializerUpgradeTestBase} for {@link LinkedListSerializer}. 
*/
-@RunWith(Parameterized.class)
 public class LinkedListSerializerUpgradeTest

Review Comment:
   ```suggestion
   class LinkedListSerializerUpgradeTest
   ```



##########
flink-streaming-java/src/test/java/org/apache/flink/streaming/api/operators/TimerSerializerUpgradeTest.java:
##########
@@ -26,31 +26,19 @@
 import org.apache.flink.api.common.typeutils.base.StringSerializer;
 
 import org.hamcrest.Matcher;
-import org.junit.runner.RunWith;
-import org.junit.runners.Parameterized;
 
 import java.util.ArrayList;
 import java.util.Collection;
 
 import static org.hamcrest.Matchers.is;
 
 /** Migration test for {@link TimerSerializer}. */
-@RunWith(Parameterized.class)
-public class TimerSerializerUpgradeTest
+class TimerSerializerUpgradeTest
         extends TypeSerializerUpgradeTestBase<
                 TimerHeapInternalTimer<String, Integer>, 
TimerHeapInternalTimer<String, Integer>> {
 
-    public TimerSerializerUpgradeTest(
-            TestSpecification<
-                            TimerHeapInternalTimer<String, Integer>,
-                            TimerHeapInternalTimer<String, Integer>>
-                    testSpecification) {
-        super(testSpecification);
-    }
-
     @SuppressWarnings("unchecked")

Review Comment:
   ```suggestion
       @SuppressWarnings("unchecked")
   ```
   unused



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: [email protected]

For queries about this service, please contact Infrastructure at:
[email protected]

Reply via email to