gfe-relnote: Add QuicUdpSocketApi, a de-platformized version of QuicSocketUtils, and use it in some non-production code. No behavior change, not protected.

This is the first step to kill quic_socket_utils.(h|cc).

Chromium merge notes:
1) The platform impl function, GetGooglePacketHeadersFromControlMessageImpl, can simply return false.
2) Please add quic/core/quic_udp_socket* to "epoll_quic_tools" in BUILD.gn: http://shortn/_zpf4caOcsa

PiperOrigin-RevId: 289439832
Change-Id: I3bf2d1f21314bcfb2c3a72f74fab25a22d49d49d
diff --git a/quic/core/quic_default_packet_writer.cc b/quic/core/quic_default_packet_writer.cc
index d092958..4222005 100644
--- a/quic/core/quic_default_packet_writer.cc
+++ b/quic/core/quic_default_packet_writer.cc
@@ -4,7 +4,7 @@
 
 #include "net/third_party/quiche/src/quic/core/quic_default_packet_writer.h"
 
-#include "net/quic/platform/impl/quic_socket_utils.h"
+#include "net/third_party/quiche/src/quic/core/quic_udp_socket.h"
 
 namespace quic {
 
@@ -22,8 +22,11 @@
   DCHECK(!write_blocked_);
   DCHECK(nullptr == options)
       << "QuicDefaultPacketWriter does not accept any options.";
-  WriteResult result = QuicSocketUtils::WritePacket(fd_, buffer, buf_len,
-                                                    self_address, peer_address);
+  QuicUdpPacketInfo packet_info;
+  packet_info.SetPeerAddress(peer_address);
+  packet_info.SetSelfIp(self_address);
+  WriteResult result =
+      QuicUdpSocketApi().WritePacket(fd_, buffer, buf_len, packet_info);
   if (IsWriteBlockedStatus(result.status)) {
     write_blocked_ = true;
   }
diff --git a/quic/core/quic_udp_socket.h b/quic/core/quic_udp_socket.h
new file mode 100644
index 0000000..86972d1
--- /dev/null
+++ b/quic/core/quic_udp_socket.h
@@ -0,0 +1,225 @@
+// Copyright 2019 The Chromium Authors. All rights reserved.
+// Use of this source code is governed by a BSD-style license that can be
+// found in the LICENSE file.
+
+#ifndef QUICHE_QUIC_CORE_QUIC_UDP_SOCKET_H_
+#define QUICHE_QUIC_CORE_QUIC_UDP_SOCKET_H_
+
+#include <cstddef>
+#include <cstdint>
+
+#include <type_traits>
+
+#include "net/third_party/quiche/src/quic/core/quic_types.h"
+#include "net/third_party/quiche/src/quic/core/quic_utils.h"
+#include "net/third_party/quiche/src/quic/platform/api/quic_ip_address.h"
+#include "net/third_party/quiche/src/quic/platform/api/quic_socket_address.h"
+
+namespace quic {
+
+#if defined(_WIN32)
+using QuicUdpSocketFd = SOCKET;
+const QuicUdpSocketFd kQuicInvalidSocketFd = INVALID_SOCKET;
+#else
+using QuicUdpSocketFd = int;
+const QuicUdpSocketFd kQuicInvalidSocketFd = -1;
+#endif
+
+enum class QuicUdpPacketInfoBit : uint8_t {
+  DROPPED_PACKETS = 0,   // Read
+  V4_SELF_IP,            // Read
+  V6_SELF_IP,            // Read
+  PEER_ADDRESS,          // Read & Write
+  RECV_TIMESTAMP,        // Read
+  TTL,                   // Read & Write
+  GOOGLE_PACKET_HEADER,  // Read
+  NUM_BITS,
+};
+static_assert(static_cast<size_t>(QuicUdpPacketInfoBit::NUM_BITS) <=
+                  BitMask64::NumBits(),
+              "BitMask64 not wide enough to hold all bits.");
+
+// BufferSpan points to an unowned buffer, copying this structure only copies
+// the pointer and length, not the buffer itself.
+struct QUIC_EXPORT_PRIVATE BufferSpan {
+  BufferSpan(char* buffer, size_t buffer_len)
+      : buffer(buffer), buffer_len(buffer_len) {}
+
+  BufferSpan() = default;
+  BufferSpan(const BufferSpan& other) = default;
+  BufferSpan& operator=(const BufferSpan& other) = default;
+
+  char* buffer = nullptr;
+  size_t buffer_len = 0;
+};
+
+// QuicUdpPacketInfo contains per-packet information used for sending and
+// receiving.
+class QUIC_EXPORT_PRIVATE QuicUdpPacketInfo {
+ public:
+  BitMask64 bitmask() const { return bitmask_; }
+
+  bool HasValue(QuicUdpPacketInfoBit bit) const { return bitmask_.IsSet(bit); }
+
+  QuicPacketCount dropped_packets() const {
+    DCHECK(HasValue(QuicUdpPacketInfoBit::DROPPED_PACKETS));
+    return dropped_packets_;
+  }
+
+  void SetDroppedPackets(QuicPacketCount dropped_packets) {
+    dropped_packets_ = dropped_packets;
+    bitmask_.Set(QuicUdpPacketInfoBit::DROPPED_PACKETS);
+  }
+
+  const QuicIpAddress& self_v4_ip() const {
+    DCHECK(HasValue(QuicUdpPacketInfoBit::V4_SELF_IP));
+    return self_v4_ip_;
+  }
+
+  void SetSelfV4Ip(QuicIpAddress self_v4_ip) {
+    self_v4_ip_ = self_v4_ip;
+    bitmask_.Set(QuicUdpPacketInfoBit::V4_SELF_IP);
+  }
+
+  const QuicIpAddress& self_v6_ip() const {
+    DCHECK(HasValue(QuicUdpPacketInfoBit::V6_SELF_IP));
+    return self_v6_ip_;
+  }
+
+  void SetSelfV6Ip(QuicIpAddress self_v6_ip) {
+    self_v6_ip_ = self_v6_ip;
+    bitmask_.Set(QuicUdpPacketInfoBit::V6_SELF_IP);
+  }
+
+  void SetSelfIp(QuicIpAddress self_ip) {
+    if (self_ip.IsIPv4()) {
+      SetSelfV4Ip(self_ip);
+    } else {
+      SetSelfV6Ip(self_ip);
+    }
+  }
+
+  const QuicSocketAddress& peer_address() const {
+    DCHECK(HasValue(QuicUdpPacketInfoBit::PEER_ADDRESS));
+    return peer_address_;
+  }
+
+  void SetPeerAddress(QuicSocketAddress peer_address) {
+    peer_address_ = peer_address;
+    bitmask_.Set(QuicUdpPacketInfoBit::PEER_ADDRESS);
+  }
+
+  QuicWallTime receive_timestamp() const {
+    DCHECK(HasValue(QuicUdpPacketInfoBit::RECV_TIMESTAMP));
+    return receive_timestamp_;
+  }
+
+  void SetReceiveTimestamp(QuicWallTime receive_timestamp) {
+    receive_timestamp_ = receive_timestamp;
+    bitmask_.Set(QuicUdpPacketInfoBit::RECV_TIMESTAMP);
+  }
+
+  int ttl() const {
+    DCHECK(HasValue(QuicUdpPacketInfoBit::TTL));
+    return ttl_;
+  }
+
+  void SetTtl(int ttl) {
+    ttl_ = ttl;
+    bitmask_.Set(QuicUdpPacketInfoBit::TTL);
+  }
+
+  void SetGooglePacketHeaders(BufferSpan google_packet_headers) {
+    google_packet_headers_ = google_packet_headers;
+    bitmask_.Set(QuicUdpPacketInfoBit::GOOGLE_PACKET_HEADER);
+  }
+
+ private:
+  BitMask64 bitmask_;
+  QuicPacketCount dropped_packets_;
+  QuicIpAddress self_v4_ip_;
+  QuicIpAddress self_v6_ip_;
+  QuicSocketAddress peer_address_;
+  QuicWallTime receive_timestamp_ = QuicWallTime::Zero();
+  int ttl_;
+  BufferSpan google_packet_headers_;
+};
+
+// QuicUdpSocketApi provides a minimal set of apis for sending and receiving
+// udp packets. The low level udp socket apis differ between kernels and kernel
+// versions, the goal of QuicUdpSocketApi is to hide such differences.
+// We use non-static functions because it is easier to be mocked in tests when
+// needed.
+class QUIC_EXPORT_PRIVATE QuicUdpSocketApi {
+ public:
+  // Creates a non-blocking udp socket, sets the receive/send buffer and enable
+  // receiving of self ip addresses on read.
+  // Return kQuicInvalidSocketFd if failed.
+  QuicUdpSocketFd Create(int address_family,
+                         int receive_buffer_size,
+                         int send_buffer_size);
+
+  // Closes |fd|. No-op if |fd| equals to kQuicInvalidSocketFd.
+  void Destroy(QuicUdpSocketFd fd);
+
+  // Bind |fd| to |address|. If |address|'s port number is 0, kernel will choose
+  // a random port to bind to. Caller can use QuicSocketAddress::FromSocket(fd)
+  // to get the bound random port.
+  bool Bind(QuicUdpSocketFd fd, QuicSocketAddress address);
+
+  // Enable receiving of various per-packet information. Return true if the
+  // corresponding information can be received on read.
+  bool EnableDroppedPacketCount(QuicUdpSocketFd fd);
+  bool EnableReceiveTimestamp(QuicUdpSocketFd fd);
+  bool EnableReceiveTtlForV4(QuicUdpSocketFd fd);
+  bool EnableReceiveTtlForV6(QuicUdpSocketFd fd);
+
+  // Wait for |fd| to become readable, up to |timeout|.
+  // Return true if |fd| is readable upon return.
+  bool WaitUntilReadable(QuicUdpSocketFd fd, QuicTime::Delta timeout);
+
+  struct QUIC_EXPORT_PRIVATE ReadPacketResult {
+    bool ok = false;
+    QuicUdpPacketInfo packet_info;
+    BufferSpan packet_buffer;
+    BufferSpan control_buffer;
+  };
+  // Read a packet from |fd|:
+  // packet_info_interested: Bitmask indicating what information caller wants to
+  //                         receive into |result->packet_info|.
+  // result->packet_info:    Received per packet information.
+  // result->packet_buffer:  The packet buffer, to be filled with packet data.
+  // result->control_buffer: The control buffer, used by ReadPacket internally.
+  //                         It is recommended to be at least 512 bytes.
+  // result->ok:             True iff a packet is successfully received.
+  void ReadPacket(QuicUdpSocketFd fd,
+                  BitMask64 packet_info_interested,
+                  ReadPacketResult* result);
+
+  using ReadPacketResults = std::vector<ReadPacketResult>;
+  // Read up to |results->size()| packets from |fd|. The meaning of each element
+  // in |*results| has been documented on top of |ReadPacket|.
+  size_t ReadMultiplePackets(QuicUdpSocketFd fd,
+                             BitMask64 packet_info_interested,
+                             ReadPacketResults* results);
+
+  // Write a packet to |fd|.
+  // packet_buffer, packet_buffer_len:  The packet buffer to write.
+  // packet_info:                       The per packet information to set.
+  WriteResult WritePacket(QuicUdpSocketFd fd,
+                          const char* packet_buffer,
+                          size_t packet_buffer_len,
+                          const QuicUdpPacketInfo& packet_info);
+
+ protected:
+  bool SetupSocket(QuicUdpSocketFd fd,
+                   int address_family,
+                   int receive_buffer_size,
+                   int send_buffer_size);
+  bool EnableReceiveSelfIpAddressForV4(QuicUdpSocketFd fd);
+  bool EnableReceiveSelfIpAddressForV6(QuicUdpSocketFd fd);
+};
+
+}  // namespace quic
+
+#endif  // QUICHE_QUIC_CORE_QUIC_UDP_SOCKET_H_
diff --git a/quic/core/quic_udp_socket_posix.cc b/quic/core/quic_udp_socket_posix.cc
new file mode 100644
index 0000000..f62dbe8
--- /dev/null
+++ b/quic/core/quic_udp_socket_posix.cc
@@ -0,0 +1,608 @@
+// Copyright 2019 The Chromium Authors. All rights reserved.
+// Use of this source code is governed by a BSD-style license that can be
+// found in the LICENSE file.
+
+#include "net/third_party/quiche/src/quic/core/quic_udp_socket.h"
+#include "net/third_party/quiche/src/quic/platform/api/quic_bug_tracker.h"
+#include "net/third_party/quiche/src/quic/platform/api/quic_udp_socket_platform_api.h"
+
+#include <arpa/inet.h>
+#include <fcntl.h>
+#include <netinet/in.h>
+#include <sys/select.h>
+#include <sys/socket.h>
+#include <sys/types.h>
+
+#if defined(__APPLE__) && !defined(__APPLE_USE_RFC_3542)
+#error "__APPLE_USE_RFC_3542 needs to be defined."
+#endif
+
+#if defined(__linux__)
+#include <alloca.h>
+// For SO_TIMESTAMPING.
+#include <linux/net_tstamp.h>
+#endif
+
+#if defined(__linux__) && !defined(__ANDROID__)
+#define QUIC_UDP_SOCKET_SUPPORT_TTL 1
+#endif
+
+namespace quic {
+namespace {
+QuicUdpSocketFd CreateNonblockingSocket(int address_family) {
+#if defined(__linux__) && defined(SOCK_NONBLOCK)
+
+  // Create a nonblocking socket directly.
+  int fd = socket(address_family, SOCK_DGRAM | SOCK_NONBLOCK, IPPROTO_UDP);
+  if (fd < 0) {
+    QUIC_LOG_FIRST_N(ERROR, 100)
+        << "socket() failed with address_family=" << address_family << ": "
+        << strerror(errno);
+    return kQuicInvalidSocketFd;
+  }
+
+  return fd;
+
+#else
+
+  // Create a socket and use fcntl to set it to nonblocking.
+  // This implementation is used when building for iOS, OSX and old versions of
+  // Linux (< 2.6.27) and old versions of Android (< API 21).
+  int fd = socket(address_family, SOCK_DGRAM, IPPROTO_UDP);
+  if (fd < 0) {
+    QUIC_LOG_FIRST_N(ERROR, 100)
+        << "socket() failed with address_family=" << address_family << ": "
+        << strerror(errno);
+    return kQuicInvalidSocketFd;
+  }
+  int current_flags = fcntl(fd, F_GETFL, 0);
+  if (current_flags == -1) {
+    QUIC_LOG_FIRST_N(ERROR, 100)
+        << "failed to get current socket flags: " << strerror(errno);
+    close(fd);
+    return kQuicInvalidSocketFd;
+  }
+
+  int rc = fcntl(fd, F_SETFL, current_flags | O_NONBLOCK);
+  if (rc == -1) {
+    QUIC_LOG_FIRST_N(ERROR, 100)
+        << "failed to set socket to non-blocking: " << strerror(errno);
+    close(fd);
+    return kQuicInvalidSocketFd;
+  }
+
+  return fd;
+
+#endif
+}  // End CreateNonblockingSocket
+
+void SetV4SelfIpInControlMessage(const QuicIpAddress& self_address,
+                                 cmsghdr* cmsg) {
+  DCHECK(self_address.IsIPv4());
+  in_pktinfo* pktinfo = reinterpret_cast<in_pktinfo*>(CMSG_DATA(cmsg));
+  memset(pktinfo, 0, sizeof(in_pktinfo));
+  pktinfo->ipi_ifindex = 0;
+  std::string address_string = self_address.ToPackedString();
+  memcpy(&pktinfo->ipi_spec_dst, address_string.c_str(),
+         address_string.length());
+}
+
+void SetV6SelfIpInControlMessage(const QuicIpAddress& self_address,
+                                 cmsghdr* cmsg) {
+  DCHECK(self_address.IsIPv6());
+  in6_pktinfo* pktinfo = reinterpret_cast<in6_pktinfo*>(CMSG_DATA(cmsg));
+  memset(pktinfo, 0, sizeof(in6_pktinfo));
+  std::string address_string = self_address.ToPackedString();
+  memcpy(&pktinfo->ipi6_addr, address_string.c_str(), address_string.length());
+}
+
+void PopulatePacketInfoFromControlMessage(struct cmsghdr* cmsg,
+                                          QuicUdpPacketInfo* packet_info,
+                                          BitMask64 packet_info_interested) {
+#if defined(__linux__) && defined(SO_RXQ_OVFL)
+  if (cmsg->cmsg_level == SOL_SOCKET && cmsg->cmsg_type == SO_RXQ_OVFL) {
+    if (packet_info_interested.IsSet(QuicUdpPacketInfoBit::DROPPED_PACKETS)) {
+      packet_info->SetDroppedPackets(
+          *(reinterpret_cast<uint32_t*> CMSG_DATA(cmsg)));
+    }
+    return;
+  }
+#endif
+
+#if defined(__linux__) && (!defined(__ANDROID_API__) || __ANDROID_API__ >= 21)
+  if (cmsg->cmsg_level == SOL_SOCKET && cmsg->cmsg_type == SO_TIMESTAMPING) {
+    // This is the structure that SO_TIMESTAMPING fills into the cmsg header.
+    // It is well-defined, but does not have a definition in a public header.
+    // See https://www.kernel.org/doc/Documentation/networking/timestamping.txt
+    // for more information.
+    struct LinuxTimestamping {
+      // The converted system time of the timestamp.
+      struct timespec systime;
+      // Deprecated; serves only as padding.
+      struct timespec hwtimetrans;
+      // The raw hardware timestamp.
+      struct timespec hwtimeraw;
+    };
+
+    if (packet_info_interested.IsSet(QuicUdpPacketInfoBit::RECV_TIMESTAMP)) {
+      LinuxTimestamping* linux_ts =
+          reinterpret_cast<LinuxTimestamping*>(CMSG_DATA(cmsg));
+      timespec* ts = &linux_ts->systime;
+      int64_t usec = (static_cast<int64_t>(ts->tv_sec) * 1000 * 1000) +
+                     (static_cast<int64_t>(ts->tv_nsec) / 1000);
+      packet_info->SetReceiveTimestamp(
+          QuicWallTime::FromUNIXMicroseconds(usec));
+    }
+    return;
+  }
+#endif
+
+  if (cmsg->cmsg_level == IPPROTO_IPV6 && cmsg->cmsg_type == IPV6_PKTINFO) {
+    if (packet_info_interested.IsSet(QuicUdpPacketInfoBit::V6_SELF_IP)) {
+      const in6_pktinfo* info = reinterpret_cast<in6_pktinfo*>(CMSG_DATA(cmsg));
+      const char* addr_data = reinterpret_cast<const char*>(&info->ipi6_addr);
+      int addr_len = sizeof(in6_addr);
+      QuicIpAddress self_v6_ip;
+      if (self_v6_ip.FromPackedString(addr_data, addr_len)) {
+        packet_info->SetSelfV6Ip(self_v6_ip);
+      } else {
+        QUIC_BUG << "QuicIpAddress::FromPackedString failed";
+      }
+    }
+    return;
+  }
+
+  if (cmsg->cmsg_level == IPPROTO_IP && cmsg->cmsg_type == IP_PKTINFO) {
+    if (packet_info_interested.IsSet(QuicUdpPacketInfoBit::V4_SELF_IP)) {
+      const in_pktinfo* info = reinterpret_cast<in_pktinfo*>(CMSG_DATA(cmsg));
+      const char* addr_data = reinterpret_cast<const char*>(&info->ipi_addr);
+      int addr_len = sizeof(in_addr);
+      QuicIpAddress self_v4_ip;
+      if (self_v4_ip.FromPackedString(addr_data, addr_len)) {
+        packet_info->SetSelfV4Ip(self_v4_ip);
+      } else {
+        QUIC_BUG << "QuicIpAddress::FromPackedString failed";
+      }
+    }
+    return;
+  }
+
+  if ((cmsg->cmsg_level == IPPROTO_IP && cmsg->cmsg_type == IP_TTL) ||
+      (cmsg->cmsg_level == IPPROTO_IPV6 && cmsg->cmsg_type == IPV6_HOPLIMIT)) {
+    if (packet_info_interested.IsSet(QuicUdpPacketInfoBit::TTL)) {
+      packet_info->SetTtl(*(reinterpret_cast<int*>(CMSG_DATA(cmsg))));
+    }
+    return;
+  }
+
+  if (packet_info_interested.IsSet(
+          QuicUdpPacketInfoBit::GOOGLE_PACKET_HEADER)) {
+    BufferSpan google_packet_headers;
+    if (GetGooglePacketHeadersFromControlMessage(
+            cmsg, &google_packet_headers.buffer,
+            &google_packet_headers.buffer_len)) {
+      packet_info->SetGooglePacketHeaders(google_packet_headers);
+    }
+  }
+}
+
+bool NextCmsg(msghdr* hdr,
+              char* control_buffer,
+              size_t control_buffer_len,
+              int cmsg_level,
+              int cmsg_type,
+              size_t data_size,
+              cmsghdr** cmsg /*in, out*/) {
+  // msg_controllen needs to be increased first, otherwise CMSG_NXTHDR will
+  // return nullptr.
+  hdr->msg_controllen += CMSG_SPACE(data_size);
+  if (hdr->msg_controllen > control_buffer_len) {
+    return false;
+  }
+
+  if ((*cmsg) == nullptr) {
+    DCHECK_EQ(nullptr, hdr->msg_control);
+    memset(control_buffer, 0, control_buffer_len);
+    hdr->msg_control = control_buffer;
+    (*cmsg) = CMSG_FIRSTHDR(hdr);
+  } else {
+    DCHECK_NE(nullptr, hdr->msg_control);
+    (*cmsg) = CMSG_NXTHDR(hdr, (*cmsg));
+  }
+
+  if (nullptr == (*cmsg)) {
+    return false;
+  }
+
+  (*cmsg)->cmsg_len = CMSG_LEN(data_size);
+  (*cmsg)->cmsg_level = cmsg_level;
+  (*cmsg)->cmsg_type = cmsg_type;
+
+  return true;
+}
+}  // namespace
+
+QuicUdpSocketFd QuicUdpSocketApi::Create(int address_family,
+                                         int receive_buffer_size,
+                                         int send_buffer_size) {
+  QuicUdpSocketFd fd = CreateNonblockingSocket(address_family);
+
+  if (fd == kQuicInvalidSocketFd) {
+    return kQuicInvalidSocketFd;
+  }
+
+  if (!SetupSocket(fd, address_family, receive_buffer_size, send_buffer_size)) {
+    Destroy(fd);
+    return kQuicInvalidSocketFd;
+  }
+
+  return fd;
+}
+
+bool QuicUdpSocketApi::SetupSocket(QuicUdpSocketFd fd,
+                                   int address_family,
+                                   int receive_buffer_size,
+                                   int send_buffer_size) {
+  // Receive buffer size.
+  if (setsockopt(fd, SOL_SOCKET, SO_RCVBUF, &receive_buffer_size,
+                 sizeof(receive_buffer_size)) != 0) {
+    QUIC_LOG_FIRST_N(ERROR, 100) << "Failed to set socket recv size";
+    return false;
+  }
+
+  // Send buffer size.
+  if (setsockopt(fd, SOL_SOCKET, SO_SNDBUF, &send_buffer_size,
+                 sizeof(send_buffer_size)) != 0) {
+    QUIC_LOG_FIRST_N(ERROR, 100) << "Failed to set socket send size";
+    return false;
+  }
+
+  if (!EnableReceiveSelfIpAddressForV4(fd)) {
+    QUIC_LOG_FIRST_N(ERROR, 100) << "Failed to enable receiving of self v4 ip";
+    return false;
+  }
+
+  if (address_family == AF_INET6 && !EnableReceiveSelfIpAddressForV6(fd)) {
+    QUIC_LOG_FIRST_N(ERROR, 100) << "Failed to enable receiving of self v6 ip";
+    return false;
+  }
+
+  return true;
+}
+
+void QuicUdpSocketApi::Destroy(QuicUdpSocketFd fd) {
+  if (fd != kQuicInvalidSocketFd) {
+    close(fd);
+  }
+}
+
+bool QuicUdpSocketApi::Bind(QuicUdpSocketFd fd, QuicSocketAddress address) {
+  sockaddr_storage addr = address.generic_address();
+  int addr_len =
+      address.host().IsIPv4() ? sizeof(sockaddr_in) : sizeof(sockaddr_in6);
+  return 0 == bind(fd, reinterpret_cast<sockaddr*>(&addr), addr_len);
+}
+
+bool QuicUdpSocketApi::EnableDroppedPacketCount(QuicUdpSocketFd fd) {
+#if defined(__linux__) && defined(SO_RXQ_OVFL)
+  int get_overflow = 1;
+  return 0 == setsockopt(fd, SOL_SOCKET, SO_RXQ_OVFL, &get_overflow,
+                         sizeof(get_overflow));
+#else
+  (void)fd;
+  return false;
+#endif
+}
+
+bool QuicUdpSocketApi::EnableReceiveSelfIpAddressForV4(QuicUdpSocketFd fd) {
+  int get_self_ip = 1;
+  return 0 == setsockopt(fd, IPPROTO_IP, IP_PKTINFO, &get_self_ip,
+                         sizeof(get_self_ip));
+}
+
+bool QuicUdpSocketApi::EnableReceiveSelfIpAddressForV6(QuicUdpSocketFd fd) {
+  int get_self_ip = 1;
+  return 0 == setsockopt(fd, IPPROTO_IPV6, IPV6_RECVPKTINFO, &get_self_ip,
+                         sizeof(get_self_ip));
+}
+
+bool QuicUdpSocketApi::EnableReceiveTimestamp(QuicUdpSocketFd fd) {
+#if defined(__linux__) && (!defined(__ANDROID_API__) || __ANDROID_API__ >= 21)
+  int timestamping = SOF_TIMESTAMPING_RX_SOFTWARE | SOF_TIMESTAMPING_SOFTWARE;
+  return 0 == setsockopt(fd, SOL_SOCKET, SO_TIMESTAMPING, &timestamping,
+                         sizeof(timestamping));
+#else
+  (void)fd;
+  return false;
+#endif
+}
+
+bool QuicUdpSocketApi::EnableReceiveTtlForV4(QuicUdpSocketFd fd) {
+#if defined(QUIC_UDP_SOCKET_SUPPORT_TTL)
+  int get_ttl = 1;
+  return 0 == setsockopt(fd, IPPROTO_IP, IP_RECVTTL, &get_ttl, sizeof(get_ttl));
+#else
+  (void)fd;
+  return false;
+#endif
+}
+
+bool QuicUdpSocketApi::EnableReceiveTtlForV6(QuicUdpSocketFd fd) {
+#if defined(QUIC_UDP_SOCKET_SUPPORT_TTL)
+  int get_ttl = 1;
+  return 0 == setsockopt(fd, IPPROTO_IPV6, IPV6_RECVHOPLIMIT, &get_ttl,
+                         sizeof(get_ttl));
+#else
+  (void)fd;
+  return false;
+#endif
+}
+
+bool QuicUdpSocketApi::WaitUntilReadable(QuicUdpSocketFd fd,
+                                         QuicTime::Delta timeout) {
+  fd_set read_fds;
+  FD_ZERO(&read_fds);
+  FD_SET(fd, &read_fds);
+
+  timeval select_timeout;
+  select_timeout.tv_sec = timeout.ToSeconds();
+  select_timeout.tv_usec = timeout.ToMicroseconds() % 1000000;
+
+  return 1 == select(1 + fd, &read_fds, nullptr, nullptr, &select_timeout);
+}
+
+void QuicUdpSocketApi::ReadPacket(QuicUdpSocketFd fd,
+                                  BitMask64 packet_info_interested,
+                                  ReadPacketResult* result) {
+  result->ok = false;
+  BufferSpan& packet_buffer = result->packet_buffer;
+  BufferSpan& control_buffer = result->control_buffer;
+  QuicUdpPacketInfo* packet_info = &result->packet_info;
+
+  struct iovec iov = {packet_buffer.buffer, packet_buffer.buffer_len};
+  struct sockaddr_storage raw_peer_address;
+
+  if (control_buffer.buffer_len > 0) {
+    reinterpret_cast<struct cmsghdr*>(control_buffer.buffer)->cmsg_len =
+        control_buffer.buffer_len;
+  }
+
+  msghdr hdr;
+  hdr.msg_name = &raw_peer_address;
+  hdr.msg_namelen = sizeof(raw_peer_address);
+  hdr.msg_iov = &iov;
+  hdr.msg_iovlen = 1;
+  hdr.msg_flags = 0;
+  hdr.msg_control = control_buffer.buffer;
+  hdr.msg_controllen = control_buffer.buffer_len;
+
+#if defined(__linux__) && !defined(__ANDROID__)
+  // If MSG_TRUNC is set on Linux, recvmsg will return the real packet size even
+  // if |packet_buffer| is too small to receive it.
+  int flags = MSG_TRUNC;
+#else
+  int flags = 0;
+#endif
+
+  int bytes_read = recvmsg(fd, &hdr, flags);
+  if (bytes_read < 0) {
+    const int error_num = errno;
+    if (error_num != EAGAIN) {
+      QUIC_LOG_FIRST_N(ERROR, 100)
+          << "Error reading packet: " << strerror(error_num);
+    }
+    return;
+  }
+
+  if (QUIC_PREDICT_FALSE(hdr.msg_flags & MSG_CTRUNC)) {
+    QUIC_BUG << "Control buffer too small. size:" << control_buffer.buffer_len;
+    return;
+  }
+
+  if (QUIC_PREDICT_FALSE(hdr.msg_flags & MSG_TRUNC)) {
+    QUIC_LOG_FIRST_N(WARNING, 100)
+        << "Received truncated QUIC packet: buffer size:"
+        << packet_buffer.buffer_len << " packet size:" << bytes_read;
+    return;
+  }
+
+  packet_buffer.buffer_len = bytes_read;
+  if (packet_info_interested.IsSet(QuicUdpPacketInfoBit::PEER_ADDRESS)) {
+    packet_info->SetPeerAddress(QuicSocketAddress(raw_peer_address));
+  }
+
+  if (hdr.msg_controllen > 0) {
+    for (struct cmsghdr* cmsg = CMSG_FIRSTHDR(&hdr); cmsg != nullptr;
+         cmsg = CMSG_NXTHDR(&hdr, cmsg)) {
+      BitMask64 prior_bitmask = packet_info->bitmask();
+      PopulatePacketInfoFromControlMessage(cmsg, packet_info,
+                                           packet_info_interested);
+      if (packet_info->bitmask() == prior_bitmask) {
+        QUIC_DLOG(INFO) << "Ignored cmsg_level:" << cmsg->cmsg_level
+                        << ", cmsg_type:" << cmsg->cmsg_type;
+      }
+    }
+  }
+
+  result->ok = true;
+}
+
+size_t QuicUdpSocketApi::ReadMultiplePackets(QuicUdpSocketFd fd,
+                                             BitMask64 packet_info_interested,
+                                             ReadPacketResults* results) {
+#if defined(__linux__) && !defined(__ANDROID__)
+  // Use recvmmsg.
+  size_t hdrs_size = sizeof(mmsghdr) * results->size();
+  mmsghdr* hdrs = static_cast<mmsghdr*>(alloca(hdrs_size));
+  memset(hdrs, 0, hdrs_size);
+
+  struct TempPerPacketData {
+    iovec iov;
+    sockaddr_storage raw_peer_address;
+  };
+  TempPerPacketData* packet_data_array = static_cast<TempPerPacketData*>(
+      alloca(sizeof(TempPerPacketData) * results->size()));
+
+  for (size_t i = 0; i < results->size(); ++i) {
+    (*results)[i].ok = false;
+
+    msghdr* hdr = &hdrs[i].msg_hdr;
+    TempPerPacketData* packet_data = &packet_data_array[i];
+    packet_data->iov.iov_base = (*results)[i].packet_buffer.buffer;
+    packet_data->iov.iov_len = (*results)[i].packet_buffer.buffer_len;
+
+    hdr->msg_name = &packet_data->raw_peer_address;
+    hdr->msg_namelen = sizeof(sockaddr_storage);
+    hdr->msg_iov = &packet_data->iov;
+    hdr->msg_iovlen = 1;
+    hdr->msg_flags = 0;
+    hdr->msg_control = (*results)[i].control_buffer.buffer;
+    hdr->msg_controllen = (*results)[i].control_buffer.buffer_len;
+  }
+  // If MSG_TRUNC is set on Linux, recvmmsg will return the real packet size in
+  // |hdrs[i].msg_len| even if packet buffer is too small to receive it.
+  int packets_read = recvmmsg(fd, hdrs, results->size(), MSG_TRUNC, nullptr);
+  if (packets_read <= 0) {
+    const int error_num = errno;
+    if (error_num != EAGAIN) {
+      QUIC_LOG_FIRST_N(ERROR, 100)
+          << "Error reading packets: " << strerror(error_num);
+    }
+    return 0;
+  }
+
+  size_t num_good_packets = 0;
+  for (int i = 0; i < packets_read; ++i) {
+    if (hdrs[i].msg_len == 0) {
+      continue;
+    }
+
+    msghdr& hdr = hdrs[i].msg_hdr;
+    if (QUIC_PREDICT_FALSE(hdr.msg_flags & MSG_CTRUNC)) {
+      QUIC_BUG << "Control buffer too small. size:"
+               << (*results)[i].control_buffer.buffer_len
+               << ", need:" << hdr.msg_controllen;
+      continue;
+    }
+
+    if (QUIC_PREDICT_FALSE(hdr.msg_flags & MSG_TRUNC)) {
+      QUIC_LOG_FIRST_N(WARNING, 100)
+          << "Received truncated QUIC packet: buffer size:"
+          << (*results)[i].packet_buffer.buffer_len
+          << " packet size:" << hdrs[i].msg_len;
+      continue;
+    }
+
+    ++num_good_packets;
+    (*results)[i].ok = true;
+    (*results)[i].packet_buffer.buffer_len = hdrs[i].msg_len;
+
+    QuicUdpPacketInfo* packet_info = &(*results)[i].packet_info;
+    if (packet_info_interested.IsSet(QuicUdpPacketInfoBit::PEER_ADDRESS)) {
+      packet_info->SetPeerAddress(
+          QuicSocketAddress(packet_data_array[i].raw_peer_address));
+    }
+
+    if (hdr.msg_controllen > 0) {
+      for (struct cmsghdr* cmsg = CMSG_FIRSTHDR(&hdr); cmsg != nullptr;
+           cmsg = CMSG_NXTHDR(&hdr, cmsg)) {
+        PopulatePacketInfoFromControlMessage(cmsg, packet_info,
+                                             packet_info_interested);
+      }
+    }
+  }
+  return num_good_packets;
+#else
+  size_t num_packets = 0;
+  for (ReadPacketResult& result : *results) {
+    result.ok = false;
+  }
+  for (ReadPacketResult& result : *results) {
+    ReadPacket(fd, packet_info_interested, &result);
+    if (!result.ok) {
+      break;
+    }
+    ++num_packets;
+  }
+  return num_packets;
+#endif
+}
+
+WriteResult QuicUdpSocketApi::WritePacket(
+    QuicUdpSocketFd fd,
+    const char* packet_buffer,
+    size_t packet_buffer_len,
+    const QuicUdpPacketInfo& packet_info) {
+  if (!packet_info.HasValue(QuicUdpPacketInfoBit::PEER_ADDRESS)) {
+    return WriteResult(WRITE_STATUS_ERROR, EINVAL);
+  }
+
+  char control_buffer[512];
+  sockaddr_storage raw_peer_address =
+      packet_info.peer_address().generic_address();
+  iovec iov = {const_cast<char*>(packet_buffer), packet_buffer_len};
+
+  msghdr hdr;
+  hdr.msg_name = &raw_peer_address;
+  hdr.msg_namelen = packet_info.peer_address().host().IsIPv4()
+                        ? sizeof(sockaddr_in)
+                        : sizeof(sockaddr_in6);
+  hdr.msg_iov = &iov;
+  hdr.msg_iovlen = 1;
+  hdr.msg_flags = 0;
+  hdr.msg_control = nullptr;
+  hdr.msg_controllen = 0;
+
+  cmsghdr* cmsg = nullptr;
+
+  // Set self IP.
+  if (packet_info.HasValue(QuicUdpPacketInfoBit::V4_SELF_IP) &&
+      packet_info.self_v4_ip().IsInitialized()) {
+    if (!NextCmsg(&hdr, control_buffer, sizeof(control_buffer), IPPROTO_IP,
+                  IP_PKTINFO, sizeof(in_pktinfo), &cmsg)) {
+      QUIC_LOG_FIRST_N(ERROR, 100)
+          << "Not enough buffer to set self v4 ip address.";
+      return WriteResult(WRITE_STATUS_ERROR, EINVAL);
+    }
+    SetV4SelfIpInControlMessage(packet_info.self_v4_ip(), cmsg);
+  } else if (packet_info.HasValue(QuicUdpPacketInfoBit::V6_SELF_IP) &&
+             packet_info.self_v6_ip().IsInitialized()) {
+    if (!NextCmsg(&hdr, control_buffer, sizeof(control_buffer), IPPROTO_IPV6,
+                  IPV6_PKTINFO, sizeof(in6_pktinfo), &cmsg)) {
+      QUIC_LOG_FIRST_N(ERROR, 100)
+          << "Not enough buffer to set self v6 ip address.";
+      return WriteResult(WRITE_STATUS_ERROR, EINVAL);
+    }
+    SetV6SelfIpInControlMessage(packet_info.self_v6_ip(), cmsg);
+  }
+
+#if defined(QUIC_UDP_SOCKET_SUPPORT_TTL)
+  // Set ttl.
+  if (packet_info.HasValue(QuicUdpPacketInfoBit::TTL)) {
+    int cmsg_level =
+        packet_info.peer_address().host().IsIPv4() ? IPPROTO_IP : IPPROTO_IPV6;
+    int cmsg_type =
+        packet_info.peer_address().host().IsIPv4() ? IP_TTL : IPV6_HOPLIMIT;
+    if (!NextCmsg(&hdr, control_buffer, sizeof(control_buffer), cmsg_level,
+                  cmsg_type, sizeof(int), &cmsg)) {
+      QUIC_LOG_FIRST_N(ERROR, 100) << "Not enough buffer to set ttl.";
+      return WriteResult(WRITE_STATUS_ERROR, EINVAL);
+    }
+    *reinterpret_cast<int*>(CMSG_DATA(cmsg)) = packet_info.ttl();
+  }
+#endif
+
+  int rc;
+  do {
+    rc = sendmsg(fd, &hdr, 0);
+  } while (rc < 0 && errno == EINTR);
+  if (rc >= 0) {
+    return WriteResult(WRITE_STATUS_OK, rc);
+  }
+  return WriteResult((errno == EAGAIN || errno == EWOULDBLOCK)
+                         ? WRITE_STATUS_BLOCKED
+                         : WRITE_STATUS_ERROR,
+                     errno);
+}
+
+}  // namespace quic
diff --git a/quic/core/quic_udp_socket_test.cc b/quic/core/quic_udp_socket_test.cc
new file mode 100644
index 0000000..697e5e9
--- /dev/null
+++ b/quic/core/quic_udp_socket_test.cc
@@ -0,0 +1,254 @@
+// Copyright 2019 The Chromium Authors. All rights reserved.
+// Use of this source code is governed by a BSD-style license that can be
+// found in the LICENSE file.
+
+#include "net/third_party/quiche/src/quic/core/quic_udp_socket.h"
+
+#include "net/third_party/quiche/src/quic/core/quic_constants.h"
+#include "net/third_party/quiche/src/quic/platform/api/quic_test.h"
+
+namespace quic {
+namespace test {
+
+class QuicUdpSocketTest : public QuicTest {
+ protected:
+  void SetUp() override {
+    // Try creating AF_INET socket, if it fails because of unsupported address
+    // family then tests are being run under IPv6-only environment, initialize
+    // address family to use for running the test under as AF_INET6 otherwise
+    // initialize it as AF_INET.
+    address_family_ = AF_INET;
+    fd_client_ =
+        api_.Create(address_family_,
+                    /*receive_buffer_size =*/kDefaultSocketReceiveBuffer,
+                    /*send_buffer_size =*/kDefaultSocketReceiveBuffer);
+    if (fd_client_ == kQuicInvalidSocketFd) {
+      address_family_ = AF_INET6;
+      fd_client_ =
+          api_.Create(address_family_,
+                      /*receive_buffer_size =*/kDefaultSocketReceiveBuffer,
+                      /*send_buffer_size =*/kDefaultSocketReceiveBuffer);
+    }
+    ASSERT_NE(fd_client_, kQuicInvalidSocketFd);
+
+    fd_server_ =
+        api_.Create(address_family_,
+                    /*receive_buffer_size =*/kDefaultSocketReceiveBuffer,
+                    /*send_buffer_size =*/kDefaultSocketReceiveBuffer);
+    ASSERT_NE(fd_server_, kQuicInvalidSocketFd);
+
+    ASSERT_TRUE(
+        api_.Bind(fd_server_, QuicSocketAddress(Loopback(), /*port=*/0)));
+
+    ASSERT_EQ(0, server_address_.FromSocket(fd_server_));
+
+    QUIC_LOG(INFO) << "Testing under IP"
+                   << std::string((address_family_ == AF_INET) ? "v4" : "v6");
+  }
+
+  ~QuicUdpSocketTest() {
+    api_.Destroy(fd_client_);
+    api_.Destroy(fd_server_);
+  }
+
+  QuicIpAddress Loopback() const {
+    return (address_family_ == AF_INET) ? QuicIpAddress::Loopback4()
+                                        : QuicIpAddress::Loopback6();
+  }
+
+  // Client sends the first |packet_size| bytes in |client_packet_buffer_| to
+  // server.
+  WriteResult SendPacketFromClient(size_t packet_size) {
+    EXPECT_LE(packet_size, sizeof(client_packet_buffer_));
+    QuicUdpPacketInfo packet_info;
+    packet_info.SetPeerAddress(server_address_);
+    return api_.WritePacket(fd_client_, client_packet_buffer_, packet_size,
+                            packet_info);
+  }
+
+  WriteResult SendPacketFromClientWithTtl(size_t packet_size, int ttl) {
+    EXPECT_LE(packet_size, sizeof(client_packet_buffer_));
+    QuicUdpPacketInfo packet_info;
+    packet_info.SetPeerAddress(server_address_);
+    packet_info.SetTtl(ttl);
+    return api_.WritePacket(fd_client_, client_packet_buffer_, packet_size,
+                            packet_info);
+  }
+
+  // Server waits for an incoming packet and reads it into
+  // |server_packet_buffer_|.
+  QuicUdpSocketApi::ReadPacketResult ReadPacketFromServer(
+      BitMask64 packet_info_interested) {
+    EXPECT_TRUE(
+        api_.WaitUntilReadable(fd_server_, QuicTime::Delta::FromSeconds(5)));
+    memset(server_packet_buffer_, 0, sizeof(server_packet_buffer_));
+    QuicUdpSocketApi::ReadPacketResult result;
+    result.packet_buffer = {server_packet_buffer_,
+                            sizeof(server_packet_buffer_)};
+    result.control_buffer = {server_control_buffer_,
+                             sizeof(server_control_buffer_)};
+    api_.ReadPacket(fd_server_, packet_info_interested, &result);
+    return result;
+  }
+
+  int ComparePacketBuffers(size_t packet_size) {
+    return memcmp(client_packet_buffer_, server_packet_buffer_, packet_size);
+  }
+
+  QuicUdpSocketApi api_;
+  QuicUdpSocketFd fd_client_;
+  QuicUdpSocketFd fd_server_;
+  QuicSocketAddress server_address_;
+  int address_family_;
+  char client_packet_buffer_[kEthernetMTU] = {0};
+  char server_packet_buffer_[kDefaultMaxPacketSize] = {0};
+  char server_control_buffer_[512] = {0};
+};
+
+TEST_F(QuicUdpSocketTest, ReadPacketOnly) {
+  const size_t kPacketSize = 512;
+  memset(client_packet_buffer_, '-', kPacketSize);
+  ASSERT_EQ(WriteResult(WRITE_STATUS_OK, kPacketSize),
+            SendPacketFromClient(kPacketSize));
+
+  QuicUdpSocketApi::ReadPacketResult read_result =
+      ReadPacketFromServer(/*packet_info_interested=*/BitMask64());
+  ASSERT_TRUE(read_result.ok);
+  ASSERT_EQ(kPacketSize, read_result.packet_buffer.buffer_len);
+  ASSERT_EQ(0, ComparePacketBuffers(kPacketSize));
+}
+
+TEST_F(QuicUdpSocketTest, ReadTruncated) {
+  const size_t kPacketSize = kDefaultMaxPacketSize + 1;
+  memset(client_packet_buffer_, '*', kPacketSize);
+  ASSERT_EQ(WriteResult(WRITE_STATUS_OK, kPacketSize),
+            SendPacketFromClient(kPacketSize));
+
+  QuicUdpSocketApi::ReadPacketResult read_result =
+      ReadPacketFromServer(/*packet_info_interested=*/BitMask64());
+  ASSERT_FALSE(read_result.ok);
+}
+
+TEST_F(QuicUdpSocketTest, ReadDroppedPackets) {
+  const size_t kPacketSize = kDefaultMaxPacketSize;
+  memset(client_packet_buffer_, '-', kPacketSize);
+  ASSERT_EQ(WriteResult(WRITE_STATUS_OK, kPacketSize),
+            SendPacketFromClient(kPacketSize));
+  ASSERT_EQ(WriteResult(WRITE_STATUS_OK, kPacketSize),
+            SendPacketFromClient(kPacketSize));
+
+  // Read the first packet without enabling DROPPED_PACKETS.
+  QuicUdpSocketApi::ReadPacketResult read_result =
+      ReadPacketFromServer(BitMask64(QuicUdpPacketInfoBit::DROPPED_PACKETS));
+  ASSERT_TRUE(read_result.ok);
+  ASSERT_EQ(kPacketSize, read_result.packet_buffer.buffer_len);
+  ASSERT_EQ(0, ComparePacketBuffers(kPacketSize));
+  ASSERT_FALSE(
+      read_result.packet_info.HasValue(QuicUdpPacketInfoBit::DROPPED_PACKETS));
+
+  // Enable DROPPED_PACKETS and read the second packet.
+  if (!api_.EnableDroppedPacketCount(fd_server_)) {
+    QUIC_LOG(INFO) << "DROPPED_PACKETS is not supported";
+    return;
+  }
+  read_result =
+      ReadPacketFromServer(BitMask64(QuicUdpPacketInfoBit::DROPPED_PACKETS));
+  ASSERT_TRUE(read_result.ok);
+  ASSERT_EQ(kPacketSize, read_result.packet_buffer.buffer_len);
+  ASSERT_EQ(0, ComparePacketBuffers(kPacketSize));
+  if (read_result.packet_info.HasValue(QuicUdpPacketInfoBit::DROPPED_PACKETS)) {
+    EXPECT_EQ(0u, read_result.packet_info.dropped_packets());
+  }
+}
+
+TEST_F(QuicUdpSocketTest, ReadSelfIp) {
+  const QuicUdpPacketInfoBit self_ip_bit =
+      (address_family_ == AF_INET) ? QuicUdpPacketInfoBit::V4_SELF_IP
+                                   : QuicUdpPacketInfoBit::V6_SELF_IP;
+
+  const size_t kPacketSize = 512;
+  memset(client_packet_buffer_, '&', kPacketSize);
+  ASSERT_EQ(WriteResult(WRITE_STATUS_OK, kPacketSize),
+            SendPacketFromClient(kPacketSize));
+
+  QuicUdpSocketApi::ReadPacketResult read_result =
+      ReadPacketFromServer(BitMask64(self_ip_bit));
+  ASSERT_TRUE(read_result.ok);
+  ASSERT_EQ(kPacketSize, read_result.packet_buffer.buffer_len);
+  ASSERT_EQ(0, ComparePacketBuffers(kPacketSize));
+  ASSERT_TRUE(read_result.packet_info.HasValue(self_ip_bit));
+  EXPECT_EQ(Loopback(), (address_family_ == AF_INET)
+                            ? read_result.packet_info.self_v4_ip()
+                            : read_result.packet_info.self_v6_ip());
+}
+
+TEST_F(QuicUdpSocketTest, ReadReceiveTimestamp) {
+  const size_t kPacketSize = kDefaultMaxPacketSize;
+  memset(client_packet_buffer_, '-', kPacketSize);
+  ASSERT_EQ(WriteResult(WRITE_STATUS_OK, kPacketSize),
+            SendPacketFromClient(kPacketSize));
+  ASSERT_EQ(WriteResult(WRITE_STATUS_OK, kPacketSize),
+            SendPacketFromClient(kPacketSize));
+
+  // Read the first packet without enabling RECV_TIMESTAMP.
+  QuicUdpSocketApi::ReadPacketResult read_result =
+      ReadPacketFromServer(BitMask64(QuicUdpPacketInfoBit::RECV_TIMESTAMP));
+  ASSERT_TRUE(read_result.ok);
+  ASSERT_EQ(kPacketSize, read_result.packet_buffer.buffer_len);
+  ASSERT_EQ(0, ComparePacketBuffers(kPacketSize));
+  ASSERT_FALSE(
+      read_result.packet_info.HasValue(QuicUdpPacketInfoBit::RECV_TIMESTAMP));
+
+  // Enable RECV_TIMESTAMP and read the second packet.
+  if (!api_.EnableReceiveTimestamp(fd_server_)) {
+    QUIC_LOG(INFO) << "RECV_TIMESTAMP is not supported";
+    return;
+  }
+  read_result =
+      ReadPacketFromServer(BitMask64(QuicUdpPacketInfoBit::RECV_TIMESTAMP));
+  ASSERT_TRUE(read_result.ok);
+  ASSERT_EQ(kPacketSize, read_result.packet_buffer.buffer_len);
+  ASSERT_EQ(0, ComparePacketBuffers(kPacketSize));
+  ASSERT_TRUE(
+      read_result.packet_info.HasValue(QuicUdpPacketInfoBit::RECV_TIMESTAMP));
+  QuicWallTime recv_timestamp = read_result.packet_info.receive_timestamp();
+  // 1577836800 is the unix seconds for 2020-01-01
+  EXPECT_TRUE(
+      QuicWallTime::FromUNIXSeconds(1577836800).IsBefore(recv_timestamp));
+}
+
+TEST_F(QuicUdpSocketTest, Ttl) {
+  const size_t kPacketSize = 512;
+  memset(client_packet_buffer_, '$', kPacketSize);
+  ASSERT_EQ(WriteResult(WRITE_STATUS_OK, kPacketSize),
+            SendPacketFromClientWithTtl(kPacketSize, 13));
+  ASSERT_EQ(WriteResult(WRITE_STATUS_OK, kPacketSize),
+            SendPacketFromClientWithTtl(kPacketSize, 13));
+
+  // Read the first packet without enabling ttl reporting.
+  QuicUdpSocketApi::ReadPacketResult read_result =
+      ReadPacketFromServer(BitMask64(QuicUdpPacketInfoBit::TTL));
+  ASSERT_TRUE(read_result.ok);
+  ASSERT_EQ(kPacketSize, read_result.packet_buffer.buffer_len);
+  ASSERT_EQ(0, ComparePacketBuffers(kPacketSize));
+  ASSERT_FALSE(read_result.packet_info.HasValue(QuicUdpPacketInfoBit::TTL));
+
+  // Enable ttl reporting and read the second packet.
+  if (!((address_family_ == AF_INET)
+            ? api_.EnableReceiveTtlForV4(fd_server_)
+            : api_.EnableReceiveTtlForV6(fd_server_))) {
+    QUIC_LOG(INFO) << "TTL is not supported for address family "
+                   << address_family_;
+    return;
+  }
+
+  read_result = ReadPacketFromServer(BitMask64(QuicUdpPacketInfoBit::TTL));
+  ASSERT_TRUE(read_result.ok);
+  ASSERT_EQ(kPacketSize, read_result.packet_buffer.buffer_len);
+  ASSERT_EQ(0, ComparePacketBuffers(kPacketSize));
+  ASSERT_TRUE(read_result.packet_info.HasValue(QuicUdpPacketInfoBit::TTL));
+  EXPECT_EQ(13, read_result.packet_info.ttl());
+}
+
+}  // namespace test
+}  // namespace quic
diff --git a/quic/core/quic_utils.h b/quic/core/quic_utils.h
index ab8d6af..590d377 100644
--- a/quic/core/quic_utils.h
+++ b/quic/core/quic_utils.h
@@ -7,7 +7,9 @@
 
 #include <cstddef>
 #include <cstdint>
+#include <sstream>
 #include <string>
+#include <type_traits>
 
 #include "net/third_party/quiche/src/quic/core/crypto/quic_random.h"
 #include "net/third_party/quiche/src/quic/core/frames/quic_frame.h"
@@ -222,6 +224,63 @@
                                            Perspective perspective);
 };
 
+template <typename Mask>
+class QUIC_EXPORT_PRIVATE BitMask {
+ public:
+  // explicit to prevent (incorrect) usage like "BitMask bitmask = 0;".
+  template <typename... Bits>
+  explicit BitMask(Bits... bits) {
+    mask_ = MakeMask(bits...);
+  }
+
+  BitMask() = default;
+  BitMask(const BitMask& other) = default;
+  BitMask& operator=(const BitMask& other) = default;
+
+  template <typename... Bits>
+  void Set(Bits... bits) {
+    mask_ |= MakeMask(bits...);
+  }
+
+  template <typename Bit>
+  bool IsSet(Bit bit) const {
+    return (MakeMask(bit) & mask_) != 0;
+  }
+
+  static constexpr size_t NumBits() { return 8 * sizeof(Mask); }
+
+  friend bool operator==(const BitMask& lhs, const BitMask& rhs) {
+    return lhs.mask_ == rhs.mask_;
+  }
+
+  std::string DebugString() const {
+    std::ostringstream oss;
+    oss << "0x" << std::hex << mask_;
+    return oss.str();
+  }
+
+ private:
+  template <typename Bit>
+  static std::enable_if_t<std::is_enum<Bit>::value, Mask> MakeMask(Bit bit) {
+    using IntType = typename std::underlying_type<Bit>::type;
+    return Mask(1) << static_cast<IntType>(bit);
+  }
+
+  template <typename Bit>
+  static std::enable_if_t<!std::is_enum<Bit>::value, Mask> MakeMask(Bit bit) {
+    return Mask(1) << bit;
+  }
+
+  template <typename Bit, typename... Bits>
+  static Mask MakeMask(Bit first_bit, Bits... other_bits) {
+    return MakeMask(first_bit) | MakeMask(other_bits...);
+  }
+
+  Mask mask_ = 0;
+};
+
+using BitMask64 = BitMask<uint64_t>;
+
 }  // namespace quic
 
 #endif  // QUICHE_QUIC_CORE_QUIC_UTILS_H_
diff --git a/quic/core/quic_utils_test.cc b/quic/core/quic_utils_test.cc
index baaa3ae..69be371 100644
--- a/quic/core/quic_utils_test.cc
+++ b/quic/core/quic_utils_test.cc
@@ -272,6 +272,71 @@
   EXPECT_NE(token1a, token2);
 }
 
+enum class TestEnumClassBit : uint8_t {
+  BIT_ZERO = 0,
+  BIT_ONE,
+  BIT_TWO,
+};
+
+enum TestEnumBit {
+  TEST_BIT_0 = 0,
+  TEST_BIT_1,
+  TEST_BIT_2,
+};
+
+TEST(QuicBitMaskTest, EnumClass) {
+  BitMask64 mask(TestEnumClassBit::BIT_ZERO, TestEnumClassBit::BIT_TWO);
+  EXPECT_TRUE(mask.IsSet(TestEnumClassBit::BIT_ZERO));
+  EXPECT_FALSE(mask.IsSet(TestEnumClassBit::BIT_ONE));
+  EXPECT_TRUE(mask.IsSet(TestEnumClassBit::BIT_TWO));
+}
+
+TEST(QuicBitMaskTest, Enum) {
+  BitMask64 mask(TEST_BIT_1, TEST_BIT_2);
+  EXPECT_FALSE(mask.IsSet(TEST_BIT_0));
+  EXPECT_TRUE(mask.IsSet(TEST_BIT_1));
+  EXPECT_TRUE(mask.IsSet(TEST_BIT_2));
+}
+
+TEST(QuicBitMaskTest, Integer) {
+  BitMask64 mask(1, 3);
+  mask.Set(3);
+  mask.Set(5, 7, 9);
+  EXPECT_FALSE(mask.IsSet(0));
+  EXPECT_TRUE(mask.IsSet(1));
+  EXPECT_FALSE(mask.IsSet(2));
+  EXPECT_TRUE(mask.IsSet(3));
+  EXPECT_FALSE(mask.IsSet(4));
+  EXPECT_TRUE(mask.IsSet(5));
+  EXPECT_FALSE(mask.IsSet(6));
+  EXPECT_TRUE(mask.IsSet(7));
+  EXPECT_FALSE(mask.IsSet(8));
+  EXPECT_TRUE(mask.IsSet(9));
+}
+
+TEST(QuicBitMaskTest, NumBits) {
+  EXPECT_EQ(64u, BitMask64::NumBits());
+  EXPECT_EQ(32u, BitMask<uint32_t>::NumBits());
+}
+
+TEST(QuicBitMaskTest, Constructor) {
+  BitMask64 empty_mask;
+  for (size_t bit = 0; bit < empty_mask.NumBits(); ++bit) {
+    EXPECT_FALSE(empty_mask.IsSet(bit));
+  }
+
+  BitMask64 mask(1, 3);
+  BitMask64 mask2 = mask;
+  BitMask64 mask3(mask2);
+
+  for (size_t bit = 0; bit < mask.NumBits(); ++bit) {
+    EXPECT_EQ(mask.IsSet(bit), mask2.IsSet(bit));
+    EXPECT_EQ(mask.IsSet(bit), mask3.IsSet(bit));
+  }
+
+  EXPECT_TRUE(std::is_trivially_copyable<BitMask64>::value);
+}
+
 }  // namespace
 }  // namespace test
 }  // namespace quic
diff --git a/quic/platform/api/quic_udp_socket_platform_api.h b/quic/platform/api/quic_udp_socket_platform_api.h
new file mode 100644
index 0000000..78d35f8
--- /dev/null
+++ b/quic/platform/api/quic_udp_socket_platform_api.h
@@ -0,0 +1,22 @@
+// Copyright 2019 The Chromium Authors. All rights reserved.
+// Use of this source code is governed by a BSD-style license that can be
+// found in the LICENSE file.
+
+#ifndef QUICHE_QUIC_PLATFORM_API_QUIC_UDP_SOCKET_PLATFORM_API_H_
+#define QUICHE_QUIC_PLATFORM_API_QUIC_UDP_SOCKET_PLATFORM_API_H_
+
+#include "net/quic/platform/impl/quic_udp_socket_platform_impl.h"
+
+namespace quic {
+
+inline bool GetGooglePacketHeadersFromControlMessage(
+    struct ::cmsghdr* cmsg,
+    char** packet_headers,
+    size_t* packet_headers_len) {
+  return GetGooglePacketHeadersFromControlMessageImpl(cmsg, packet_headers,
+                                                      packet_headers_len);
+}
+
+}  // namespace quic
+
+#endif  // QUICHE_QUIC_PLATFORM_API_QUIC_UDP_SOCKET_PLATFORM_API_H_
diff --git a/quic/tools/quic_client.cc b/quic/tools/quic_client.cc
index 379cfac..50ba040 100644
--- a/quic/tools/quic_client.cc
+++ b/quic/tools/quic_client.cc
@@ -26,13 +26,8 @@
 #include "net/third_party/quiche/src/quic/platform/api/quic_logging.h"
 #include "net/third_party/quiche/src/quic/platform/api/quic_ptr_util.h"
 #include "net/third_party/quiche/src/quic/platform/api/quic_socket_address.h"
-#include "net/quic/platform/impl/quic_socket_utils.h"
 #include "net/third_party/quiche/src/quic/tools/quic_simple_client_session.h"
 
-#ifndef SO_RXQ_OVFL
-#define SO_RXQ_OVFL 40
-#endif
-
 namespace quic {
 
 namespace tools {
diff --git a/quic/tools/quic_client_epoll_network_helper.cc b/quic/tools/quic_client_epoll_network_helper.cc
index 2f6d47d..60d04c7 100644
--- a/quic/tools/quic_client_epoll_network_helper.cc
+++ b/quic/tools/quic_client_epoll_network_helper.cc
@@ -19,14 +19,10 @@
 #include "net/third_party/quiche/src/quic/core/quic_epoll_connection_helper.h"
 #include "net/third_party/quiche/src/quic/core/quic_packets.h"
 #include "net/third_party/quiche/src/quic/core/quic_server_id.h"
+#include "net/third_party/quiche/src/quic/core/quic_udp_socket.h"
 #include "net/third_party/quiche/src/quic/platform/api/quic_bug_tracker.h"
 #include "net/third_party/quiche/src/quic/platform/api/quic_logging.h"
 #include "net/third_party/quiche/src/quic/platform/api/quic_system_event_loop.h"
-#include "net/quic/platform/impl/quic_socket_utils.h"
-
-#ifndef SO_RXQ_OVFL
-#define SO_RXQ_OVFL 40
-#endif
 
 namespace quic {
 
@@ -200,9 +196,16 @@
 int QuicClientEpollNetworkHelper::CreateUDPSocket(
     QuicSocketAddress server_address,
     bool* overflow_supported) {
-  return QuicSocketUtils::CreateUDPSocket(
-      server_address,
-      /*receive_buffer_size =*/kDefaultSocketReceiveBuffer,
-      /*send_buffer_size =*/kDefaultSocketReceiveBuffer, overflow_supported);
+  QuicUdpSocketApi api;
+  int fd = api.Create(server_address.host().AddressFamilyToInt(),
+                      /*receive_buffer_size =*/kDefaultSocketReceiveBuffer,
+                      /*send_buffer_size =*/kDefaultSocketReceiveBuffer);
+  if (fd < 0) {
+    return fd;
+  }
+
+  *overflow_supported = api.EnableDroppedPacketCount(fd);
+  api.EnableReceiveTimestamp(fd);
+  return fd;
 }
 }  // namespace quic
diff --git a/quic/tools/quic_server.cc b/quic/tools/quic_server.cc
index 04cc950..ef31d86 100644
--- a/quic/tools/quic_server.cc
+++ b/quic/tools/quic_server.cc
@@ -28,15 +28,10 @@
 #include "net/third_party/quiche/src/quic/platform/api/quic_flags.h"
 #include "net/third_party/quiche/src/quic/platform/api/quic_logging.h"
 #include "net/quic/platform/impl/quic_epoll_clock.h"
-#include "net/quic/platform/impl/quic_socket_utils.h"
 #include "net/third_party/quiche/src/quic/tools/quic_simple_crypto_server_stream_helper.h"
 #include "net/third_party/quiche/src/quic/tools/quic_simple_dispatcher.h"
 #include "net/third_party/quiche/src/quic/tools/quic_simple_server_backend.h"
 
-#ifndef SO_RXQ_OVFL
-#define SO_RXQ_OVFL 40
-#endif
-
 namespace quic {
 
 namespace {
@@ -118,15 +113,18 @@
 QuicServer::~QuicServer() = default;
 
 bool QuicServer::CreateUDPSocketAndListen(const QuicSocketAddress& address) {
-  fd_ = QuicSocketUtils::CreateUDPSocket(
-      address,
-      /*receive_buffer_size =*/kDefaultSocketReceiveBuffer,
-      /*send_buffer_size =*/kDefaultSocketReceiveBuffer, &overflow_supported_);
-  if (fd_ < 0) {
+  QuicUdpSocketApi socket_api;
+  fd_ = socket_api.Create(address.host().AddressFamilyToInt(),
+                          /*receive_buffer_size =*/kDefaultSocketReceiveBuffer,
+                          /*send_buffer_size =*/kDefaultSocketReceiveBuffer);
+  if (fd_ == kQuicInvalidSocketFd) {
     QUIC_LOG(ERROR) << "CreateSocket() failed: " << strerror(errno);
     return false;
   }
 
+  overflow_supported_ = socket_api.EnableDroppedPacketCount(fd_);
+  socket_api.EnableReceiveTimestamp(fd_);
+
   sockaddr_storage addr = address.generic_address();
   int rc = bind(fd_, reinterpret_cast<sockaddr*>(&addr), sizeof(addr));
   if (rc < 0) {
diff --git a/quic/tools/quic_server.h b/quic/tools/quic_server.h
index d2d9782..1fb17f6 100644
--- a/quic/tools/quic_server.h
+++ b/quic/tools/quic_server.h
@@ -18,6 +18,7 @@
 #include "net/third_party/quiche/src/quic/core/quic_epoll_connection_helper.h"
 #include "net/third_party/quiche/src/quic/core/quic_framer.h"
 #include "net/third_party/quiche/src/quic/core/quic_packet_writer.h"
+#include "net/third_party/quiche/src/quic/core/quic_udp_socket.h"
 #include "net/third_party/quiche/src/quic/core/quic_version_manager.h"
 #include "net/third_party/quiche/src/quic/platform/api/quic_epoll.h"
 #include "net/third_party/quiche/src/quic/platform/api/quic_socket_address.h"
@@ -128,7 +129,7 @@
   int port_;
 
   // Listening connection.  Also used for outbound client communication.
-  int fd_;
+  QuicUdpSocketFd fd_;
 
   // If overflow_supported_ is true this will be the number of packets dropped
   // during the lifetime of the server.  This may overflow if enough packets