Parcourir la source

YARN-10803. [JDK 11] TestRMFailoverProxyProvider and TestNoHaRMFailoverProxyProvider fails by ClassCastException. (#3068)

Reviewed-by: Takanobu Asanuma <tasanuma@apache.org>
(cherry picked from commit 9a17f9713376b23fb3dfeb898282dbfd61c64962)
Akira Ajisaka il y a 4 ans
Parent
commit
19c969981d

+ 10 - 10
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/TestNoHaRMFailoverProxyProvider.java

@@ -140,9 +140,9 @@ public class TestNoHaRMFailoverProxyProvider {
     // Create a proxy and mock a RMProxy
     Proxy mockProxy1 = new TestProxy((proxy, method, args) -> null);
     Class protocol = ApplicationClientProtocol.class;
-    RMProxy mockRMProxy = mock(RMProxy.class);
-    DefaultNoHARMFailoverProxyProvider <RMProxy> fpp =
-        new DefaultNoHARMFailoverProxyProvider<RMProxy>();
+    RMProxy<Proxy> mockRMProxy = mock(RMProxy.class);
+    DefaultNoHARMFailoverProxyProvider<Proxy> fpp =
+        new DefaultNoHARMFailoverProxyProvider<>();
 
     InetSocketAddress mockAdd1 = new InetSocketAddress(RM1_PORT);
 
@@ -154,7 +154,7 @@ public class TestNoHaRMFailoverProxyProvider {
 
     // Initialize failover proxy provider and get proxy from it.
     fpp.init(conf, mockRMProxy, protocol);
-    FailoverProxyProvider.ProxyInfo<RMProxy> actualProxy1 = fpp.getProxy();
+    FailoverProxyProvider.ProxyInfo<Proxy> actualProxy1 = fpp.getProxy();
     assertEquals(
         "AutoRefreshRMFailoverProxyProvider doesn't generate " +
         "expected proxy",
@@ -180,7 +180,7 @@ public class TestNoHaRMFailoverProxyProvider {
 
     // Perform Failover and get proxy again from failover proxy provider
     fpp.performFailover(actualProxy1.proxy);
-    FailoverProxyProvider.ProxyInfo<RMProxy> actualProxy2 = fpp.getProxy();
+    FailoverProxyProvider.ProxyInfo<Proxy> actualProxy2 = fpp.getProxy();
     assertEquals("AutoRefreshRMFailoverProxyProvider " +
         "doesn't generate expected proxy after failover",
         mockProxy1, actualProxy2.proxy);
@@ -208,9 +208,9 @@ public class TestNoHaRMFailoverProxyProvider {
     Proxy mockProxy1 = new TestProxy((proxy, method, args) -> null);
     Proxy mockProxy2 = new TestProxy((proxy, method, args) -> null);
     Class protocol = ApplicationClientProtocol.class;
-    RMProxy mockRMProxy = mock(RMProxy.class);
-    AutoRefreshNoHARMFailoverProxyProvider<RMProxy> fpp =
-        new AutoRefreshNoHARMFailoverProxyProvider<RMProxy>();
+    RMProxy<Proxy> mockRMProxy = mock(RMProxy.class);
+    AutoRefreshNoHARMFailoverProxyProvider<Proxy> fpp =
+        new AutoRefreshNoHARMFailoverProxyProvider<>();
 
     // generate two address with different ports.
     InetSocketAddress mockAdd1 = new InetSocketAddress(RM1_PORT);
@@ -224,7 +224,7 @@ public class TestNoHaRMFailoverProxyProvider {
 
     // Initialize proxy provider and get proxy from it.
     fpp.init(conf, mockRMProxy, protocol);
-    FailoverProxyProvider.ProxyInfo <RMProxy> actualProxy1 = fpp.getProxy();
+    FailoverProxyProvider.ProxyInfo<Proxy> actualProxy1 = fpp.getProxy();
     assertEquals(
         "AutoRefreshRMFailoverProxyProvider doesn't generate " +
         "expected proxy",
@@ -259,7 +259,7 @@ public class TestNoHaRMFailoverProxyProvider {
 
     // Perform Failover and get proxy again from failover proxy provider
     fpp.performFailover(actualProxy1.proxy);
-    FailoverProxyProvider.ProxyInfo <RMProxy> actualProxy2 = fpp.getProxy();
+    FailoverProxyProvider.ProxyInfo<Proxy> actualProxy2 = fpp.getProxy();
     assertEquals("AutoRefreshNoHARMFailoverProxyProvider " +
         "doesn't generate expected proxy after failover",
         mockProxy2, actualProxy2.proxy);

+ 12 - 13
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/TestRMFailoverProxyProvider.java

@@ -87,9 +87,9 @@ public class TestRMFailoverProxyProvider {
     Proxy mockProxy1 = new TestProxy((proxy, method, args) -> null);
 
     Class protocol = ApplicationClientProtocol.class;
-    RMProxy mockRMProxy = mock(RMProxy.class);
-    ConfiguredRMFailoverProxyProvider<RMProxy> fpp =
-        new ConfiguredRMFailoverProxyProvider<RMProxy>();
+    RMProxy<Proxy> mockRMProxy = mock(RMProxy.class);
+    ConfiguredRMFailoverProxyProvider<Proxy> fpp =
+        new ConfiguredRMFailoverProxyProvider<>();
 
     // generate two address with different ports.
     // Default port of yarn RM
@@ -104,7 +104,7 @@ public class TestRMFailoverProxyProvider {
 
     // Initialize failover proxy provider and get proxy from it.
     fpp.init(conf, mockRMProxy, protocol);
-    FailoverProxyProvider.ProxyInfo<RMProxy> actualProxy1 = fpp.getProxy();
+    FailoverProxyProvider.ProxyInfo<Proxy> actualProxy1 = fpp.getProxy();
     assertEquals(
         "ConfiguredRMFailoverProxyProvider doesn't generate " +
         "expected proxy",
@@ -139,7 +139,7 @@ public class TestRMFailoverProxyProvider {
 
     // Perform Failover and get proxy again from failover proxy provider
     fpp.performFailover(actualProxy1.proxy);
-    FailoverProxyProvider.ProxyInfo <RMProxy> actualProxy2 = fpp.getProxy();
+    FailoverProxyProvider.ProxyInfo<Proxy> actualProxy2 = fpp.getProxy();
     assertEquals("ConfiguredRMFailoverProxyProvider " +
         "doesn't generate expected proxy after failover",
         mockProxy2, actualProxy2.proxy);
@@ -167,7 +167,7 @@ public class TestRMFailoverProxyProvider {
 
     // Perform Failover and get proxy again from failover proxy provider
     fpp.performFailover(actualProxy2.proxy);
-    FailoverProxyProvider.ProxyInfo <RMProxy> actualProxy3 = fpp.getProxy();
+    FailoverProxyProvider.ProxyInfo<Proxy> actualProxy3 = fpp.getProxy();
 
     // check the proxy is the same as the one we created before.
     assertEquals("ConfiguredRMFailoverProxyProvider " +
@@ -202,11 +202,10 @@ public class TestRMFailoverProxyProvider {
     // Create three proxies and mock a RMProxy
     Proxy mockProxy1 = new TestProxy((proxy, method, args) -> null);
     Proxy mockProxy2 = new TestProxy((proxy, method, args) -> null);
-    Proxy mockProxy3 = new TestProxy((proxy, method, args) -> null);
     Class protocol = ApplicationClientProtocol.class;
-    RMProxy mockRMProxy = mock(RMProxy.class);
-    AutoRefreshRMFailoverProxyProvider<RMProxy> fpp =
-        new AutoRefreshRMFailoverProxyProvider<RMProxy>();
+    RMProxy<Proxy> mockRMProxy = mock(RMProxy.class);
+    AutoRefreshRMFailoverProxyProvider<Proxy> fpp =
+        new AutoRefreshRMFailoverProxyProvider<>();
 
     // generate three address with different ports.
     InetSocketAddress mockAdd1 = new InetSocketAddress(RM1_PORT);
@@ -222,7 +221,7 @@ public class TestRMFailoverProxyProvider {
 
     // Initialize failover proxy provider and get proxy from it.
     fpp.init(conf, mockRMProxy, protocol);
-    FailoverProxyProvider.ProxyInfo <RMProxy> actualProxy1 = fpp.getProxy();
+    FailoverProxyProvider.ProxyInfo<Proxy> actualProxy1 = fpp.getProxy();
     assertEquals(
         "AutoRefreshRMFailoverProxyProvider doesn't generate " +
         "expected proxy",
@@ -257,7 +256,7 @@ public class TestRMFailoverProxyProvider {
 
     // Perform Failover and get proxy again from failover proxy provider
     fpp.performFailover(actualProxy1.proxy);
-    FailoverProxyProvider.ProxyInfo <RMProxy> actualProxy2 = fpp.getProxy();
+    FailoverProxyProvider.ProxyInfo<Proxy> actualProxy2 = fpp.getProxy();
     assertEquals("AutoRefreshRMFailoverProxyProvider " +
         "doesn't generate expected proxy after failover",
         mockProxy2, actualProxy2.proxy);
@@ -285,7 +284,7 @@ public class TestRMFailoverProxyProvider {
 
     // Perform Failover and get proxy again from failover proxy provider
     fpp.performFailover(actualProxy2.proxy);
-    FailoverProxyProvider.ProxyInfo <RMProxy> actualProxy3 = fpp.getProxy();
+    FailoverProxyProvider.ProxyInfo<Proxy> actualProxy3 = fpp.getProxy();
 
     // check the proxy is the same as the one we created before.
     assertEquals("ConfiguredRMFailoverProxyProvider " +