[ 
https://issues.apache.org/jira/browse/HADOOP-18302?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17580203#comment-17580203
 ] 

ASF GitHub Bot commented on HADOOP-18302:
-----------------------------------------

aajisaka commented on code in PR #4457:
URL: https://github.com/apache/hadoop/pull/4457#discussion_r946542663


##########
hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/ipc/TestRPC.java:
##########
@@ -1219,9 +1218,8 @@ public void testClientBackOff() throws Exception {
     server = setupTestServer(builder);
 
     @SuppressWarnings("unchecked")

Review Comment:
   We can remove `@SuppressWarnings("unchecked")` here



##########
hadoop-common-project/hadoop-nfs/src/main/java/org/apache/hadoop/portmap/RpcProgramPortmap.java:
##########
@@ -208,4 +208,8 @@ public void exceptionCaught(ChannelHandlerContext ctx, 
Throwable t) {
     LOG.warn("Encountered ", t);
     ctx.channel().close();
   }
+
+  public ConcurrentHashMap<String, PortmapMapping> getMap() {
+    return map;
+  }

Review Comment:
   This method can be package-private and we should use more abstract return 
type (in this case, `Map`).



##########
hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/http/TestHttpServer.java:
##########
@@ -740,8 +738,7 @@ public void testBacklogSize() throws Exception
     Configuration conf = new Configuration();
     conf.setInt(HttpServer2.HTTP_SOCKET_BACKLOG_SIZE_KEY, backlogSize);
     HttpServer2 srv = createServer("test", conf);
-    List<?> listeners = (List<?>) Whitebox.getInternalState(srv,
-            "listeners");
+    List<?> listeners = srv.getListeners();
     ServerConnector listener = (ServerConnector)listeners.get(0);

Review Comment:
   The same as above



##########
hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/metrics2/impl/TestStatsDMetrics.java:
##########
@@ -31,13 +31,13 @@
 import java.util.List;
 import java.util.Set;
 
+import org.apache.commons.lang3.reflect.FieldUtils;

Review Comment:
   We should use getter and setter instead of relying on the FieldUtils.



##########
hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/ipc/TestIPC.java:
##########
@@ -881,9 +880,8 @@ private void checkBlocking(int readers, int readerQ, int 
callQ) throws Exception
     // start server
     final TestServerQueue server =
         new TestServerQueue(clients, readers, callQ, handlers, conf);
-    CallQueueManager<Call> spy = spy(
-        (CallQueueManager<Call>)Whitebox.getInternalState(server, 
"callQueue"));
-    Whitebox.setInternalState(server, "callQueue", spy);
+    CallQueueManager<Call> spy = spy(server.getCallQueue());
+    server.setCallQueue(spy);

Review Comment:
   We can remove `@SuppressWarnings("unchecked")` from this method.



##########
hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/ipc/TestRPC.java:
##########
@@ -1282,9 +1280,8 @@ public void testClientBackOffByResponseTime() throws 
Exception {
     Server server = setupDecayRpcSchedulerandTestServer(ns + ".");
 
     @SuppressWarnings("unchecked")

Review Comment:
   The same as above



##########
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/metrics2/sink/GraphiteSink.java:
##########
@@ -37,171 +37,176 @@
 import java.nio.charset.StandardCharsets;
 
 /**
- * A metrics sink that writes to a Graphite server
+ * A metrics sink that writes to a Graphite server.
  */
 @InterfaceAudience.Public
 @InterfaceStability.Evolving
 public class GraphiteSink implements MetricsSink, Closeable {
-    private static final Logger LOG =
-        LoggerFactory.getLogger(GraphiteSink.class);
-    private static final String SERVER_HOST_KEY = "server_host";
-    private static final String SERVER_PORT_KEY = "server_port";
-    private static final String METRICS_PREFIX = "metrics_prefix";
-    private String metricsPrefix = null;
-    private Graphite graphite = null;
-
-    @Override
-    public void init(SubsetConfiguration conf) {
-        // Get Graphite host configurations.
-        final String serverHost = conf.getString(SERVER_HOST_KEY);
-        final int serverPort = 
Integer.parseInt(conf.getString(SERVER_PORT_KEY));
-
-        // Get Graphite metrics graph prefix.
-        metricsPrefix = conf.getString(METRICS_PREFIX);
-        if (metricsPrefix == null)
-            metricsPrefix = "";
-
-        graphite = new Graphite(serverHost, serverPort);
-        graphite.connect();
+  private static final Logger LOG =
+      LoggerFactory.getLogger(GraphiteSink.class);
+  private static final String SERVER_HOST_KEY = "server_host";
+  private static final String SERVER_PORT_KEY = "server_port";
+  private static final String METRICS_PREFIX = "metrics_prefix";
+  private String metricsPrefix = null;
+  private Graphite graphite = null;
+
+  @Override
+  public void init(SubsetConfiguration conf) {
+    // Get Graphite host configurations.
+    final String serverHost = conf.getString(SERVER_HOST_KEY);
+    final int serverPort = Integer.parseInt(conf.getString(SERVER_PORT_KEY));
+
+    // Get Graphite metrics graph prefix.
+    metricsPrefix = conf.getString(METRICS_PREFIX);
+    if (metricsPrefix == null) {
+      metricsPrefix = "";
     }
 
-    @Override
-    public void putMetrics(MetricsRecord record) {
-        StringBuilder lines = new StringBuilder();
-        StringBuilder metricsPathPrefix = new StringBuilder();
-
-        // Configure the hierarchical place to display the graph.
-        metricsPathPrefix.append(metricsPrefix).append(".")
-                .append(record.context()).append(".").append(record.name());
-
-        for (MetricsTag tag : record.tags()) {
-            if (tag.value() != null) {
-                metricsPathPrefix.append(".")
-                    .append(tag.name())
-                    .append("=")
-                    .append(tag.value());
-            }
-        }
-
-        // The record timestamp is in milliseconds while Graphite expects an 
epoc time in seconds.
-        long timestamp = record.timestamp() / 1000L;
+    graphite = new Graphite(serverHost, serverPort);
+    graphite.connect();
+  }
+
+  @Override
+  public void putMetrics(MetricsRecord record) {
+    StringBuilder lines = new StringBuilder();
+    StringBuilder metricsPathPrefix = new StringBuilder();
+
+    // Configure the hierarchical place to display the graph.
+    metricsPathPrefix.append(metricsPrefix).append(".")
+        .append(record.context()).append(".").append(record.name());
+
+    for (MetricsTag tag : record.tags()) {
+      if (tag.value() != null) {
+        metricsPathPrefix.append(".")
+            .append(tag.name())
+            .append("=")
+            .append(tag.value());
+      }
+    }
 
-        // Collect datapoints.
-        for (AbstractMetric metric : record.metrics()) {
-            lines.append(
-                    metricsPathPrefix.toString() + "."
-                            + metric.name().replace(' ', '.')).append(" ")
-                    .append(metric.value()).append(" ").append(timestamp)
-                    .append("\n");
-        }
+    // The record timestamp is in milliseconds while Graphite expects an epoc 
time in seconds.
+    long timestamp = record.timestamp() / 1000L;
 
-        try {
-          graphite.write(lines.toString());
-        } catch (Exception e) {
-          LOG.warn("Error sending metrics to Graphite", e);
-          try {
-            graphite.close();
-          } catch (Exception e1) {
-            throw new MetricsException("Error closing connection to Graphite", 
e1);
-          }
-        }
+    // Collect datapoints.
+    for (AbstractMetric metric : record.metrics()) {
+      lines.append(metricsPathPrefix + "." + metric.name().replace(' ', 
'.')).append(" ")
+           .append(metric.value()).append(" ").append(timestamp)
+           .append("\n");
     }
 
-    @Override
-    public void flush() {
+    try {
+      graphite.write(lines.toString());
+    } catch (Exception e) {
+      LOG.warn("Error sending metrics to Graphite.", e);
       try {
-        graphite.flush();
-      } catch (Exception e) {
-        LOG.warn("Error flushing metrics to Graphite", e);
-        try {
-          graphite.close();
-        } catch (Exception e1) {
-          throw new MetricsException("Error closing connection to Graphite", 
e1);
-        }
+        graphite.close();
+      } catch (Exception e1) {
+        throw new MetricsException("Error closing connection to Graphite", e1);
       }
     }
-
-    @Override
-    public void close() throws IOException {
-      graphite.close();
+  }
+
+  @Override
+  public void flush() {
+    try {
+      graphite.flush();
+    } catch (Exception e) {
+      LOG.warn("Error flushing metrics to Graphite.", e);
+      try {
+        graphite.close();
+      } catch (Exception e1) {
+        throw new MetricsException("Error closing connection to Graphite.", 
e1);
+      }
     }
+  }
 
-    public static class Graphite {
-      private final static int MAX_CONNECTION_FAILURES = 5;
+  @Override
+  public void close() throws IOException {
+    graphite.close();
+  }
 
-      private String serverHost;
-      private int serverPort;
-      private Writer writer = null;
-      private Socket socket = null;
-      private int connectionFailures = 0;
+  public static class Graphite {
+    private final static int MAX_CONNECTION_FAILURES = 5;
 
-      public Graphite(String serverHost, int serverPort) {
-        this.serverHost = serverHost;
-        this.serverPort = serverPort;
-      }
+    private String serverHost;
+    private int serverPort;
+    private Writer writer = null;
+    private Socket socket = null;
+    private int connectionFailures = 0;
 
-      public void connect() {
-        if (isConnected()) {
-          throw new MetricsException("Already connected to Graphite");
-        }
-        if (tooManyConnectionFailures()) {
-          // return silently (there was ERROR in logs when we reached limit 
for the first time)
-          return;
-        }
-        try {
+    public Graphite(String serverHost, int serverPort) {
+      this.serverHost = serverHost;
+      this.serverPort = serverPort;
+    }
+
+    public void connect() {
+      if (isConnected()) {
+        throw new MetricsException("Already connected to Graphite");
+      }
+      if (tooManyConnectionFailures()) {
+        // return silently (there was ERROR in logs when we reached limit for 
the first time)
+        return;
+      }
+      try {
           // Open a connection to Graphite server.
-          socket = new Socket(serverHost, serverPort);
+        socket = new Socket(serverHost, serverPort);
         writer = new OutputStreamWriter(socket.getOutputStream(),
                 StandardCharsets.UTF_8);
-        } catch (Exception e) {
-          connectionFailures++;
-          if (tooManyConnectionFailures()) {
-            // first time when connection limit reached, report to logs
-            LOG.error("Too many connection failures, would not try to connect 
again.");
-          }
-          throw new MetricsException("Error creating connection, "
-              + serverHost + ":" + serverPort, e);
+      } catch (Exception e) {
+        connectionFailures++;
+        if (tooManyConnectionFailures()) {
+          // first time when connection limit reached, report to logs
+          LOG.error("Too many connection failures, would not try to connect 
again.");
         }
+        throw new MetricsException("Error creating connection, " +
+            serverHost + ":" + serverPort, e);
       }
+    }
 
-      public void write(String msg) throws IOException {
-        if (!isConnected()) {
-          connect();
-        }
-        if (isConnected()) {
-          writer.write(msg);
-        }
+    public void write(String msg) throws IOException {
+      if (!isConnected()) {
+        connect();
       }
-
-      public void flush() throws IOException {
-        if (isConnected()) {
-          writer.flush();
-        }
+      if (isConnected()) {
+        writer.write(msg);
       }
+    }
 
-      public boolean isConnected() {
-        return socket != null && socket.isConnected() && !socket.isClosed();
+    public void flush() throws IOException {
+      if (isConnected()) {
+        writer.flush();
       }
+    }
 
-      public void close() throws IOException {
-        try {
-          if (writer != null) {
-            writer.close();
-          }
-        } catch (IOException ex) {
-          if (socket != null) {
-            socket.close();
-          }
-        } finally {
-          socket = null;
-          writer = null;
-        }
-      }
+    public boolean isConnected() {
+      return socket != null && socket.isConnected() && !socket.isClosed();
+    }
 
-      private boolean tooManyConnectionFailures() {
-        return connectionFailures > MAX_CONNECTION_FAILURES;
+    public void close() throws IOException {
+      try {
+        if (writer != null) {
+          writer.close();
+        }
+      } catch (IOException ex) {
+        if (socket != null) {
+          socket.close();
+        }
+      } finally {
+        socket = null;
+        writer = null;
       }
+    }
 
+    private boolean tooManyConnectionFailures() {
+      return connectionFailures > MAX_CONNECTION_FAILURES;
     }
+  }
+
+  public Graphite getGraphite() {
+    return graphite;
+  }
 
+  public void setGraphite(Graphite graphite) {
+    this.graphite = graphite;
+  }

Review Comment:
   The methods are too public. Could you move the package of 
`TestGraphiteMetrics` to org.apache.hadoop.metrics2.sink? That way we can make 
the methods package-private.



##########
hadoop-common-project/hadoop-nfs/src/test/java/org/apache/hadoop/portmap/TestPortmap.java:
##########
@@ -76,7 +75,7 @@ public void testIdle() throws InterruptedException, 
IOException {
   }
 
   @Test(timeout = 10000)
-  public void testRegistration() throws IOException, InterruptedException {
+  public void testRegistration() throws IOException, InterruptedException, 
IllegalAccessException {

Review Comment:
   I don't think this change is needed.



##########
hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/http/TestHttpServer.java:
##########
@@ -663,8 +662,7 @@ private HttpServer2 checkBindAddress(String host, int port, 
boolean findPort)
     HttpServer2 server = createServer(host, port);
     try {
       // not bound, ephemeral should return requested port (0 for ephemeral)
-      List<?> listeners = (List<?>) Whitebox.getInternalState(server,
-          "listeners");
+      List<?> listeners = server.getListeners();
       ServerConnector listener = (ServerConnector)listeners.get(0);

Review Comment:
   We can use `List<ServerConnector>` instead of `List<?>` and then we can 
remove the casting `(ServerConnector)`.





> Remove WhiteBox in hadoop-common module.
> ----------------------------------------
>
>                 Key: HADOOP-18302
>                 URL: https://issues.apache.org/jira/browse/HADOOP-18302
>             Project: Hadoop Common
>          Issue Type: Sub-task
>    Affects Versions: 3.4.0, 3.3.9
>            Reporter: fanshilun
>            Assignee: fanshilun
>            Priority: Minor
>              Labels: pull-request-available
>          Time Spent: 4h 10m
>  Remaining Estimate: 0h
>
> WhiteBox is deprecated, try to remove this method in hadoop-common.



--
This message was sent by Atlassian Jira
(v8.20.10#820010)

---------------------------------------------------------------------
To unsubscribe, e-mail: common-issues-unsubscr...@hadoop.apache.org
For additional commands, e-mail: common-issues-h...@hadoop.apache.org

Reply via email to