You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@mesos.apache.org by id...@apache.org on 2016/02/18 18:27:22 UTC

mesos git commit: Added per container SNMP statistics.

Repository: mesos
Updated Branches:
  refs/heads/master b45b9df71 -> 9a2c10d7d


Added per container SNMP statistics.

Review: https://reviews.apache.org/r/38117/


Project: http://git-wip-us.apache.org/repos/asf/mesos/repo
Commit: http://git-wip-us.apache.org/repos/asf/mesos/commit/9a2c10d7
Tree: http://git-wip-us.apache.org/repos/asf/mesos/tree/9a2c10d7
Diff: http://git-wip-us.apache.org/repos/asf/mesos/diff/9a2c10d7

Branch: refs/heads/master
Commit: 9a2c10d7d44e24bfbeeeef11aa7e7f3bf5b79a40
Parents: b45b9df
Author: Cong Wang <xi...@gmail.com>
Authored: Tue Feb 16 15:06:08 2016 -0800
Committer: Ian Downes <id...@twitter.com>
Committed: Thu Feb 18 09:25:24 2016 -0800

----------------------------------------------------------------------
 docs/configuration.md                           |  10 +
 include/mesos/mesos.proto                       |  96 ++++++
 .../mesos/isolators/network/port_mapping.cpp    | 305 +++++++++++++++++++
 .../mesos/isolators/network/port_mapping.hpp    |   1 +
 src/slave/flags.cpp                             |   7 +
 src/slave/flags.hpp                             |   1 +
 src/tests/containerizer/port_mapping_tests.cpp  |  25 +-
 7 files changed, 440 insertions(+), 5 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/mesos/blob/9a2c10d7/docs/configuration.md
----------------------------------------------------------------------
diff --git a/docs/configuration.md b/docs/configuration.md
index 3d82368..801472c 100644
--- a/docs/configuration.md
+++ b/docs/configuration.md
@@ -1634,6 +1634,16 @@ each container. This flag is used for the <code>network/port_mapping</code>
 isolator. (default: false)
   </td>
 </tr>
+<tr>
+  <td>
+    --[no-]network_enable_snmp_statistics
+  </td>
+  <td>
+Whether to collect SNMP statistics details (e.g., TCPRetransSegs) for
+each container. This flag is used for the 'network/port_mapping'
+isolator. (default: false)
+  </td>
+</tr>
 </table>
 
 

http://git-wip-us.apache.org/repos/asf/mesos/blob/9a2c10d7/include/mesos/mesos.proto
----------------------------------------------------------------------
diff --git a/include/mesos/mesos.proto b/include/mesos/mesos.proto
index e24d3e0..8047946 100644
--- a/include/mesos/mesos.proto
+++ b/include/mesos/mesos.proto
@@ -733,6 +733,99 @@ message TrafficControlStatistics {
 }
 
 
+message IpStatistics {
+  optional int64 Forwarding =  1;
+  optional int64 DefaultTTL =  2;
+  optional int64 InReceives =  3;
+  optional int64 InHdrErrors =  4;
+  optional int64 InAddrErrors =  5;
+  optional int64 ForwDatagrams =  6;
+  optional int64 InUnknownProtos = 7;
+  optional int64 InDiscards =  8;
+  optional int64 InDelivers  = 9;
+  optional int64 OutRequests = 10;
+  optional int64 OutDiscards = 11;
+  optional int64 OutNoRoutes = 12;
+  optional int64 ReasmTimeout = 13;
+  optional int64 ReasmReqds = 14;
+  optional int64 ReasmOKs = 15;
+  optional int64 ReasmFails = 16;
+  optional int64 FragOKs = 17;
+  optional int64 FragFails = 18;
+  optional int64 FragCreates = 19;
+}
+
+
+message IcmpStatistics {
+  optional int64 InMsgs = 1;
+  optional int64 InErrors =  2;
+  optional int64 InCsumErrors = 3;
+  optional int64 InDestUnreachs =  4;
+  optional int64 InTimeExcds = 5;
+  optional int64 InParmProbs =  6;
+  optional int64 InSrcQuenchs = 7;
+  optional int64 InRedirects = 8;
+  optional int64 InEchos = 9;
+  optional int64 InEchoReps = 10;
+  optional int64 InTimestamps = 11;
+  optional int64 InTimestampReps = 12;
+  optional int64 InAddrMasks = 13;
+  optional int64 InAddrMaskReps = 14;
+  optional int64 OutMsgs = 15;
+  optional int64 OutErrors = 16;
+  optional int64 OutDestUnreachs = 17;
+  optional int64 OutTimeExcds = 18;
+  optional int64 OutParmProbs = 19;
+  optional int64 OutSrcQuenchs = 20;
+  optional int64 OutRedirects = 21;
+  optional int64 OutEchos = 22;
+  optional int64 OutEchoReps = 23;
+  optional int64 OutTimestamps = 24;
+  optional int64 OutTimestampReps = 25;
+  optional int64 OutAddrMasks = 26;
+  optional int64 OutAddrMaskReps = 27;
+}
+
+
+message TcpStatistics {
+  optional int64 RtoAlgorithm = 1;
+  optional int64 RtoMin = 2;
+  optional int64 RtoMax = 3;
+  optional int64 MaxConn = 4;
+  optional int64 ActiveOpens = 5;
+  optional int64 PassiveOpens = 6;
+  optional int64 AttemptFails = 7;
+  optional int64 EstabResets = 8;
+  optional int64 CurrEstab = 9;
+  optional int64 InSegs = 10;
+  optional int64 OutSegs = 11;
+  optional int64 RetransSegs = 12;
+  optional int64 InErrs = 13;
+  optional int64 OutRsts = 14;
+  optional int64 InCsumErrors = 15;
+}
+
+
+message UdpStatistics {
+  optional int64 InDatagrams = 1;
+  optional int64 NoPorts = 2;
+  optional int64 InErrors = 3;
+  optional int64 OutDatagrams = 4;
+  optional int64 RcvbufErrors = 5;
+  optional int64 SndbufErrors = 6;
+  optional int64 InCsumErrors = 7;
+  optional int64 IgnoredMulti = 8;
+}
+
+
+message SNMPStatistics {
+  optional IpStatistics ip_stats = 1;
+  optional IcmpStatistics icmp_stats = 2;
+  optional TcpStatistics tcp_stats = 3;
+  optional UdpStatistics udp_stats = 4;
+}
+
+
 /**
  * A snapshot of resource usage statistics.
  */
@@ -834,6 +927,9 @@ message ResourceStatistics {
   // or dropped due to congestion or policy inside and outside the
   // container.
   repeated TrafficControlStatistics net_traffic_control_statistics = 35;
+
+  // Network SNMP statistics for each container.
+  optional SNMPStatistics net_snmp_statistics = 42;
 }
 
 

http://git-wip-us.apache.org/repos/asf/mesos/blob/9a2c10d7/src/slave/containerizer/mesos/isolators/network/port_mapping.cpp
----------------------------------------------------------------------
diff --git a/src/slave/containerizer/mesos/isolators/network/port_mapping.cpp b/src/slave/containerizer/mesos/isolators/network/port_mapping.cpp
index 1c2fbe9..134b6c7 100644
--- a/src/slave/containerizer/mesos/isolators/network/port_mapping.cpp
+++ b/src/slave/containerizer/mesos/isolators/network/port_mapping.cpp
@@ -676,6 +676,12 @@ PortMappingStatistics::Flags::Flags()
       "Whether to collect socket statistics details (e.g., TCP RTT)\n"
       "for this container.",
       false);
+
+  add(&enable_snmp_statistics,
+      "enable_snmp_statistics",
+      "Whether to collect SNMP statistics details (e.g., TCPRetransSegs)\n"
+      "for this container.",
+      false);
 }
 
 
@@ -721,6 +727,257 @@ static void addTrafficControlStatistics(
 }
 
 
+static void addIPStatistics(
+    const hashmap<string, int64_t>& statistics,
+    ResourceStatistics* result)
+{
+  SNMPStatistics *snmp = result->mutable_net_snmp_statistics();
+  IpStatistics *ip = snmp->mutable_ip_stats();
+
+  // TODO(cwang): Use protobuf reflection here.
+  if (statistics.contains("Forwarding")) {
+    ip->set_forwarding(statistics.at("Forwarding"));
+  }
+  if (statistics.contains("DefaultTTL")) {
+    ip->set_defaultttl(statistics.at("DefaultTTL"));
+  }
+  if (statistics.contains("InReceives")) {
+    ip->set_inreceives(statistics.at("InReceives"));
+  }
+  if (statistics.contains("InHdrErrors")) {
+    ip->set_inhdrerrors(statistics.at("InHdrErrors"));
+  }
+  if (statistics.contains("InAddrErrors")) {
+    ip->set_inaddrerrors(statistics.at("InAddrErrors"));
+  }
+  if (statistics.contains("ForwDatagrams")) {
+    ip->set_forwdatagrams(statistics.at("ForwDatagrams"));
+  }
+  if (statistics.contains("InUnknownProtos")) {
+    ip->set_inunknownprotos(statistics.at("InUnknownProtos"));
+  }
+  if (statistics.contains("InDiscards")) {
+    ip->set_indiscards(statistics.at("InDiscards"));
+  }
+  if (statistics.contains("InDelivers")) {
+    ip->set_indelivers(statistics.at("InDelivers"));
+  }
+  if (statistics.contains("OutRequests")) {
+    ip->set_outrequests(statistics.at("OutRequests"));
+  }
+  if (statistics.contains("OutDiscards")) {
+    ip->set_outdiscards(statistics.at("OutDiscards"));
+  }
+  if (statistics.contains("OutNoRoutes")) {
+    ip->set_outnoroutes(statistics.at("OutNoRoutes"));
+  }
+  if (statistics.contains("ReasmTimeout")) {
+    ip->set_reasmtimeout(statistics.at("ReasmTimeout"));
+  }
+  if (statistics.contains("ReasmReqds")) {
+    ip->set_reasmreqds(statistics.at("ReasmReqds"));
+  }
+  if (statistics.contains("ReasmOKs")) {
+    ip->set_reasmoks(statistics.at("ReasmOKs"));
+  }
+  if (statistics.contains("ReasmFails")) {
+    ip->set_reasmfails(statistics.at("ReasmFails"));
+  }
+  if (statistics.contains("FragOKs")) {
+    ip->set_fragoks(statistics.at("FragOKs"));
+  }
+  if (statistics.contains("FragFails")) {
+    ip->set_fragfails(statistics.at("FragFails"));
+  }
+  if (statistics.contains("FragCreates")) {
+    ip->set_fragcreates(statistics.at("FragCreates"));
+  }
+}
+
+
+static void addICMPStatistics(
+    const hashmap<string, int64_t>& statistics,
+    ResourceStatistics* result)
+{
+  SNMPStatistics *snmp = result->mutable_net_snmp_statistics();
+  IcmpStatistics *icmp = snmp->mutable_icmp_stats();
+
+  // TODO(cwang): Use protobuf reflection here.
+  if (statistics.contains("InMsgs")) {
+    icmp->set_inmsgs(statistics.at("InMsgs"));
+  }
+  if (statistics.contains("InErrors")) {
+    icmp->set_inerrors(statistics.at("InErrors"));
+  }
+  if (statistics.contains("InCsumErrors")) {
+    icmp->set_incsumerrors(statistics.at("InCsumErrors"));
+  }
+  if (statistics.contains("InDestUnreachs")) {
+    icmp->set_indestunreachs(statistics.at("InDestUnreachs"));
+  }
+  if (statistics.contains("InTimeExcds")) {
+    icmp->set_intimeexcds(statistics.at("InTimeExcds"));
+  }
+  if (statistics.contains("InParmProbs")) {
+    icmp->set_inparmprobs(statistics.at("InParmProbs"));
+  }
+  if (statistics.contains("InSrcQuenchs")) {
+    icmp->set_insrcquenchs(statistics.at("InSrcQuenchs"));
+  }
+  if (statistics.contains("InRedirects")) {
+    icmp->set_inredirects(statistics.at("InRedirects"));
+  }
+  if (statistics.contains("InEchos")) {
+    icmp->set_inechos(statistics.at("InEchos"));
+  }
+  if (statistics.contains("InEchoReps")) {
+    icmp->set_inechoreps(statistics.at("InEchoReps"));
+  }
+  if (statistics.contains("InTimestamps")) {
+    icmp->set_intimestamps(statistics.at("InTimestamps"));
+  }
+  if (statistics.contains("InTimestampReps")) {
+    icmp->set_intimestampreps(statistics.at("InTimestampReps"));
+  }
+  if (statistics.contains("InAddrMasks")) {
+    icmp->set_inaddrmasks(statistics.at("InAddrMasks"));
+  }
+  if (statistics.contains("InAddrMaskReps")) {
+    icmp->set_inaddrmaskreps(statistics.at("InAddrMaskReps"));
+  }
+  if (statistics.contains("OutMsgs")) {
+    icmp->set_outmsgs(statistics.at("OutMsgs"));
+  }
+  if (statistics.contains("OutErrors")) {
+    icmp->set_outerrors(statistics.at("OutErrors"));
+  }
+  if (statistics.contains("OutDestUnreachs")) {
+    icmp->set_outdestunreachs(statistics.at("OutDestUnreachs"));
+  }
+  if (statistics.contains("OutTimeExcds")) {
+    icmp->set_outtimeexcds(statistics.at("OutTimeExcds"));
+  }
+  if (statistics.contains("OutParmProbs")) {
+    icmp->set_outparmprobs(statistics.at("OutParmProbs"));
+  }
+  if (statistics.contains("OutSrcQuenchs")) {
+    icmp->set_outsrcquenchs(statistics.at("OutSrcQuenchs"));
+  }
+  if (statistics.contains("OutRedirects")) {
+    icmp->set_outredirects(statistics.at("OutRedirects"));
+  }
+  if (statistics.contains("OutEchos")) {
+    icmp->set_outechos(statistics.at("OutEchos"));
+  }
+  if (statistics.contains("OutEchoReps")) {
+    icmp->set_outechoreps(statistics.at("OutEchoReps"));
+  }
+  if (statistics.contains("OutTimestamps")) {
+    icmp->set_outtimestamps(statistics.at("OutTimestamps"));
+  }
+  if (statistics.contains("OutTimestampReps")) {
+    icmp->set_outtimestampreps(statistics.at("OutTimestampReps"));
+  }
+  if (statistics.contains("OutAddrMasks")) {
+    icmp->set_outaddrmasks(statistics.at("OutAddrMasks"));
+  }
+  if (statistics.contains("OutAddrMaskReps")) {
+    icmp->set_outaddrmaskreps(statistics.at("OutAddrMaskReps"));
+  }
+}
+
+
+static void addTCPStatistics(
+    const hashmap<string, int64_t>& statistics,
+    ResourceStatistics* result)
+{
+  SNMPStatistics *snmp = result->mutable_net_snmp_statistics();
+  TcpStatistics *tcp = snmp->mutable_tcp_stats();
+
+  // TODO(cwang): Use protobuf reflection here.
+  if (statistics.contains("RtoAlgorithm")) {
+    tcp->set_rtoalgorithm(statistics.at("RtoAlgorithm"));
+  }
+  if (statistics.contains("RtoMin")) {
+    tcp->set_rtomin(statistics.at("RtoMin"));
+  }
+  if (statistics.contains("RtoMax")) {
+    tcp->set_rtomax(statistics.at("RtoMax"));
+  }
+  if (statistics.contains("MaxConn")) {
+    tcp->set_maxconn(statistics.at("MaxConn"));
+  }
+  if (statistics.contains("ActiveOpens")) {
+    tcp->set_activeopens(statistics.at("ActiveOpens"));
+  }
+  if (statistics.contains("PassiveOpens")) {
+    tcp->set_passiveopens(statistics.at("PassiveOpens"));
+  }
+  if (statistics.contains("AttemptFails")) {
+    tcp->set_attemptfails(statistics.at("AttemptFails"));
+  }
+  if (statistics.contains("EstabResets")) {
+    tcp->set_estabresets(statistics.at("EstabResets"));
+  }
+  if (statistics.contains("CurrEstab")) {
+    tcp->set_currestab(statistics.at("CurrEstab"));
+  }
+  if (statistics.contains("InSegs")) {
+    tcp->set_insegs(statistics.at("InSegs"));
+  }
+  if (statistics.contains("OutSegs")) {
+    tcp->set_outsegs(statistics.at("OutSegs"));
+  }
+  if (statistics.contains("RetransSegs")) {
+    tcp->set_retranssegs(statistics.at("RetransSegs"));
+  }
+  if (statistics.contains("InErrs")) {
+    tcp->set_inerrs(statistics.at("InErrs"));
+  }
+  if (statistics.contains("OutRsts")) {
+    tcp->set_outrsts(statistics.at("OutRsts"));
+  }
+  if (statistics.contains("InCsumErrors")) {
+    tcp->set_incsumerrors(statistics.at("InCsumErrors"));
+  }
+}
+
+
+static void addUDPStatistics(
+    const hashmap<string, int64_t>& statistics,
+    ResourceStatistics* result)
+{
+  SNMPStatistics *snmp = result->mutable_net_snmp_statistics();
+  UdpStatistics *udp = snmp->mutable_udp_stats();
+
+  // TODO(cwang): Use protobuf reflection here.
+  if (statistics.contains("InDatagrams")) {
+    udp->set_indatagrams(statistics.at("InDatagrams"));
+  }
+  if (statistics.contains("NoPorts")) {
+    udp->set_noports(statistics.at("NoPorts"));
+  }
+  if (statistics.contains("InErrors")) {
+    udp->set_inerrors(statistics.at("InErrors"));
+  }
+  if (statistics.contains("OutDatagrams")) {
+    udp->set_outdatagrams(statistics.at("OutDatagrams"));
+  }
+  if (statistics.contains("RcvbufErrors")) {
+    udp->set_rcvbuferrors(statistics.at("RcvbufErrors"));
+  }
+  if (statistics.contains("SndbufErrors")) {
+    udp->set_sndbuferrors(statistics.at("SndbufErrors"));
+  }
+  if (statistics.contains("InCsumErrors")) {
+    udp->set_incsumerrors(statistics.at("InCsumErrors"));
+  }
+  if (statistics.contains("IgnoredMulti")) {
+    udp->set_ignoredmulti(statistics.at("IgnoredMulti"));
+  }
+}
+
+
 int PortMappingStatistics::execute()
 {
   if (flags.help) {
@@ -871,6 +1128,52 @@ int PortMappingStatistics::execute()
     }
   }
 
+  if (flags.enable_snmp_statistics) {
+    Try<string> value = os::read("/proc/net/snmp");
+    if (value.isError()) {
+      cerr << "Failed to read /proc/net/snmp: " << value.error() << endl;
+      return 1;
+    }
+
+    hashmap<string, hashmap<string, int64_t>> SNMPStats;
+    vector<string> keys;
+    bool isKeyLine = true;
+    foreach (const string& line, strings::tokenize(value.get(), "\n")) {
+      vector<string> fields = strings::tokenize(line, ":");
+      if (fields.size() != 2) {
+        cerr << "Failed to tokenize line '" << line << "' "
+             << " in /proc/net/snmp" << endl;
+        return 1;
+      }
+      vector<string> tokens = strings::tokenize(fields[1], " ");
+      if (isKeyLine) {
+        for (size_t i = 0; i < tokens.size(); i++) {
+          keys.push_back(tokens[i]);
+        }
+      } else {
+        hashmap<string, int64_t> stats;
+        for (size_t i = 0; i < tokens.size(); i++) {
+          Try<int64_t> val = numify<int64_t>(tokens[i]);
+
+          if (val.isError()) {
+            cerr << "Failed to parse the statistics in " <<  fields[0]
+                 << val.error() << endl;
+            return 1;
+          }
+          stats[keys[i]] = val.get();
+        }
+        SNMPStats[fields[0]] = stats;
+        keys.clear();
+      }
+      isKeyLine = !isKeyLine;
+    }
+
+    addIPStatistics(SNMPStats["Ip"], &result);
+    addICMPStatistics(SNMPStats["Icmp"], &result);
+    addTCPStatistics(SNMPStats["Tcp"], &result);
+    addUDPStatistics(SNMPStats["Udp"], &result);
+  }
+
   // Collect traffic statistics for the container from the container
   // virtual interface and export them in JSON.
   const string& eth0 = flags.eth0_name.get();
@@ -2842,6 +3145,8 @@ Future<ResourceStatistics> PortMappingIsolatorProcess::usage(
     flags.network_enable_socket_statistics_summary;
   statistics.flags.enable_socket_statistics_details =
     flags.network_enable_socket_statistics_details;
+  statistics.flags.enable_snmp_statistics =
+    flags.network_enable_snmp_statistics;
 
   vector<string> argv(2);
   argv[0] = "mesos-network-helper";

http://git-wip-us.apache.org/repos/asf/mesos/blob/9a2c10d7/src/slave/containerizer/mesos/isolators/network/port_mapping.hpp
----------------------------------------------------------------------
diff --git a/src/slave/containerizer/mesos/isolators/network/port_mapping.hpp b/src/slave/containerizer/mesos/isolators/network/port_mapping.hpp
index ebf820a..0fe2f48 100644
--- a/src/slave/containerizer/mesos/isolators/network/port_mapping.hpp
+++ b/src/slave/containerizer/mesos/isolators/network/port_mapping.hpp
@@ -384,6 +384,7 @@ public:
     Option<pid_t> pid;
     bool enable_socket_statistics_summary;
     bool enable_socket_statistics_details;
+    bool enable_snmp_statistics;
   };
 
   PortMappingStatistics() : Subcommand(NAME) {}

http://git-wip-us.apache.org/repos/asf/mesos/blob/9a2c10d7/src/slave/flags.cpp
----------------------------------------------------------------------
diff --git a/src/slave/flags.cpp b/src/slave/flags.cpp
index d4b4e52..855812e 100644
--- a/src/slave/flags.cpp
+++ b/src/slave/flags.cpp
@@ -587,6 +587,13 @@ mesos::internal::slave::Flags::Flags()
       "isolator.",
       false);
 
+  add(&Flags::network_enable_snmp_statistics,
+      "network_enable_snmp_statistics",
+      "Whether to collect SNMP statistics details (e.g., TCPRetransSegs) for\n"
+      "each container. This flag is used for the 'network/port_mapping'\n"
+      "isolator.",
+      false);
+
 #endif // WITH_NETWORK_ISOLATOR
 
   add(&Flags::container_disk_watch_interval,

http://git-wip-us.apache.org/repos/asf/mesos/blob/9a2c10d7/src/slave/flags.hpp
----------------------------------------------------------------------
diff --git a/src/slave/flags.hpp b/src/slave/flags.hpp
index bd52b4f..54c1a69 100644
--- a/src/slave/flags.hpp
+++ b/src/slave/flags.hpp
@@ -120,6 +120,7 @@ public:
   std::string egress_flow_classifier_parent;
   bool network_enable_socket_statistics_summary;
   bool network_enable_socket_statistics_details;
+  bool network_enable_snmp_statistics;
 #endif
   Duration container_disk_watch_interval;
   bool enforce_container_disk_quota;

http://git-wip-us.apache.org/repos/asf/mesos/blob/9a2c10d7/src/tests/containerizer/port_mapping_tests.cpp
----------------------------------------------------------------------
diff --git a/src/tests/containerizer/port_mapping_tests.cpp b/src/tests/containerizer/port_mapping_tests.cpp
index 1e6cb05..983a633 100644
--- a/src/tests/containerizer/port_mapping_tests.cpp
+++ b/src/tests/containerizer/port_mapping_tests.cpp
@@ -350,7 +350,8 @@ protected:
   Result<ResourceStatistics> statisticsHelper(
       pid_t pid,
       bool enable_summary,
-      bool enable_details)
+      bool enable_details,
+      bool enable_snmp)
   {
     // Retrieve the socket information from inside the container.
     PortMappingStatistics statistics;
@@ -358,6 +359,7 @@ protected:
     statistics.flags.eth0_name = eth0;
     statistics.flags.enable_socket_statistics_summary = enable_summary;
     statistics.flags.enable_socket_statistics_details = enable_details;
+    statistics.flags.enable_snmp_statistics = enable_snmp;
 
     vector<string> argv(2);
     argv[0] = "mesos-network-helper";
@@ -1624,6 +1626,14 @@ bool HasTCPSocketsRTT(const ResourceStatistics& statistics)
 }
 
 
+bool HasTCPRetransSegs(const ResourceStatistics& statistics)
+{
+  return statistics.has_net_snmp_statistics() &&
+         statistics.net_snmp_statistics().has_tcp_stats() &&
+         statistics.net_snmp_statistics().tcp_stats().has_retranssegs();
+}
+
+
 // Test that RTT can be returned properly from usage(). This test is
 // very similar to SmallEgressLimitTest in its setup.
 TEST_F(PortMappingIsolatorTest, ROOT_NC_PortMappingStatistics)
@@ -1637,6 +1647,7 @@ TEST_F(PortMappingIsolatorTest, ROOT_NC_PortMappingStatistics)
   flags.egress_rate_limit_per_container = rate;
   flags.network_enable_socket_statistics_summary = true;
   flags.network_enable_socket_statistics_details = true;
+  flags.network_enable_snmp_statistics = true;
 
   Try<Isolator*> isolator = PortMappingIsolatorProcess::create(flags);
   CHECK_SOME(isolator);
@@ -1747,25 +1758,29 @@ TEST_F(PortMappingIsolatorTest, ROOT_NC_PortMappingStatistics)
   // While the connection is still active, try out different flag
   // combinations.
   Result<ResourceStatistics> statistics =
-      statisticsHelper(pid.get(), true, true);
+      statisticsHelper(pid.get(), true, true, true);
   ASSERT_SOME(statistics);
   EXPECT_TRUE(HasTCPSocketsCount(statistics.get()));
   EXPECT_TRUE(HasTCPSocketsRTT(statistics.get()));
+  EXPECT_TRUE(HasTCPRetransSegs(statistics.get()));
 
-  statistics = statisticsHelper(pid.get(), true, false);
+  statistics = statisticsHelper(pid.get(), true, false, false);
   ASSERT_SOME(statistics);
   EXPECT_TRUE(HasTCPSocketsCount(statistics.get()));
   EXPECT_FALSE(HasTCPSocketsRTT(statistics.get()));
+  EXPECT_FALSE(HasTCPRetransSegs(statistics.get()));
 
-  statistics = statisticsHelper(pid.get(), false, true);
+  statistics = statisticsHelper(pid.get(), false, true, true);
   ASSERT_SOME(statistics);
   EXPECT_FALSE(HasTCPSocketsCount(statistics.get()));
   EXPECT_TRUE(HasTCPSocketsRTT(statistics.get()));
+  EXPECT_TRUE(HasTCPRetransSegs(statistics.get()));
 
-  statistics = statisticsHelper(pid.get(), false, false);
+  statistics = statisticsHelper(pid.get(), false, false, false);
   ASSERT_SOME(statistics);
   EXPECT_FALSE(HasTCPSocketsCount(statistics.get()));
   EXPECT_FALSE(HasTCPSocketsRTT(statistics.get()));
+  EXPECT_FALSE(HasTCPRetransSegs(statistics.get()));
 
   // Wait for the command to finish.
   ASSERT_TRUE(waitForFileCreation(container1Ready));