Add unused QuicSession interfaces to initiating IETF connection migration: MigratePath(), ValidatePath(). These new interfaces are used by toy client in e2e tests.

Add QuicPathValidator into QuicConnection to do IETF path validation.

Behavior changes:
Cancel path validation if there is any during connection close;

Don't handle socket write error if sending PATH_CHALLENGE on the alternative socket or if the package sent to the alternative peer address exceeds path MTU;

Propagate PATH_RESPONSE to path validator to complete path validation.

Added and modified end-to-end tests to demonstrate connection migration with path validation works on client side.

Protected by quic_reloadable_flag_quic_pass_path_response_to_validator.

PiperOrigin-RevId: 343304329
Change-Id: I53dc52e82fbb9c69f5d2e90e5c90e1be45bb7bb6
diff --git a/quic/core/http/end_to_end_test.cc b/quic/core/http/end_to_end_test.cc
index 6b188fc..df3b1f9 100644
--- a/quic/core/http/end_to_end_test.cc
+++ b/quic/core/http/end_to_end_test.cc
@@ -2422,6 +2422,33 @@
   SendSynchronousBarRequestAndCheckResponse();
 }
 
+TEST_P(EndToEndTest, AsynchronousConnectionMigrationClientIPChanged) {
+  ASSERT_TRUE(Initialize());
+  if (!version_.HasIetfQuicFrames() ||
+      !client_->client()->session()->connection()->use_path_validator()) {
+    return;
+  }
+  client_.reset(CreateQuicClient(nullptr));
+
+  SendSynchronousFooRequestAndCheckResponse();
+
+  // Store the client IP address which was used to send the first request.
+  QuicIpAddress old_host =
+      client_->client()->network_helper()->GetLatestClientAddress().host();
+
+  // Migrate socket to the new IP address.
+  QuicIpAddress new_host = TestLoopback(2);
+  EXPECT_NE(old_host, new_host);
+  ASSERT_TRUE(client_->client()->ValidateAndMigrateSocket(new_host));
+
+  while (client_->client()->HasPendingPathValidation()) {
+    client_->client()->WaitForEvents();
+  }
+  EXPECT_EQ(new_host, client_->client()->session()->self_address().host());
+  // Send a request using the new socket.
+  SendSynchronousBarRequestAndCheckResponse();
+}
+
 TEST_P(EndToEndTest, ConnectionMigrationClientPortChanged) {
   // Tests that the client's port can change during an established QUIC
   // connection, and that doing so does not result in the connection being
@@ -4397,8 +4424,6 @@
 TEST_P(EndToEndPacketReorderingTest, ReorderedConnectivityProbing) {
   ASSERT_TRUE(Initialize());
   if (version_.HasIetfQuicFrames()) {
-    // TODO(b/143909619): Reenable this test when supporting IETF connection
-    // migration.
     return;
   }
 
@@ -4452,6 +4477,148 @@
             client_connection->GetStats().num_connectivity_probing_received);
 }
 
+// A writer which holds the next packet to be sent till ReleasePacket() is
+// called.
+class PacketHoldingWriter : public QuicPacketWriterWrapper {
+ public:
+  WriteResult WritePacket(const char* buffer,
+                          size_t buf_len,
+                          const QuicIpAddress& self_address,
+                          const QuicSocketAddress& peer_address,
+                          PerPacketOptions* options) override {
+    if (!hold_next_packet_) {
+      return QuicPacketWriterWrapper::WritePacket(buffer, buf_len, self_address,
+                                                  peer_address, options);
+    }
+    QUIC_DLOG(INFO) << "Packet is held by the writer";
+    packet_content_ = std::string(buffer, buf_len);
+    self_address_ = self_address;
+    peer_address_ = peer_address;
+    options_ = (options == nullptr ? nullptr : options->Clone());
+    hold_next_packet_ = false;
+    return WriteResult(WRITE_STATUS_OK, buf_len);
+  }
+
+  void HoldNextPacket() {
+    DCHECK(packet_content_.empty()) << "There is already one packet on hold.";
+    hold_next_packet_ = true;
+  }
+
+  void ReleasePacket() {
+    QUIC_DLOG(INFO) << "Release packet";
+    ASSERT_EQ(WRITE_STATUS_OK,
+              QuicPacketWriterWrapper::WritePacket(
+                  packet_content_.data(), packet_content_.length(),
+                  self_address_, peer_address_, options_.release())
+                  .status);
+    packet_content_.clear();
+  }
+
+ private:
+  bool hold_next_packet_{false};
+  std::string packet_content_;
+  QuicIpAddress self_address_;
+  QuicSocketAddress peer_address_;
+  std::unique_ptr<PerPacketOptions> options_;
+};
+
+TEST_P(EndToEndPacketReorderingTest, ReorderedPathChallenge) {
+  ASSERT_TRUE(Initialize());
+  if (!version_.HasIetfQuicFrames() ||
+      !client_->client()->session()->connection()->use_path_validator()) {
+    return;
+  }
+  client_.reset(EndToEndTest::CreateQuicClient(nullptr));
+
+  // Finish one request to make sure handshake established.
+  EXPECT_EQ(kFooResponseBody, client_->SendSynchronousRequest("/foo"));
+
+  // Wait for the connection to become idle, to make sure the packet gets
+  // delayed is the connectivity probing packet.
+  client_->WaitForDelayedAcks();
+
+  QuicSocketAddress old_addr =
+      client_->client()->network_helper()->GetLatestClientAddress();
+
+  // Migrate socket to the new IP address.
+  QuicIpAddress new_host = TestLoopback(2);
+  EXPECT_NE(old_addr.host(), new_host);
+
+  // Setup writer wrapper to hold the probing packet.
+  auto holding_writer = new PacketHoldingWriter();
+  client_->UseWriter(holding_writer);
+  // Write a connectivity probing after the next /foo request.
+  holding_writer->HoldNextPacket();
+
+  // A packet with PATH_CHALLENGE will be held in the writer.
+  ASSERT_TRUE(client_->client()->ValidateAndMigrateSocket(new_host));
+
+  // Send (on-hold) PATH_CHALLENGE after this request.
+  client_->SendRequest("/foo");
+  holding_writer->ReleasePacket();
+
+  client_->WaitForResponse();
+
+  EXPECT_EQ(kFooResponseBody, client_->response_body());
+  // Send yet another request after the PATH_CHALLENGE, when this request
+  // returns, the probing is guaranteed to have been received by the server, and
+  // the server's response to probing is guaranteed to have been received by the
+  // client.
+  EXPECT_EQ(kBarResponseBody, client_->SendSynchronousRequest("/bar"));
+
+  server_thread_->Pause();
+  QuicConnection* server_connection = GetServerConnection();
+  if (server_connection != nullptr) {
+    EXPECT_EQ(1u,
+              server_connection->GetStats().num_connectivity_probing_received);
+  } else {
+    ADD_FAILURE() << "Missing server connection";
+  }
+  server_thread_->Resume();
+}
+
+TEST_P(EndToEndPacketReorderingTest, PathValidationFailure) {
+  ASSERT_TRUE(Initialize());
+  if (!version_.HasIetfQuicFrames() ||
+      !client_->client()->session()->connection()->use_path_validator()) {
+    return;
+  }
+
+  client_.reset(CreateQuicClient(nullptr));
+  // Finish one request to make sure handshake established.
+  EXPECT_EQ(kFooResponseBody, client_->SendSynchronousRequest("/foo"));
+
+  // Wait for the connection to become idle, to make sure the packet gets
+  // delayed is the connectivity probing packet.
+  client_->WaitForDelayedAcks();
+
+  QuicSocketAddress old_addr = client_->client()->session()->self_address();
+
+  // Migrate socket to the new IP address.
+  QuicIpAddress new_host = TestLoopback(2);
+  EXPECT_NE(old_addr.host(), new_host);
+
+  // Drop PATH_RESPONSE packets to timeout the path validation.
+  server_writer_->set_fake_packet_loss_percentage(100);
+  ASSERT_TRUE(client_->client()->ValidateAndMigrateSocket(new_host));
+  while (client_->client()->HasPendingPathValidation()) {
+    client_->client()->WaitForEvents();
+  }
+  server_thread_->Pause();
+  QuicConnection* server_connection = GetServerConnection();
+  if (server_connection != nullptr) {
+    EXPECT_EQ(3u,
+              server_connection->GetStats().num_connectivity_probing_received);
+  } else {
+    ADD_FAILURE() << "Missing server connection";
+  }
+  server_thread_->Resume();
+
+  EXPECT_EQ(old_addr, client_->client()->session()->self_address());
+  server_writer_->set_fake_packet_loss_percentage(0);
+  EXPECT_EQ(kBarResponseBody, client_->SendSynchronousRequest("/bar"));
+}
+
 TEST_P(EndToEndPacketReorderingTest, Buffer0RttRequest) {
   ASSERT_TRUE(Initialize());
   // Finish one request to make sure handshake established.
diff --git a/quic/core/quic_connection.cc b/quic/core/quic_connection.cc
index 6e6ded6..b6ea5b1 100644
--- a/quic/core/quic_connection.cc
+++ b/quic/core/quic_connection.cc
@@ -30,6 +30,7 @@
 #include "net/third_party/quiche/src/quic/core/quic_legacy_version_encapsulator.h"
 #include "net/third_party/quiche/src/quic/core/quic_packet_creator.h"
 #include "net/third_party/quiche/src/quic/core/quic_packet_writer.h"
+#include "net/third_party/quiche/src/quic/core/quic_path_validator.h"
 #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_bug_tracker.h"
@@ -355,7 +356,8 @@
           packet_creator_.let_connection_handle_pings()),
       use_encryption_level_context_(
           encrypted_control_frames_ &&
-          GetQuicReloadableFlag(quic_use_encryption_level_context)) {
+          GetQuicReloadableFlag(quic_use_encryption_level_context)),
+      path_validator_(alarm_factory_, &arena_, this, random_generator_) {
   QUIC_BUG_IF(!start_peer_migration_earlier_ && send_path_response_);
   if (GetQuicReloadableFlag(quic_connection_set_initial_self_address)) {
     DCHECK(perspective_ == Perspective::IS_CLIENT ||
@@ -1108,7 +1110,7 @@
   }
 }
 
-void QuicConnection::OnSuccessfulMigrationAfterProbing() {
+void QuicConnection::OnSuccessfulMigration() {
   DCHECK_EQ(perspective_, Perspective::IS_CLIENT);
   if (IsPathDegrading()) {
     // If path was previously degrading, and migration is successful after
@@ -1597,13 +1599,18 @@
     debug_visitor_->OnPathResponseFrame(frame);
   }
   MaybeUpdateAckTimeout();
-  if (!transmitted_connectivity_probe_payload_ ||
-      *transmitted_connectivity_probe_payload_ != frame.data_buffer) {
-    // Is not for the probe we sent, ignore it.
-    return true;
+  if (use_path_validator_) {
+    path_validator_.OnPathResponse(frame.data_buffer,
+                                   last_packet_destination_address_);
+  } else {
+    if (!transmitted_connectivity_probe_payload_ ||
+        *transmitted_connectivity_probe_payload_ != frame.data_buffer) {
+      // Is not for the probe we sent, ignore it.
+      return true;
+    }
+    // Have received the matching PATH RESPONSE, saved payload no longer valid.
+    transmitted_connectivity_probe_payload_ = nullptr;
   }
-  // Have received the matching PATH RESPONSE, saved payload no longer valid.
-  transmitted_connectivity_probe_payload_ = nullptr;
   return true;
 }
 
@@ -2833,7 +2840,7 @@
   QUIC_DVLOG(1) << ENDPOINT << "Sending packet " << packet_number << " : "
                 << (IsRetransmittable(*packet) == HAS_RETRANSMITTABLE_DATA
                         ? "data bearing "
-                        : " ack only ")
+                        : " ack or probing only ")
                 << ", encryption level: " << packet->encryption_level
                 << ", encrypted length:" << encrypted_length
                 << ", fate: " << fate;
@@ -2850,6 +2857,8 @@
   WriteResult result(WRITE_STATUS_OK, encrypted_length);
   QuicSocketAddress send_to_address =
       (send_path_response_) ? packet->peer_address : peer_address();
+  // Self address is always the default self address on this code path.
+  bool send_on_current_path = send_to_address == peer_address();
   switch (fate) {
     case DISCARD:
       ++stats_.packets_discarded;
@@ -3001,17 +3010,23 @@
 
   // In some cases, an MTU probe can cause EMSGSIZE. This indicates that the
   // MTU discovery is permanently unsuccessful.
-  if (IsMsgTooBig(result) && is_mtu_discovery) {
-    // When MSG_TOO_BIG is returned, the system typically knows what the
-    // actual MTU is, so there is no need to probe further.
-    // TODO(wub): Reduce max packet size to a safe default, or the actual MTU.
-    QUIC_DVLOG(1) << ENDPOINT
-                  << " MTU probe packet too big, size:" << encrypted_length
-                  << ", long_term_mtu_:" << long_term_mtu_;
-    mtu_discoverer_.Disable();
-    mtu_discovery_alarm_->Cancel();
-    // The write failed, but the writer is not blocked, so return true.
-    return true;
+  if (IsMsgTooBig(result)) {
+    if (is_mtu_discovery) {
+      // When MSG_TOO_BIG is returned, the system typically knows what the
+      // actual MTU is, so there is no need to probe further.
+      // TODO(wub): Reduce max packet size to a safe default, or the actual MTU.
+      QUIC_DVLOG(1) << ENDPOINT
+                    << " MTU probe packet too big, size:" << encrypted_length
+                    << ", long_term_mtu_:" << long_term_mtu_;
+      mtu_discoverer_.Disable();
+      mtu_discovery_alarm_->Cancel();
+      // The write failed, but the writer is not blocked, so return true.
+      return true;
+    }
+    if (use_path_validator_ && !send_on_current_path) {
+      // Only handle MSG_TOO_BIG as error on current path.
+      return true;
+    }
   }
 
   if (IsWriteError(result.status)) {
@@ -3072,14 +3087,13 @@
   }
 
   // Do not measure rtt of this packet if it's not sent on current path.
-  const bool measure_rtt = send_to_address == peer_address();
-  QUIC_DLOG_IF(INFO, !measure_rtt)
+  QUIC_DLOG_IF(INFO, !send_on_current_path)
       << ENDPOINT << " Sent packet " << packet->packet_number
       << " on a different path with remote address " << send_to_address
       << " while current path has peer address " << peer_address();
   const bool in_flight = sent_packet_manager_.OnPacketSent(
       packet, packet_send_time, packet->transmission_type,
-      IsRetransmittable(*packet), measure_rtt);
+      IsRetransmittable(*packet), /*measure_rtt=*/send_on_current_path);
   QUIC_BUG_IF(default_enable_5rto_blackhole_detection_ &&
               blackhole_detector_.IsDetectionInProgress() &&
               !sent_packet_manager_.HasInFlightPackets())
@@ -4034,6 +4048,7 @@
   // Cancel the alarms so they don't trigger any action now that the
   // connection is closed.
   CancelAllAlarms();
+  path_validator_.CancelPathValidation();
 }
 
 void QuicConnection::CancelAllAlarms() {
@@ -5439,17 +5454,21 @@
   return sent_packet_manager_.GetRetransmissionTime();
 }
 
-void QuicConnection::SendPathChallenge(const QuicPathFrameBuffer& data_buffer,
+bool QuicConnection::SendPathChallenge(const QuicPathFrameBuffer& data_buffer,
                                        const QuicSocketAddress& self_address,
                                        const QuicSocketAddress& peer_address,
                                        QuicPacketWriter* writer) {
   if (writer == writer_) {
-    // It's on current path, add the PATH_CHALLENGE the same way as other
-    // frames.
-    QuicPacketCreator::ScopedPeerAddressContext context(&packet_creator_,
-                                                        peer_address);
-    packet_creator_.AddPathChallengeFrame(data_buffer);
-    return;
+    {
+      // It's on current path, add the PATH_CHALLENGE the same way as other
+      // frames.
+      QuicPacketCreator::ScopedPeerAddressContext context(&packet_creator_,
+                                                          peer_address);
+      // This may cause connection to be closed.
+      packet_creator_.AddPathChallengeFrame(data_buffer);
+    }
+    // Return outside of the scope so that the flush result can be reflected.
+    return connected_;
   }
   std::unique_ptr<SerializedPacket> probing_packet =
       packet_creator_.SerializePathChallengeConnectivityProbingPacket(
@@ -5457,6 +5476,24 @@
   DCHECK_EQ(IsRetransmittable(*probing_packet), NO_RETRANSMITTABLE_DATA);
   WritePacketUsingWriter(std::move(probing_packet), writer, self_address,
                          peer_address, /*measure_rtt=*/false);
+  return true;
+}
+
+QuicTime QuicConnection::GetRetryTimeout(
+    const QuicSocketAddress& peer_address_to_use,
+    QuicPacketWriter* writer_to_use) const {
+  if (writer_to_use == writer_ && peer_address_to_use == peer_address()) {
+    return clock_->ApproximateNow() + sent_packet_manager_.GetPtoDelay();
+  }
+  return clock_->ApproximateNow() +
+         QuicTime::Delta::FromMilliseconds(3 * kInitialRttMs);
+}
+
+void QuicConnection::ValidatePath(
+    std::unique_ptr<QuicPathValidationContext> context,
+    std::unique_ptr<QuicPathValidator::ResultDelegate> result_delegate) {
+  path_validator_.StartValidingPath(std::move(context),
+                                    std::move(result_delegate));
 }
 
 bool QuicConnection::SendPathResponse(const QuicPathFrameBuffer& data_buffer,
@@ -5481,5 +5518,22 @@
   SendControlFrame(QuicFrame(QuicPingFrame()));
 }
 
+bool QuicConnection::HasPendingPathValidation() const {
+  return path_validator_.HasPendingPathValidation();
+}
+
+void QuicConnection::MigratePath(const QuicSocketAddress& self_address,
+                                 const QuicSocketAddress& peer_address,
+                                 QuicPacketWriter* writer,
+                                 bool owns_writer) {
+  if (!connected_) {
+    return;
+  }
+  SetSelfAddress(self_address);
+  UpdatePeerAddress(peer_address);
+  SetQuicPacketWriter(writer, owns_writer);
+  OnSuccessfulMigration();
+}
+
 #undef ENDPOINT  // undef for jumbo builds
 }  // namespace quic
diff --git a/quic/core/quic_connection.h b/quic/core/quic_connection.h
index b19796a..599099d 100644
--- a/quic/core/quic_connection.h
+++ b/quic/core/quic_connection.h
@@ -47,6 +47,7 @@
 #include "net/third_party/quiche/src/quic/core/quic_packet_creator.h"
 #include "net/third_party/quiche/src/quic/core/quic_packet_writer.h"
 #include "net/third_party/quiche/src/quic/core/quic_packets.h"
+#include "net/third_party/quiche/src/quic/core/quic_path_validator.h"
 #include "net/third_party/quiche/src/quic/core/quic_sent_packet_manager.h"
 #include "net/third_party/quiche/src/quic/core/quic_time.h"
 #include "net/third_party/quiche/src/quic/core/quic_types.h"
@@ -432,7 +433,8 @@
       public QuicPacketCreator::DelegateInterface,
       public QuicSentPacketManager::NetworkChangeVisitor,
       public QuicNetworkBlackholeDetector::Delegate,
-      public QuicIdleNetworkDetector::Delegate {
+      public QuicIdleNetworkDetector::Delegate,
+      public QuicPathValidator::SendDelegate {
  public:
   // Constructs a new QuicConnection for |connection_id| and
   // |initial_peer_address| using |writer| to write packets. |owns_writer|
@@ -1078,7 +1080,7 @@
   void OnSuccessfulVersionNegotiation();
 
   // Called when self migration succeeds after probing.
-  void OnSuccessfulMigrationAfterProbing();
+  void OnSuccessfulMigration();
 
   // Called for QUIC+TLS versions when we send transport parameters.
   void OnTransportParametersSent(
@@ -1104,21 +1106,37 @@
 
   bool send_path_response() const { return send_path_response_; }
 
+  bool use_path_validator() const { return use_path_validator_; }
+
   // If now is close to idle timeout, returns true and sends a connectivity
   // probing packet to test the connection for liveness. Otherwise, returns
   // false.
   bool MaybeTestLiveness();
 
+  // QuicPathValidator::SendDelegate
   // Send PATH_CHALLENGE using the given path information. If |writer| is the
   // default writer, PATH_CHALLENGE can be bundled with other frames, and the
   // containing packet can be buffered if the writer is blocked. Otherwise,
   // PATH_CHALLENGE will be written in an individual packet and it will be
   // dropped if write fails. |data_buffer| will be populated with the payload
   // for future validation.
-  void SendPathChallenge(const QuicPathFrameBuffer& data_buffer,
+  // Return false if the connection is closed thus the caller will not continue
+  // the validation, otherwise return true.
+  bool SendPathChallenge(const QuicPathFrameBuffer& data_buffer,
                          const QuicSocketAddress& self_address,
                          const QuicSocketAddress& peer_address,
-                         QuicPacketWriter* writer);
+                         QuicPacketWriter* writer) override;
+  // If |writer| is the default writer and |peer_address| is the same as
+  // peer_address(), return the PTO of this connection. Otherwise, return 3 *
+  // kInitialRtt.
+  QuicTime GetRetryTimeout(const QuicSocketAddress& peer_address_to_use,
+                           QuicPacketWriter* writer_to_use) const override;
+
+  // Start vaildating the path defined by |context| asynchronously and call the
+  // |result_delegate| after validation finishes.
+  void ValidatePath(
+      std::unique_ptr<QuicPathValidationContext> context,
+      std::unique_ptr<QuicPathValidator::ResultDelegate> result_delegate);
 
   bool can_receive_ack_frequency_frame() const {
     return can_receive_ack_frequency_frame_;
@@ -1138,6 +1156,13 @@
     return use_encryption_level_context_;
   }
 
+  bool HasPendingPathValidation() const;
+
+  void MigratePath(const QuicSocketAddress& self_address,
+                   const QuicSocketAddress& peer_address,
+                   QuicPacketWriter* writer,
+                   bool owns_writer);
+
  protected:
   // Calls cancel() on all the alarms owned by this connection.
   void CancelAllAlarms();
@@ -1912,6 +1937,11 @@
   // --gfe2_reloadable_flag_quic_start_peer_migration_earlier.
   bool send_path_response_ = start_peer_migration_earlier_ &&
                              GetQuicReloadableFlag(quic_send_path_response);
+
+  bool use_path_validator_ =
+      send_path_response_ &&
+      GetQuicReloadableFlag(quic_pass_path_response_to_validator);
+
   // True if AckFrequencyFrame is supported.
   bool can_receive_ack_frequency_frame_ = false;
 
@@ -1941,6 +1971,8 @@
   const bool encrypted_control_frames_;
 
   const bool use_encryption_level_context_;
+
+  QuicPathValidator path_validator_;
 };
 
 }  // namespace quic
diff --git a/quic/core/quic_connection_test.cc b/quic/core/quic_connection_test.cc
index 5f8af1c..93169ea 100644
--- a/quic/core/quic_connection_test.cc
+++ b/quic/core/quic_connection_test.cc
@@ -24,6 +24,7 @@
 #include "net/third_party/quiche/src/quic/core/quic_constants.h"
 #include "net/third_party/quiche/src/quic/core/quic_error_codes.h"
 #include "net/third_party/quiche/src/quic/core/quic_packets.h"
+#include "net/third_party/quiche/src/quic/core/quic_path_validator.h"
 #include "net/third_party/quiche/src/quic/core/quic_simple_buffer_allocator.h"
 #include "net/third_party/quiche/src/quic/core/quic_types.h"
 #include "net/third_party/quiche/src/quic/core/quic_utils.h"
@@ -40,6 +41,7 @@
 #include "net/third_party/quiche/src/quic/test_tools/quic_connection_peer.h"
 #include "net/third_party/quiche/src/quic/test_tools/quic_framer_peer.h"
 #include "net/third_party/quiche/src/quic/test_tools/quic_packet_creator_peer.h"
+#include "net/third_party/quiche/src/quic/test_tools/quic_path_validator_peer.h"
 #include "net/third_party/quiche/src/quic/test_tools/quic_sent_packet_manager_peer.h"
 #include "net/third_party/quiche/src/quic/test_tools/quic_test_utils.h"
 #include "net/third_party/quiche/src/quic/test_tools/simple_data_producer.h"
@@ -1389,6 +1391,24 @@
     // Prevent packets from being coalesced.
     EXPECT_CALL(visitor_, GetHandshakeState())
         .WillRepeatedly(Return(HANDSHAKE_CONFIRMED));
+    // Clear direct_peer_address.
+    QuicConnectionPeer::SetDirectPeerAddress(&connection_, QuicSocketAddress());
+    // Clear effective_peer_address, it is the same as direct_peer_address for
+    // this test.
+    QuicConnectionPeer::SetEffectivePeerAddress(&connection_,
+                                                QuicSocketAddress());
+    EXPECT_FALSE(connection_.effective_peer_address().IsInitialized());
+
+    if (QuicVersionUsesCryptoFrames(connection_.transport_version())) {
+      EXPECT_CALL(visitor_, OnCryptoFrame(_)).Times(AnyNumber());
+    } else {
+      EXPECT_CALL(visitor_, OnStreamFrame(_)).Times(AnyNumber());
+    }
+    QuicPacketCreatorPeer::SetPacketNumber(&peer_creator_, 2);
+    ProcessFramePacketWithAddresses(MakeCryptoFrame(), kSelfAddress,
+                                    kPeerAddress, ENCRYPTION_FORWARD_SECURE);
+    EXPECT_EQ(kPeerAddress, connection_.peer_address());
+    EXPECT_EQ(kPeerAddress, connection_.effective_peer_address());
   }
 
   void TestClientRetryHandling(bool invalid_retry_tag,
@@ -1725,24 +1745,6 @@
 TEST_P(QuicConnectionTest, ReceivePathProbeWithNoAddressChangeAtServer) {
   PathProbeTestInit(Perspective::IS_SERVER);
 
-  // Clear direct_peer_address.
-  QuicConnectionPeer::SetDirectPeerAddress(&connection_, QuicSocketAddress());
-  // Clear effective_peer_address, it is the same as direct_peer_address for
-  // this test.
-  QuicConnectionPeer::SetEffectivePeerAddress(&connection_,
-                                              QuicSocketAddress());
-  EXPECT_FALSE(connection_.effective_peer_address().IsInitialized());
-
-  if (QuicVersionUsesCryptoFrames(connection_.transport_version())) {
-    EXPECT_CALL(visitor_, OnCryptoFrame(_)).Times(AnyNumber());
-  } else {
-    EXPECT_CALL(visitor_, OnStreamFrame(_)).Times(AnyNumber());
-  }
-  ProcessFramePacketWithAddresses(MakeCryptoFrame(), kSelfAddress, kPeerAddress,
-                                  ENCRYPTION_INITIAL);
-  EXPECT_EQ(kPeerAddress, connection_.peer_address());
-  EXPECT_EQ(kPeerAddress, connection_.effective_peer_address());
-
   EXPECT_CALL(visitor_, OnConnectionMigration(PORT_CHANGE)).Times(0);
   EXPECT_CALL(visitor_, OnPacketReceived(_, _, false)).Times(0);
 
@@ -1840,24 +1842,6 @@
 TEST_P(QuicConnectionTest, ReceivePathProbingAtServer) {
   PathProbeTestInit(Perspective::IS_SERVER);
 
-  // Clear direct_peer_address.
-  QuicConnectionPeer::SetDirectPeerAddress(&connection_, QuicSocketAddress());
-  // Clear effective_peer_address, it is the same as direct_peer_address for
-  // this test.
-  QuicConnectionPeer::SetEffectivePeerAddress(&connection_,
-                                              QuicSocketAddress());
-  EXPECT_FALSE(connection_.effective_peer_address().IsInitialized());
-
-  if (QuicVersionUsesCryptoFrames(connection_.transport_version())) {
-    EXPECT_CALL(visitor_, OnCryptoFrame(_)).Times(AnyNumber());
-  } else {
-    EXPECT_CALL(visitor_, OnStreamFrame(_)).Times(AnyNumber());
-  }
-  ProcessFramePacketWithAddresses(MakeCryptoFrame(), kSelfAddress, kPeerAddress,
-                                  ENCRYPTION_INITIAL);
-  EXPECT_EQ(kPeerAddress, connection_.peer_address());
-  EXPECT_EQ(kPeerAddress, connection_.effective_peer_address());
-
   EXPECT_CALL(visitor_, OnConnectionMigration(PORT_CHANGE)).Times(0);
   if (!GetParam().version.HasIetfQuicFrames()) {
     EXPECT_CALL(visitor_,
@@ -1979,25 +1963,6 @@
 TEST_P(QuicConnectionTest, ReceiveReorderedPathProbingAtServer) {
   PathProbeTestInit(Perspective::IS_SERVER);
 
-  // Clear direct_peer_address.
-  QuicConnectionPeer::SetDirectPeerAddress(&connection_, QuicSocketAddress());
-  // Clear effective_peer_address, it is the same as direct_peer_address for
-  // this test.
-  QuicConnectionPeer::SetEffectivePeerAddress(&connection_,
-                                              QuicSocketAddress());
-  EXPECT_FALSE(connection_.effective_peer_address().IsInitialized());
-
-  if (QuicVersionUsesCryptoFrames(connection_.transport_version())) {
-    EXPECT_CALL(visitor_, OnCryptoFrame(_)).Times(AnyNumber());
-  } else {
-    EXPECT_CALL(visitor_, OnStreamFrame(_)).Times(AnyNumber());
-  }
-  QuicPacketCreatorPeer::SetPacketNumber(&peer_creator_, 5);
-  ProcessFramePacketWithAddresses(MakeCryptoFrame(), kSelfAddress, kPeerAddress,
-                                  ENCRYPTION_INITIAL);
-  EXPECT_EQ(kPeerAddress, connection_.peer_address());
-  EXPECT_EQ(kPeerAddress, connection_.effective_peer_address());
-
   // Decrease packet number to simulate out-of-order packets.
   QuicPacketCreatorPeer::SetPacketNumber(&peer_creator_, 4);
 
@@ -2035,24 +2000,6 @@
 TEST_P(QuicConnectionTest, MigrateAfterProbingAtServer) {
   PathProbeTestInit(Perspective::IS_SERVER);
 
-  // Clear direct_peer_address.
-  QuicConnectionPeer::SetDirectPeerAddress(&connection_, QuicSocketAddress());
-  // Clear effective_peer_address, it is the same as direct_peer_address for
-  // this test.
-  QuicConnectionPeer::SetEffectivePeerAddress(&connection_,
-                                              QuicSocketAddress());
-  EXPECT_FALSE(connection_.effective_peer_address().IsInitialized());
-
-  if (QuicVersionUsesCryptoFrames(connection_.transport_version())) {
-    EXPECT_CALL(visitor_, OnCryptoFrame(_)).Times(AnyNumber());
-  } else {
-    EXPECT_CALL(visitor_, OnStreamFrame(_)).Times(AnyNumber());
-  }
-  ProcessFramePacketWithAddresses(MakeCryptoFrame(), kSelfAddress, kPeerAddress,
-                                  ENCRYPTION_INITIAL);
-  EXPECT_EQ(kPeerAddress, connection_.peer_address());
-  EXPECT_EQ(kPeerAddress, connection_.effective_peer_address());
-
   EXPECT_CALL(visitor_, OnConnectionMigration(PORT_CHANGE)).Times(0);
   if (!GetParam().version.HasIetfQuicFrames()) {
     EXPECT_CALL(visitor_,
@@ -2090,24 +2037,6 @@
   EXPECT_CALL(visitor_, OnSuccessfulVersionNegotiation(_));
   PathProbeTestInit(Perspective::IS_CLIENT);
 
-  // Clear direct_peer_address.
-  QuicConnectionPeer::SetDirectPeerAddress(&connection_, QuicSocketAddress());
-  // Clear effective_peer_address, it is the same as direct_peer_address for
-  // this test.
-  QuicConnectionPeer::SetEffectivePeerAddress(&connection_,
-                                              QuicSocketAddress());
-  EXPECT_FALSE(connection_.effective_peer_address().IsInitialized());
-
-  if (QuicVersionUsesCryptoFrames(connection_.transport_version())) {
-    EXPECT_CALL(visitor_, OnCryptoFrame(_)).Times(AnyNumber());
-  } else {
-    EXPECT_CALL(visitor_, OnStreamFrame(_)).Times(AnyNumber());
-  }
-  ProcessFramePacketWithAddresses(MakeCryptoFrame(), kSelfAddress, kPeerAddress,
-                                  ENCRYPTION_INITIAL);
-  EXPECT_EQ(kPeerAddress, connection_.peer_address());
-  EXPECT_EQ(kPeerAddress, connection_.effective_peer_address());
-
   // Client takes all padded PING packet as speculative connectivity
   // probing packet, and reports to visitor.
   EXPECT_CALL(visitor_, OnConnectionMigration(PORT_CHANGE)).Times(0);
@@ -2143,24 +2072,6 @@
   EXPECT_CALL(visitor_, OnSuccessfulVersionNegotiation(_));
   PathProbeTestInit(Perspective::IS_CLIENT);
 
-  // Clear direct_peer_address.
-  QuicConnectionPeer::SetDirectPeerAddress(&connection_, QuicSocketAddress());
-  // Clear effective_peer_address, it is the same as direct_peer_address for
-  // this test.
-  QuicConnectionPeer::SetEffectivePeerAddress(&connection_,
-                                              QuicSocketAddress());
-  EXPECT_FALSE(connection_.effective_peer_address().IsInitialized());
-
-  if (QuicVersionUsesCryptoFrames(connection_.transport_version())) {
-    EXPECT_CALL(visitor_, OnCryptoFrame(_)).Times(AnyNumber());
-  } else {
-    EXPECT_CALL(visitor_, OnStreamFrame(_)).Times(AnyNumber());
-  }
-  ProcessFramePacketWithAddresses(MakeCryptoFrame(), kSelfAddress, kPeerAddress,
-                                  ENCRYPTION_INITIAL);
-  EXPECT_EQ(kPeerAddress, connection_.peer_address());
-  EXPECT_EQ(kPeerAddress, connection_.effective_peer_address());
-
   // Process a padded PING packet with a different self address on client side
   // is effectively receiving a connectivity probing.
   EXPECT_CALL(visitor_, OnConnectionMigration(PORT_CHANGE)).Times(0);
@@ -8592,29 +8503,6 @@
   EXPECT_CALL(visitor_, OnSuccessfulVersionNegotiation(_));
   PathProbeTestInit(Perspective::IS_CLIENT);
 
-  // Clear direct_peer_address and effective_peer_address.
-  QuicConnectionPeer::SetDirectPeerAddress(&connection_, QuicSocketAddress());
-  QuicConnectionPeer::SetEffectivePeerAddress(&connection_,
-                                              QuicSocketAddress());
-  EXPECT_FALSE(connection_.effective_peer_address().IsInitialized());
-
-  EXPECT_TRUE(connection_.connected());
-  EXPECT_CALL(visitor_, ShouldKeepConnectionAlive())
-      .WillRepeatedly(Return(true));
-  EXPECT_FALSE(connection_.PathDegradingDetectionInProgress());
-  EXPECT_FALSE(connection_.IsPathDegrading());
-  EXPECT_FALSE(connection_.GetPingAlarm()->IsSet());
-
-  if (QuicVersionUsesCryptoFrames(connection_.transport_version())) {
-    EXPECT_CALL(visitor_, OnCryptoFrame(_)).Times(AnyNumber());
-  } else {
-    EXPECT_CALL(visitor_, OnStreamFrame(_)).Times(AnyNumber());
-  }
-  ProcessFramePacketWithAddresses(MakeCryptoFrame(), kSelfAddress, kPeerAddress,
-                                  ENCRYPTION_INITIAL);
-  EXPECT_EQ(kPeerAddress, connection_.peer_address());
-  EXPECT_EQ(kPeerAddress, connection_.effective_peer_address());
-
   // Send data and verify the path degrading detection is set.
   const char data[] = "data";
   size_t data_size = strlen(data);
@@ -8672,7 +8560,7 @@
 
   // Verify new path degrading detection is activated.
   EXPECT_CALL(visitor_, OnForwardProgressMadeAfterPathDegrading()).Times(1);
-  connection_.OnSuccessfulMigrationAfterProbing();
+  connection_.OnSuccessfulMigration();
   EXPECT_FALSE(connection_.IsPathDegrading());
   EXPECT_TRUE(connection_.PathDegradingDetectionInProgress());
 }
@@ -11334,44 +11222,96 @@
   EXPECT_TRUE(connection_.connected());
 }
 
-TEST_P(QuicConnectionTest, SendPathChallenge) {
+class TestQuicPathValidationContext : public QuicPathValidationContext {
+ public:
+  TestQuicPathValidationContext(const QuicSocketAddress& self_address,
+                                const QuicSocketAddress& peer_address,
+
+                                QuicPacketWriter* writer)
+      : QuicPathValidationContext(self_address, peer_address),
+        writer_(writer) {}
+
+  QuicPacketWriter* WriterToUse() override { return writer_; }
+
+ private:
+  QuicPacketWriter* writer_;
+};
+
+class TestValidationResultDelegate : public QuicPathValidator::ResultDelegate {
+ public:
+  TestValidationResultDelegate(const QuicSocketAddress& expected_self_address,
+                               const QuicSocketAddress& expected_peer_address,
+                               bool* success)
+      : QuicPathValidator::ResultDelegate(),
+        expected_self_address_(expected_self_address),
+        expected_peer_address_(expected_peer_address),
+        success_(success) {}
+  void OnPathValidationSuccess(
+      std::unique_ptr<QuicPathValidationContext> context) override {
+    EXPECT_EQ(expected_self_address_, context->self_address());
+    EXPECT_EQ(expected_peer_address_, context->peer_address());
+    *success_ = true;
+  }
+
+  void OnPathValidationFailure(
+      std::unique_ptr<QuicPathValidationContext> context) override {
+    EXPECT_EQ(expected_self_address_, context->self_address());
+    EXPECT_EQ(expected_peer_address_, context->peer_address());
+    *success_ = false;
+  }
+
+ private:
+  QuicSocketAddress expected_self_address_;
+  QuicSocketAddress expected_peer_address_;
+  bool* success_;
+};
+
+TEST_P(QuicConnectionTest, PathValidationOnNewSocketSuccess) {
   if (!VersionHasIetfQuicFrames(connection_.version().transport_version) ||
-      !connection_.send_path_response()) {
+      !connection_.use_path_validator()) {
     return;
   }
   PathProbeTestInit(Perspective::IS_CLIENT);
-  const QuicSocketAddress kNewSourceAddress(QuicIpAddress::Any6(), 12345);
-  EXPECT_NE(kNewSourceAddress, connection_.self_address());
+  const QuicSocketAddress kNewSelfAddress(QuicIpAddress::Any4(), 12345);
+  EXPECT_NE(kNewSelfAddress, connection_.self_address());
   TestPacketWriter new_writer(version(), &clock_, Perspective::IS_CLIENT);
-  QuicPathFrameBuffer payload{{0xde, 0xad, 0xbe, 0xef, 0xba, 0xdc, 0x0f, 0xfe}};
   EXPECT_CALL(*send_algorithm_, OnPacketSent(_, _, _, _, _))
+      .Times(AtLeast(1u))
       .WillOnce(Invoke([&]() {
         EXPECT_EQ(1u, new_writer.packets_write_attempts());
         EXPECT_EQ(1u, new_writer.path_challenge_frames().size());
-        EXPECT_EQ(
-            0, memcmp(payload.data(),
-                      &(new_writer.path_challenge_frames().front().data_buffer),
-                      sizeof(payload)));
         EXPECT_EQ(1u, new_writer.padding_frames().size());
-        EXPECT_EQ(kNewSourceAddress.host(),
+        EXPECT_EQ(kNewSelfAddress.host(),
                   new_writer.last_write_source_address());
       }));
-  connection_.SendPathChallenge(payload, kNewSourceAddress,
-                                connection_.peer_address(), &new_writer);
+  bool success = false;
+  connection_.ValidatePath(
+      std::make_unique<TestQuicPathValidationContext>(
+          kNewSelfAddress, connection_.peer_address(), &new_writer),
+      std::make_unique<TestValidationResultDelegate>(
+          kNewSelfAddress, connection_.peer_address(), &success));
   EXPECT_EQ(0u, writer_->packets_write_attempts());
+
+  QuicFrames frames;
+  frames.push_back(QuicFrame(new QuicPathResponseFrame(
+      99, new_writer.path_challenge_frames().front().data_buffer)));
+  ProcessFramesPacketWithAddresses(frames, kNewSelfAddress, kPeerAddress,
+                                   ENCRYPTION_FORWARD_SECURE);
+  EXPECT_TRUE(success);
 }
 
+// Tests that PATH_CHALLENGE is dropped if it is sent via a blocked alternative
+// writer.
 TEST_P(QuicConnectionTest, SendPathChallengeUsingBlockedNewSocket) {
   if (!VersionHasIetfQuicFrames(connection_.version().transport_version) ||
-      !connection_.send_path_response()) {
+      !connection_.use_path_validator()) {
     return;
   }
   PathProbeTestInit(Perspective::IS_CLIENT);
-  const QuicSocketAddress kNewSourceAddress(QuicIpAddress::Any6(), 12345);
-  EXPECT_NE(kNewSourceAddress, connection_.self_address());
+  const QuicSocketAddress kNewSelfAddress(QuicIpAddress::Any4(), 12345);
+  EXPECT_NE(kNewSelfAddress, connection_.self_address());
   TestPacketWriter new_writer(version(), &clock_, Perspective::IS_CLIENT);
   new_writer.BlockOnNextWrite();
-  QuicPathFrameBuffer payload{{0xde, 0xad, 0xbe, 0xef, 0xba, 0xdc, 0x0f, 0xfe}};
   EXPECT_CALL(visitor_, OnWriteBlocked()).Times(0);
   EXPECT_CALL(*send_algorithm_, OnPacketSent(_, _, _, _, _))
       .WillOnce(Invoke([&]() {
@@ -11379,16 +11319,16 @@
         // treated as sent.
         EXPECT_EQ(1u, new_writer.packets_write_attempts());
         EXPECT_EQ(1u, new_writer.path_challenge_frames().size());
-        EXPECT_EQ(
-            0, memcmp(payload.data(),
-                      &(new_writer.path_challenge_frames().front().data_buffer),
-                      sizeof(payload)));
         EXPECT_EQ(1u, new_writer.padding_frames().size());
-        EXPECT_EQ(kNewSourceAddress.host(),
+        EXPECT_EQ(kNewSelfAddress.host(),
                   new_writer.last_write_source_address());
       }));
-  connection_.SendPathChallenge(payload, kNewSourceAddress,
-                                connection_.peer_address(), &new_writer);
+  bool success = false;
+  connection_.ValidatePath(
+      std::make_unique<TestQuicPathValidationContext>(
+          kNewSelfAddress, connection_.peer_address(), &new_writer),
+      std::make_unique<TestValidationResultDelegate>(
+          kNewSelfAddress, connection_.peer_address(), &success));
   EXPECT_EQ(0u, writer_->packets_write_attempts());
 
   new_writer.SetWritable();
@@ -11398,7 +11338,9 @@
   EXPECT_EQ(1u, new_writer.packets_write_attempts());
 }
 
-TEST_P(QuicConnectionTest, SendPathChallengeWithDefaultSocketBlocked) {
+//  Tests that PATH_CHALLENGE is dropped if it is sent via the default writer
+//  and the writer is blocked.
+TEST_P(QuicConnectionTest, SendPathChallengeUsingBlockedDefaultSocket) {
   if (!VersionHasIetfQuicFrames(connection_.version().transport_version) ||
       !connection_.send_path_response()) {
     return;
@@ -11407,69 +11349,79 @@
   if (version().SupportsAntiAmplificationLimit()) {
     QuicConnectionPeer::SetAddressValidated(&connection_);
   }
-  const QuicSocketAddress kNewPeerAddress(QuicIpAddress::Any6(), 12345);
+  const QuicSocketAddress kNewPeerAddress(QuicIpAddress::Any4(), 12345);
   writer_->BlockOnNextWrite();
-  QuicPathFrameBuffer payload{{0xde, 0xad, 0xbe, 0xef, 0xba, 0xdc, 0x0f, 0xfe}};
   // 1st time is after writer returns WRITE_STATUS_BLOCKED. 2nd time is in
   // ShouldGeneratePacket();
   EXPECT_CALL(visitor_, OnWriteBlocked()).Times(2u);
   // This packet isn't sent actually, instead it is buffered in the connection.
   EXPECT_CALL(*send_algorithm_, OnPacketSent(_, _, _, _, _))
+      .Times(AtLeast(1u))
       .WillOnce(Invoke([&]() {
         EXPECT_EQ(1u, writer_->path_challenge_frames().size());
-        EXPECT_EQ(
-            0, memcmp(payload.data(),
-                      &(writer_->path_challenge_frames().front().data_buffer),
-                      sizeof(payload)));
         EXPECT_EQ(1u, writer_->padding_frames().size());
         EXPECT_EQ(kNewPeerAddress, writer_->last_write_peer_address());
+      }))
+      .WillRepeatedly(Invoke([&]() {
+        // Only one PATH_CHALLENGE should be sent out.
+        EXPECT_EQ(0u, writer_->path_challenge_frames().size());
       }));
-  connection_.SendPathChallenge(payload, connection_.self_address(),
-                                kNewPeerAddress, writer_.get());
+  bool success = false;
+  connection_.ValidatePath(
+      std::make_unique<TestQuicPathValidationContext>(
+          connection_.self_address(), kNewPeerAddress, writer_.get()),
+      std::make_unique<TestValidationResultDelegate>(
+          connection_.self_address(), kNewPeerAddress, &success));
   EXPECT_EQ(1u, writer_->packets_write_attempts());
 
-  memset(payload.data(), 0, sizeof(payload));
   // Try again with the new socket blocked from the beginning. The 2nd
   // PATH_CHALLENGE shouldn't be serialized, but be dropped.
-  connection_.SendPathChallenge(payload, connection_.self_address(),
-                                kNewPeerAddress, writer_.get());
+  clock_.AdvanceTime(QuicTime::Delta::FromMilliseconds(3 * kInitialRttMs));
+  static_cast<test::MockRandom*>(helper_->GetRandomGenerator())->ChangeValue();
+  static_cast<TestAlarmFactory::TestAlarm*>(
+      QuicPathValidatorPeer::retry_timer(
+          QuicConnectionPeer::path_validator(&connection_)))
+      ->Fire();
+
   // No more write attempt should be made.
   EXPECT_EQ(1u, writer_->packets_write_attempts());
 
   writer_->SetWritable();
   // OnCanWrite() should actually write out the 1st PATH_CHALLENGE packet
-  // buffered earlier, thus incrementing the write counter.
+  // buffered earlier, thus incrementing the write counter. It may also send
+  // ACKs to previously received packets.
   connection_.OnCanWrite();
-  EXPECT_EQ(2u, writer_->packets_write_attempts());
+  EXPECT_LE(2u, writer_->packets_write_attempts());
 }
 
 // Tests that write error on the alternate socket should be ignored.
 TEST_P(QuicConnectionTest, SendPathChallengeFailOnNewSocket) {
   if (!VersionHasIetfQuicFrames(connection_.version().transport_version) ||
-      !connection_.send_path_response()) {
+      !connection_.use_path_validator()) {
     return;
   }
   PathProbeTestInit(Perspective::IS_CLIENT);
-  const QuicSocketAddress kNewSourceAddress(QuicIpAddress::Any6(), 12345);
-  EXPECT_NE(kNewSourceAddress, connection_.self_address());
+  const QuicSocketAddress kNewSelfAddress(QuicIpAddress::Any4(), 12345);
+  EXPECT_NE(kNewSelfAddress, connection_.self_address());
   TestPacketWriter new_writer(version(), &clock_, Perspective::IS_CLIENT);
   new_writer.SetShouldWriteFail();
-  QuicPathFrameBuffer payload{{0xde, 0xad, 0xbe, 0xef, 0xba, 0xdc, 0x0f, 0xfe}};
   EXPECT_CALL(visitor_, OnConnectionClosed(_, ConnectionCloseSource::FROM_SELF))
       .Times(0);
+  EXPECT_CALL(*send_algorithm_, OnPacketSent(_, _, _, _, _)).Times(0u);
 
-  connection_.SendPathChallenge(payload, kNewSourceAddress,
-                                connection_.peer_address(), &new_writer);
-  // Regardless of the write error, the PATH_CHALLENGE should still be
-  // treated as sent.
+  bool success = false;
+  connection_.ValidatePath(
+      std::make_unique<TestQuicPathValidationContext>(
+          kNewSelfAddress, connection_.peer_address(), &new_writer),
+      std::make_unique<TestValidationResultDelegate>(
+          kNewSelfAddress, connection_.peer_address(), &success));
   EXPECT_EQ(1u, new_writer.packets_write_attempts());
   EXPECT_EQ(1u, new_writer.path_challenge_frames().size());
-  EXPECT_EQ(0, memcmp(payload.data(),
-                      &(new_writer.path_challenge_frames().front().data_buffer),
-                      sizeof(payload)));
   EXPECT_EQ(1u, new_writer.padding_frames().size());
-  EXPECT_EQ(kNewSourceAddress.host(), new_writer.last_write_source_address());
+  EXPECT_EQ(kNewSelfAddress.host(), new_writer.last_write_source_address());
+
   EXPECT_EQ(0u, writer_->packets_write_attempts());
+  //  Regardless of the write error, the connection should still be connected.
   EXPECT_TRUE(connection_.connected());
 }
 
@@ -11477,12 +11429,12 @@
 // should close the connection.
 TEST_P(QuicConnectionTest, SendPathChallengeFailOnDefaultPath) {
   if (!VersionHasIetfQuicFrames(connection_.version().transport_version) ||
-      !connection_.send_path_response()) {
+      !connection_.use_path_validator()) {
     return;
   }
   PathProbeTestInit(Perspective::IS_CLIENT);
+
   writer_->SetShouldWriteFail();
-  QuicPathFrameBuffer payload{{0xde, 0xad, 0xbe, 0xef, 0xba, 0xdc, 0x0f, 0xfe}};
   EXPECT_CALL(visitor_, OnConnectionClosed(_, ConnectionCloseSource::FROM_SELF))
       .WillOnce(
           Invoke([](QuicConnectionCloseFrame frame, ConnectionCloseSource) {
@@ -11493,17 +11445,83 @@
     // Add a flusher to force flush, otherwise the frames will remain in the
     // packet creator.
     QuicConnection::ScopedPacketFlusher flusher(&connection_);
-    connection_.SendPathChallenge(payload, connection_.self_address(),
-                                  connection_.peer_address(), writer_.get());
+    bool success = false;
+    connection_.ValidatePath(
+        std::make_unique<TestQuicPathValidationContext>(
+            connection_.self_address(), connection_.peer_address(),
+            writer_.get()),
+        std::make_unique<TestValidationResultDelegate>(
+            connection_.self_address(), connection_.peer_address(), &success));
   }
   EXPECT_EQ(1u, writer_->packets_write_attempts());
   EXPECT_EQ(1u, writer_->path_challenge_frames().size());
-  EXPECT_EQ(0, memcmp(payload.data(),
-                      &(writer_->path_challenge_frames().front().data_buffer),
-                      sizeof(payload)));
   EXPECT_EQ(1u, writer_->padding_frames().size());
   EXPECT_EQ(connection_.peer_address(), writer_->last_write_peer_address());
   EXPECT_FALSE(connection_.connected());
+  // Closing connection should abandon ongoing path validation.
+  EXPECT_FALSE(connection_.HasPendingPathValidation());
+}
+
+TEST_P(QuicConnectionTest, SendPathChallengeFailOnAlternativePeerAddress) {
+  if (!VersionHasIetfQuicFrames(connection_.version().transport_version) ||
+      !connection_.use_path_validator()) {
+    return;
+  }
+  PathProbeTestInit(Perspective::IS_SERVER);
+
+  writer_->SetShouldWriteFail();
+  const QuicSocketAddress kNewPeerAddress(QuicIpAddress::Any4(), 12345);
+  EXPECT_CALL(visitor_, OnConnectionClosed(_, ConnectionCloseSource::FROM_SELF))
+      .WillOnce(
+          Invoke([](QuicConnectionCloseFrame frame, ConnectionCloseSource) {
+            EXPECT_EQ(QUIC_PACKET_WRITE_ERROR, frame.quic_error_code);
+          }));
+  // Sending PATH_CHALLENGE to trigger a flush write which will fail and close
+  // the connection.
+  bool success = false;
+  connection_.ValidatePath(
+      std::make_unique<TestQuicPathValidationContext>(
+          connection_.self_address(), kNewPeerAddress, writer_.get()),
+      std::make_unique<TestValidationResultDelegate>(
+          connection_.self_address(), kNewPeerAddress, &success));
+
+  EXPECT_EQ(1u, writer_->packets_write_attempts());
+  EXPECT_FALSE(connection_.HasPendingPathValidation());
+  EXPECT_EQ(1u, writer_->path_challenge_frames().size());
+  EXPECT_EQ(1u, writer_->padding_frames().size());
+  EXPECT_EQ(kNewPeerAddress, writer_->last_write_peer_address());
+  EXPECT_FALSE(connection_.connected());
+}
+
+TEST_P(QuicConnectionTest,
+       SendPathChallengeFailPacketTooBigOnAlternativePeerAddress) {
+  if (!VersionHasIetfQuicFrames(connection_.version().transport_version) ||
+      !connection_.use_path_validator()) {
+    return;
+  }
+  PathProbeTestInit(Perspective::IS_SERVER);
+
+  writer_->SetShouldWriteFail();
+  writer_->SetWriteError(EMSGSIZE);
+  const QuicSocketAddress kNewPeerAddress(QuicIpAddress::Any4(), 12345);
+  EXPECT_CALL(visitor_, OnConnectionClosed(_, ConnectionCloseSource::FROM_SELF))
+      .Times(0u);
+  EXPECT_CALL(*send_algorithm_, OnPacketSent(_, _, _, _, _)).Times(0u);
+  // Sending PATH_CHALLENGE to trigger a flush write which will fail with
+  // MSG_TOO_BIG.
+  bool success = false;
+  connection_.ValidatePath(
+      std::make_unique<TestQuicPathValidationContext>(
+          connection_.self_address(), kNewPeerAddress, writer_.get()),
+      std::make_unique<TestValidationResultDelegate>(
+          connection_.self_address(), kNewPeerAddress, &success));
+  EXPECT_TRUE(connection_.HasPendingPathValidation());
+  // Connection shouldn't be closed.
+  EXPECT_TRUE(connection_.connected());
+  EXPECT_EQ(1u, writer_->packets_write_attempts());
+  EXPECT_EQ(1u, writer_->path_challenge_frames().size());
+  EXPECT_EQ(1u, writer_->padding_frames().size());
+  EXPECT_EQ(kNewPeerAddress, writer_->last_write_peer_address());
 }
 
 // Check that if there are two PATH_CHALLENGE frames in the packet, the latter
@@ -11514,24 +11532,6 @@
   }
   PathProbeTestInit(Perspective::IS_SERVER);
 
-  // Clear direct_peer_address.
-  QuicConnectionPeer::SetDirectPeerAddress(&connection_, QuicSocketAddress());
-  // Clear effective_peer_address, it is the same as direct_peer_address for
-  // this test.
-  QuicConnectionPeer::SetEffectivePeerAddress(&connection_,
-                                              QuicSocketAddress());
-  EXPECT_FALSE(connection_.effective_peer_address().IsInitialized());
-
-  if (QuicVersionUsesCryptoFrames(connection_.transport_version())) {
-    EXPECT_CALL(visitor_, OnCryptoFrame(_)).Times(AnyNumber());
-  } else {
-    EXPECT_CALL(visitor_, OnStreamFrame(_)).Times(AnyNumber());
-  }
-  ProcessFramePacketWithAddresses(MakeCryptoFrame(), kSelfAddress, kPeerAddress,
-                                  ENCRYPTION_FORWARD_SECURE);
-  EXPECT_EQ(kPeerAddress, connection_.peer_address());
-  EXPECT_EQ(kPeerAddress, connection_.effective_peer_address());
-
   QuicPathFrameBuffer path_frame_buffer1{0, 1, 2, 3, 4, 5, 6, 7};
   QuicPathFrameBuffer path_frame_buffer2{8, 9, 10, 11, 12, 13, 14, 15};
   QuicFrames frames;
@@ -11584,24 +11584,6 @@
   }
   PathProbeTestInit(Perspective::IS_SERVER);
 
-  // Clear direct_peer_address.
-  QuicConnectionPeer::SetDirectPeerAddress(&connection_, QuicSocketAddress());
-  // Clear effective_peer_address, it is the same as direct_peer_address for
-  // this test.
-  QuicConnectionPeer::SetEffectivePeerAddress(&connection_,
-                                              QuicSocketAddress());
-  EXPECT_FALSE(connection_.effective_peer_address().IsInitialized());
-
-  if (QuicVersionUsesCryptoFrames(connection_.transport_version())) {
-    EXPECT_CALL(visitor_, OnCryptoFrame(_)).Times(AnyNumber());
-  } else {
-    EXPECT_CALL(visitor_, OnStreamFrame(_)).Times(AnyNumber());
-  }
-  ProcessFramePacketWithAddresses(MakeCryptoFrame(), kSelfAddress, kPeerAddress,
-                                  ENCRYPTION_INITIAL);
-  EXPECT_EQ(kPeerAddress, connection_.peer_address());
-  EXPECT_EQ(kPeerAddress, connection_.effective_peer_address());
-
   QuicFrames frames;
   frames.push_back(QuicFrame(frame1_));
   QuicPathFrameBuffer path_frame_buffer{0, 1, 2, 3, 4, 5, 6, 7};
@@ -11646,24 +11628,6 @@
   }
   PathProbeTestInit(Perspective::IS_SERVER);
 
-  // Clear direct_peer_address.
-  QuicConnectionPeer::SetDirectPeerAddress(&connection_, QuicSocketAddress());
-  // Clear effective_peer_address, it is the same as direct_peer_address for
-  // this test.
-  QuicConnectionPeer::SetEffectivePeerAddress(&connection_,
-                                              QuicSocketAddress());
-  EXPECT_FALSE(connection_.effective_peer_address().IsInitialized());
-
-  if (QuicVersionUsesCryptoFrames(connection_.transport_version())) {
-    EXPECT_CALL(visitor_, OnCryptoFrame(_)).Times(AnyNumber());
-  } else {
-    EXPECT_CALL(visitor_, OnStreamFrame(_)).Times(AnyNumber());
-  }
-  ProcessFramePacketWithAddresses(MakeCryptoFrame(), kSelfAddress, kPeerAddress,
-                                  ENCRYPTION_INITIAL);
-  EXPECT_EQ(kPeerAddress, connection_.peer_address());
-  EXPECT_EQ(kPeerAddress, connection_.effective_peer_address());
-
   QuicFrames frames;
   QuicPathFrameBuffer path_frame_buffer{0, 1, 2, 3, 4, 5, 6, 7};
   frames.push_back(QuicFrame(new QuicPathChallengeFrame(0, path_frame_buffer)));
@@ -11717,25 +11681,6 @@
   }
   PathProbeTestInit(Perspective::IS_SERVER);
 
-  // Clear direct_peer_address.
-  QuicConnectionPeer::SetDirectPeerAddress(&connection_, QuicSocketAddress());
-  // Clear effective_peer_address, it is the same as direct_peer_address for
-  // this test.
-  QuicConnectionPeer::SetEffectivePeerAddress(&connection_,
-                                              QuicSocketAddress());
-  EXPECT_FALSE(connection_.effective_peer_address().IsInitialized());
-
-  if (QuicVersionUsesCryptoFrames(connection_.transport_version())) {
-    EXPECT_CALL(visitor_, OnCryptoFrame(_)).Times(AnyNumber());
-  } else {
-    EXPECT_CALL(visitor_, OnStreamFrame(_)).Times(AnyNumber());
-  }
-  QuicPacketCreatorPeer::SetPacketNumber(&peer_creator_, 2);
-  ProcessFramePacketWithAddresses(MakeCryptoFrame(), kSelfAddress, kPeerAddress,
-                                  ENCRYPTION_FORWARD_SECURE);
-  EXPECT_EQ(kPeerAddress, connection_.peer_address());
-  EXPECT_EQ(kPeerAddress, connection_.effective_peer_address());
-
   QuicFrames frames;
   frames.push_back(QuicFrame(frame1_));
   QuicPathFrameBuffer path_frame_buffer{0, 1, 2, 3, 4, 5, 6, 7};
@@ -11803,25 +11748,6 @@
   }
   PathProbeTestInit(Perspective::IS_SERVER);
 
-  // Clear direct_peer_address.
-  QuicConnectionPeer::SetDirectPeerAddress(&connection_, QuicSocketAddress());
-  // Clear effective_peer_address, it is the same as direct_peer_address for
-  // this test.
-  QuicConnectionPeer::SetEffectivePeerAddress(&connection_,
-                                              QuicSocketAddress());
-  EXPECT_FALSE(connection_.effective_peer_address().IsInitialized());
-
-  if (QuicVersionUsesCryptoFrames(connection_.transport_version())) {
-    EXPECT_CALL(visitor_, OnCryptoFrame(_)).Times(AnyNumber());
-  } else {
-    EXPECT_CALL(visitor_, OnStreamFrame(_)).Times(AnyNumber());
-  }
-  QuicPacketCreatorPeer::SetPacketNumber(&peer_creator_, 2);
-  ProcessFramePacketWithAddresses(MakeCryptoFrame(), kSelfAddress, kPeerAddress,
-                                  ENCRYPTION_INITIAL);
-  EXPECT_EQ(kPeerAddress, connection_.peer_address());
-  EXPECT_EQ(kPeerAddress, connection_.effective_peer_address());
-
   QuicFrames frames;
   QuicPathFrameBuffer path_frame_buffer{0, 1, 2, 3, 4, 5, 6, 7};
   frames.push_back(QuicFrame(new QuicPathChallengeFrame(0, path_frame_buffer)));
@@ -12945,6 +12871,20 @@
             connection_.GetRetransmissionAlarm()->deadline());
 }
 
+TEST_P(QuicConnectionTest, MigratePath) {
+  EXPECT_CALL(visitor_, OnPathDegrading());
+  connection_.OnPathDegradingDetected();
+  const QuicSocketAddress kNewSelfAddress(QuicIpAddress::Any4(), 12345);
+  EXPECT_NE(kNewSelfAddress, connection_.self_address());
+  TestPacketWriter new_writer(version(), &clock_, Perspective::IS_CLIENT);
+  EXPECT_CALL(visitor_, OnForwardProgressMadeAfterPathDegrading());
+  connection_.MigratePath(kNewSelfAddress, connection_.peer_address(),
+                          &new_writer, /*owns_writer=*/false);
+  EXPECT_EQ(kNewSelfAddress, connection_.self_address());
+  EXPECT_EQ(&new_writer, QuicConnectionPeer::GetWriter(&connection_));
+  EXPECT_FALSE(connection_.IsPathDegrading());
+}
+
 }  // namespace
 }  // namespace test
 }  // namespace quic
diff --git a/quic/core/quic_flags_list.h b/quic/core/quic_flags_list.h
index ed496e1..10c0cb7 100644
--- a/quic/core/quic_flags_list.h
+++ b/quic/core/quic_flags_list.h
@@ -63,6 +63,7 @@
 QUIC_FLAG(FLAGS_quic_reloadable_flag_quic_key_update_supported, true)
 QUIC_FLAG(FLAGS_quic_reloadable_flag_quic_let_connection_handle_pings, true)
 QUIC_FLAG(FLAGS_quic_reloadable_flag_quic_new_priority_update_frame, true)
+QUIC_FLAG(FLAGS_quic_reloadable_flag_quic_pass_path_response_to_validator, false)
 QUIC_FLAG(FLAGS_quic_reloadable_flag_quic_process_undecryptable_packets_after_async_decrypt_callback, true)
 QUIC_FLAG(FLAGS_quic_reloadable_flag_quic_record_received_min_ack_delay, true)
 QUIC_FLAG(FLAGS_quic_reloadable_flag_quic_reject_spdy_frames, true)
diff --git a/quic/core/quic_session.cc b/quic/core/quic_session.cc
index 91aca8f..44fe0c8 100644
--- a/quic/core/quic_session.cc
+++ b/quic/core/quic_session.cc
@@ -2561,5 +2561,22 @@
   return connection_->framer().GetEncryptionLevelToSendApplicationData();
 }
 
+void QuicSession::ValidatePath(
+    std::unique_ptr<QuicPathValidationContext> context,
+    std::unique_ptr<QuicPathValidator::ResultDelegate> result_delegate) {
+  connection_->ValidatePath(std::move(context), std::move(result_delegate));
+}
+
+bool QuicSession::HasPendingPathValidation() const {
+  return connection_->HasPendingPathValidation();
+}
+
+void QuicSession::MigratePath(const QuicSocketAddress& self_address,
+                              const QuicSocketAddress& peer_address,
+                              QuicPacketWriter* writer,
+                              bool owns_writer) {
+  connection_->MigratePath(self_address, peer_address, writer, owns_writer);
+}
+
 #undef ENDPOINT  // undef for jumbo builds
 }  // namespace quic
diff --git a/quic/core/quic_session.h b/quic/core/quic_session.h
index a36040e..3c95b2f 100644
--- a/quic/core/quic_session.h
+++ b/quic/core/quic_session.h
@@ -26,6 +26,7 @@
 #include "net/third_party/quiche/src/quic/core/quic_error_codes.h"
 #include "net/third_party/quiche/src/quic/core/quic_packet_creator.h"
 #include "net/third_party/quiche/src/quic/core/quic_packets.h"
+#include "net/third_party/quiche/src/quic/core/quic_path_validator.h"
 #include "net/third_party/quiche/src/quic/core/quic_stream.h"
 #include "net/third_party/quiche/src/quic/core/quic_stream_frame_data_producer.h"
 #include "net/third_party/quiche/src/quic/core/quic_types.h"
@@ -369,6 +370,77 @@
   // connection, or in a write-blocked stream.
   bool HasDataToWrite() const;
 
+  // Initiates a path validation on the path described in the given context,
+  // asynchronously calls |result_delegate| upon success or failure.
+  // The initiator should extend QuicPathValidationContext to provide the writer
+  // and ResultDelegate to react upon the validation result.
+  // Example implementations of these for path validation for connection
+  // migration could be:
+  //  class QUIC_EXPORT_PRIVATE PathMigrationContext
+  //      : public QuicPathValidationContext {
+  //   public:
+  //    PathMigrationContext(std::unique_ptr<QuicPacketWriter> writer,
+  //                         const QuicSocketAddress& self_address,
+  //                         const QuicSocketAddress& peer_address)
+  //        : QuicPathValidationContext(self_address, peer_address),
+  //          alternative_writer_(std::move(writer)) {}
+  //
+  //    QuicPacketWriter* WriterToUse() override {
+  //         return alternative_writer_.get();
+  //    }
+  //
+  //    QuicPacketWriter* ReleaseWriter() {
+  //         return alternative_writer_.release();
+  //    }
+  //
+  //   private:
+  //    std::unique_ptr<QuicPacketWriter> alternative_writer_;
+  //  };
+  //
+  //  class PathMigrationValidationResultDelegate
+  //      : public QuicPathValidator::ResultDelegate {
+  //   public:
+  //    PathMigrationValidationResultDelegate(QuicConnection* connection)
+  //        : QuicPathValidator::ResultDelegate(), connection_(connection) {}
+  //
+  //    void OnPathValidationSuccess(
+  //        std::unique_ptr<QuicPathValidationContext> context) override {
+  //    // Do some work to prepare for migration.
+  //    // ...
+  //
+  //    // Actually migrate to the validated path.
+  //    auto migration_context = std::unique_ptr<PathMigrationContext>(
+  //        static_cast<PathMigrationContext*>(context.release()));
+  //    connection_->MigratePath(migration_context->self_address(),
+  //                          migration_context->peer_address(),
+  //                          migration_context->ReleaseWriter(),
+  //                          /*owns_writer=*/true);
+  //
+  //    // Post-migration actions
+  //    // ...
+  //  }
+  //
+  //    void OnPathValidationFailure(
+  //        std::unique_ptr<QuicPathValidationContext> /*context*/) override {
+  //    // Handle validation failure.
+  //  }
+  //
+  //   private:
+  //    QuicConnection* connection_;
+  //  };
+  void ValidatePath(
+      std::unique_ptr<QuicPathValidationContext> context,
+      std::unique_ptr<QuicPathValidator::ResultDelegate> result_delegate);
+
+  // Return true if there is a path being validated.
+  bool HasPendingPathValidation() const;
+
+  // Switch to the path described in |context| without validating the path.
+  void MigratePath(const QuicSocketAddress& self_address,
+                   const QuicSocketAddress& peer_address,
+                   QuicPacketWriter* writer,
+                   bool owns_writer);
+
   // Returns the largest payload that will fit into a single MESSAGE frame.
   // Because overhead can vary during a connection, this method should be
   // checked for every message.
diff --git a/quic/test_tools/quic_connection_peer.cc b/quic/test_tools/quic_connection_peer.cc
index 30ed4fd..10bd838 100644
--- a/quic/test_tools/quic_connection_peer.cc
+++ b/quic/test_tools/quic_connection_peer.cc
@@ -399,5 +399,18 @@
   connection->SendPingAtLevel(connection->encryption_level());
 }
 
+// static
+void QuicConnectionPeer::SetLastPacketDestinationAddress(
+    QuicConnection* connection,
+    const QuicSocketAddress& address) {
+  connection->last_packet_destination_address_ = address;
+}
+
+// static
+QuicPathValidator* QuicConnectionPeer::path_validator(
+    QuicConnection* connection) {
+  return &connection->path_validator_;
+}
+
 }  // namespace test
 }  // namespace quic
diff --git a/quic/test_tools/quic_connection_peer.h b/quic/test_tools/quic_connection_peer.h
index 8859a58..1769033 100644
--- a/quic/test_tools/quic_connection_peer.h
+++ b/quic/test_tools/quic_connection_peer.h
@@ -164,6 +164,11 @@
   static void SetConnectionClose(QuicConnection* connection);
 
   static void SendPing(QuicConnection* connection);
+
+  static void SetLastPacketDestinationAddress(QuicConnection* connection,
+                                              const QuicSocketAddress& address);
+
+  static QuicPathValidator* path_validator(QuicConnection* connection);
 };
 
 }  // namespace test
diff --git a/quic/test_tools/quic_test_utils.cc b/quic/test_tools/quic_test_utils.cc
index 8bf49d9..7901b1c 100644
--- a/quic/test_tools/quic_test_utils.cc
+++ b/quic/test_tools/quic_test_utils.cc
@@ -1465,7 +1465,7 @@
   }
 
   if (ShouldWriteFail()) {
-    return WriteResult(WRITE_STATUS_ERROR, 0);
+    return WriteResult(WRITE_STATUS_ERROR, write_error_code_);
   }
 
   last_packet_size_ = packet.length();
diff --git a/quic/test_tools/quic_test_utils.h b/quic/test_tools/quic_test_utils.h
index f05859d..f764e0b 100644
--- a/quic/test_tools/quic_test_utils.h
+++ b/quic/test_tools/quic_test_utils.h
@@ -723,6 +723,13 @@
               SendMessage,
               (QuicMessageId, QuicMemSliceSpan, bool),
               (override));
+  MOCK_METHOD(bool,
+              SendPathChallenge,
+              (const QuicPathFrameBuffer&,
+               const QuicSocketAddress&,
+               const QuicSocketAddress&,
+               QuicPacketWriter*),
+              (override));
 
   MOCK_METHOD(void, OnError, (QuicFramer*), (override));
   void QuicConnection_OnError(QuicFramer* framer) {
@@ -766,6 +773,11 @@
       const QuicSocketAddress& peer_address) {
     QuicConnection::SendConnectivityProbingResponsePacket(peer_address);
   }
+
+  bool ReallyOnPathResponseFrame(const QuicPathResponseFrame& frame) {
+    return QuicConnection::OnPathResponseFrame(frame);
+  }
+
   MOCK_METHOD(bool,
               OnPathResponseFrame,
               (const QuicPathResponseFrame&),
@@ -2028,6 +2040,8 @@
 
   void SetShouldWriteFail() { write_should_fail_ = true; }
 
+  void SetWriteError(int error_code) { write_error_code_ = error_code; }
+
   QuicByteCount GetMaxPacketSize(
       const QuicSocketAddress& /*peer_address*/) const override {
     return max_packet_size_;
@@ -2223,6 +2237,7 @@
   // The soruce/peer address passed into WritePacket().
   QuicIpAddress last_write_source_address_;
   QuicSocketAddress last_write_peer_address_;
+  int write_error_code_{0};
 };
 
 // Parses a packet generated by
diff --git a/quic/tools/quic_client_base.cc b/quic/tools/quic_client_base.cc
index 5d89797..f4cd859 100644
--- a/quic/tools/quic_client_base.cc
+++ b/quic/tools/quic_client_base.cc
@@ -3,9 +3,12 @@
 // found in the LICENSE file.
 
 #include "net/third_party/quiche/src/quic/tools/quic_client_base.h"
+#include <memory>
 
 #include "net/third_party/quiche/src/quic/core/crypto/quic_random.h"
 #include "net/third_party/quiche/src/quic/core/http/spdy_utils.h"
+#include "net/third_party/quiche/src/quic/core/quic_packet_writer.h"
+#include "net/third_party/quiche/src/quic/core/quic_path_validator.h"
 #include "net/third_party/quiche/src/quic/core/quic_server_id.h"
 #include "net/third_party/quiche/src/quic/core/quic_utils.h"
 #include "net/third_party/quiche/src/quic/platform/api/quic_flags.h"
@@ -13,6 +16,60 @@
 
 namespace quic {
 
+// A path context which owns the writer.
+class QUIC_EXPORT_PRIVATE PathMigrationContext
+    : public QuicPathValidationContext {
+ public:
+  PathMigrationContext(std::unique_ptr<QuicPacketWriter> writer,
+                       const QuicSocketAddress& self_address,
+                       const QuicSocketAddress& peer_address)
+      : QuicPathValidationContext(self_address, peer_address),
+        alternative_writer_(std::move(writer)) {}
+
+  QuicPacketWriter* WriterToUse() override { return alternative_writer_.get(); }
+
+  QuicPacketWriter* ReleaseWriter() { return alternative_writer_.release(); }
+
+ private:
+  std::unique_ptr<QuicPacketWriter> alternative_writer_;
+};
+
+// Implements the basic feature of a result delegate for path validation for
+// connection migration. If the validation succeeds, migrate to the alternative
+// path. Otherwise, stay on the current path.
+class QuicClientSocketMigrationValidationResultDelegate
+    : public QuicPathValidator::ResultDelegate {
+ public:
+  QuicClientSocketMigrationValidationResultDelegate(QuicClientBase* client)
+      : QuicPathValidator::ResultDelegate(), client_(client) {}
+
+  // QuicPathValidator::ResultDelegate
+  // Overridden to start migration and takes the ownership of the writer in the
+  // context.
+  void OnPathValidationSuccess(
+      std::unique_ptr<QuicPathValidationContext> context) override {
+    QUIC_DLOG(INFO) << "Successfully validated path from " << *context
+                    << ". Migrate to it now.";
+    auto migration_context = std::unique_ptr<PathMigrationContext>(
+        static_cast<PathMigrationContext*>(context.release()));
+    client_->session()->MigratePath(
+        migration_context->self_address(), migration_context->peer_address(),
+        migration_context->WriterToUse(), /*owns_writer=*/false);
+    DCHECK(migration_context->WriterToUse() != nullptr);
+    // Hand the ownership of the alternative writer to the client.
+    client_->set_writer(migration_context->ReleaseWriter());
+  }
+
+  void OnPathValidationFailure(
+      std::unique_ptr<QuicPathValidationContext> context) override {
+    QUIC_LOG(WARNING) << "Fail to validate path " << *context
+                      << ", stop migrating.";
+  }
+
+ private:
+  QuicClientBase* client_;
+};
+
 QuicClientBase::NetworkHelper::~NetworkHelper() = default;
 
 QuicClientBase::QuicClientBase(
@@ -203,21 +260,55 @@
   }
 
   network_helper_->CleanUpAllUDPSockets();
+  std::unique_ptr<QuicPacketWriter> writer =
+      CreateWriterForNewNetwork(new_host, port);
+  if (writer == nullptr) {
+    return false;
+  }
+  session()->MigratePath(network_helper_->GetLatestClientAddress(),
+                         session()->connection()->peer_address(), writer.get(),
+                         false);
+  set_writer(writer.release());
+  return true;
+}
 
-  set_bind_to_address(new_host);
-  if (!network_helper_->CreateUDPSocketAndBind(server_address_,
-                                               bind_to_address_, port)) {
+bool QuicClientBase::ValidateAndMigrateSocket(const QuicIpAddress& new_host) {
+  DCHECK(VersionHasIetfQuicFrames(
+             session_->connection()->version().transport_version) &&
+         session_->connection()->use_path_validator());
+  if (!connected()) {
     return false;
   }
 
-  session()->connection()->SetSelfAddress(
-      network_helper_->GetLatestClientAddress());
+  std::unique_ptr<QuicPacketWriter> writer =
+      CreateWriterForNewNetwork(new_host, local_port_);
+  if (writer == nullptr) {
+    return false;
+  }
+  // Asynchronously start migration.
+  session_->ValidatePath(
+      std::make_unique<PathMigrationContext>(
+          std::move(writer), network_helper_->GetLatestClientAddress(),
+          session_->peer_address()),
+      std::make_unique<QuicClientSocketMigrationValidationResultDelegate>(
+          this));
+  return true;
+}
+
+std::unique_ptr<QuicPacketWriter> QuicClientBase::CreateWriterForNewNetwork(
+    const QuicIpAddress& new_host,
+    int port) {
+  set_bind_to_address(new_host);
+  if (!network_helper_->CreateUDPSocketAndBind(server_address_,
+                                               bind_to_address_, port)) {
+    return nullptr;
+  }
 
   QuicPacketWriter* writer = network_helper_->CreateQuicPacketWriter();
-  set_writer(writer);
-  session()->connection()->SetQuicPacketWriter(writer, false);
-
-  return true;
+  QUIC_LOG_IF(WARNING, writer == writer_.get())
+      << "The new writer is wrapped in the same wrapper as the old one, thus "
+         "appearing to have the same address as the old one.";
+  return std::unique_ptr<QuicPacketWriter>(writer);
 }
 
 bool QuicClientBase::ChangeEphemeralPort() {
@@ -351,4 +442,46 @@
   return false;
 }
 
+bool QuicClientBase::HasPendingPathValidation() {
+  return session()->HasPendingPathValidation();
+}
+
+class ValidationResultDelegate : public QuicPathValidator::ResultDelegate {
+ public:
+  ValidationResultDelegate(QuicClientBase* client)
+      : QuicPathValidator::ResultDelegate(), client_(client) {}
+
+  void OnPathValidationSuccess(
+      std::unique_ptr<QuicPathValidationContext> context) override {
+    QUIC_DLOG(INFO) << "Successfully validated path from " << *context;
+    client_->AddValidatedPath(std::move(context));
+  }
+  void OnPathValidationFailure(
+      std::unique_ptr<QuicPathValidationContext> context) override {
+    QUIC_LOG(WARNING) << "Fail to validate path " << *context
+                      << ", stop migrating.";
+  }
+
+ private:
+  QuicClientBase* client_;
+};
+
+void QuicClientBase::ValidateNewNetwork(const QuicIpAddress& host) {
+  std::unique_ptr<QuicPacketWriter> writer =
+      CreateWriterForNewNetwork(host, local_port_);
+  auto result_delegate = std::make_unique<ValidationResultDelegate>(this);
+  if (writer == nullptr) {
+    result_delegate->OnPathValidationFailure(
+        std::make_unique<PathMigrationContext>(
+            nullptr, network_helper_->GetLatestClientAddress(),
+            session_->peer_address()));
+    return;
+  }
+  session()->ValidatePath(
+      std::make_unique<PathMigrationContext>(
+          std::move(writer), network_helper_->GetLatestClientAddress(),
+          session_->peer_address()),
+      std::move(result_delegate));
+}
+
 }  // namespace quic
diff --git a/quic/tools/quic_client_base.h b/quic/tools/quic_client_base.h
index ce2444c..7a531e3 100644
--- a/quic/tools/quic_client_base.h
+++ b/quic/tools/quic_client_base.h
@@ -8,6 +8,7 @@
 #ifndef QUICHE_QUIC_TOOLS_QUIC_CLIENT_BASE_H_
 #define QUICHE_QUIC_TOOLS_QUIC_CLIENT_BASE_H_
 
+#include <memory>
 #include <string>
 
 #include "absl/base/attributes.h"
@@ -121,6 +122,11 @@
   // Migrate to a new socket (new_host, port) during an active connection.
   bool MigrateSocketWithSpecifiedPort(const QuicIpAddress& new_host, int port);
 
+  // Validate the new socket and migrate to it if the validation succeeds.
+  // Otherwise stay on the current socket. Return true if the validation has
+  // started.
+  bool ValidateAndMigrateSocket(const QuicIpAddress& new_host);
+
   // Open a new socket to change to a new ephemeral port.
   bool ChangeEphemeralPort();
 
@@ -244,6 +250,19 @@
     client_connection_id_length_ = client_connection_id_length;
   }
 
+  bool HasPendingPathValidation();
+
+  void ValidateNewNetwork(const QuicIpAddress& host);
+
+  void AddValidatedPath(std::unique_ptr<QuicPathValidationContext> context) {
+    validated_paths_.push_back(std::move(context));
+  }
+
+  const std::vector<std::unique_ptr<QuicPathValidationContext>>&
+  validated_paths() const {
+    return validated_paths_;
+  }
+
  protected:
   // TODO(rch): Move GetNumSentClientHellosFromSession and
   // GetNumReceivedServerConfigUpdatesFromSession into a new/better
@@ -299,6 +318,10 @@
   // version.
   bool CanReconnectWithDifferentVersion(ParsedQuicVersion* version) const;
 
+  std::unique_ptr<QuicPacketWriter> CreateWriterForNewNetwork(
+      const QuicIpAddress& new_host,
+      int port);
+
   // |server_id_| is a tuple (hostname, port, is_https) of the server.
   QuicServerId server_id_;
 
@@ -370,6 +393,9 @@
   // GetClientConnectionId creates a random connection ID of this length.
   // Defaults to 0.
   uint8_t client_connection_id_length_;
+
+  // Stores validated paths.
+  std::vector<std::unique_ptr<QuicPathValidationContext>> validated_paths_;
 };
 
 }  // namespace quic
diff --git a/quic/tools/quic_client_epoll_network_helper.cc b/quic/tools/quic_client_epoll_network_helper.cc
index 2a9ebba..9cde39c 100644
--- a/quic/tools/quic_client_epoll_network_helper.cc
+++ b/quic/tools/quic_client_epoll_network_helper.cc
@@ -129,7 +129,7 @@
 void QuicClientEpollNetworkHelper::OnShutdown(QuicEpollServer* /*eps*/,
                                               int /*fd*/) {}
 
-void QuicClientEpollNetworkHelper::OnEvent(int /*fd*/, QuicEpollEvent* event) {
+void QuicClientEpollNetworkHelper::OnEvent(int fd, QuicEpollEvent* event) {
   if (event->in_events & EPOLLIN) {
     QUIC_DVLOG(1) << "Read packets on EPOLLIN";
     int times_to_read = max_reads_per_epoll_loop_;
@@ -137,9 +137,8 @@
     QuicPacketCount packets_dropped = 0;
     while (client_->connected() && more_to_read && times_to_read > 0) {
       more_to_read = packet_reader_->ReadAndDispatchPackets(
-          GetLatestFD(), GetLatestClientAddress().port(),
-          *client_->helper()->GetClock(), this,
-          overflow_supported_ ? &packets_dropped : nullptr);
+          fd, GetLatestClientAddress().port(), *client_->helper()->GetClock(),
+          this, overflow_supported_ ? &packets_dropped : nullptr);
       --times_to_read;
     }
     if (packets_dropped_ < packets_dropped) {