1
0
mirror of https://github.com/facebookincubator/mvfst.git synced 2025-08-06 22:22:38 +03:00

Introduce mvfst-config

Reviewed By: mjoras

Differential Revision: D73516282

fbshipit-source-id: 9033f64660988b0dc2cfdfbc1254e12cdd118892
This commit is contained in:
Konstantin Tsoy
2025-04-29 16:38:40 -07:00
committed by Facebook GitHub Bot
parent f62ef489f9
commit 09b9ce567b
24 changed files with 153 additions and 68 deletions

View File

@@ -1,7 +1,45 @@
load("@fbcode//quic:defs.bzl", "mvfst_cpp_library")
load("@fbsource//tools/build_defs:fb_native_wrapper.bzl", "fb_native")
oncall("traffic_protocols")
fb_native.constraint_setting(
name = "constraint-profile",
visibility = ["PUBLIC"],
)
fb_native.constraint_value(
name = "constraint-mobile",
constraint_setting = ":constraint-profile",
visibility = ["PUBLIC"],
)
fb_native.config_setting(
name = "config-mobile",
constraint_values = [
"fbsource//xplat/quic:constraint-mobile",
],
visibility = ["PUBLIC"],
)
mvfst_cpp_library(
name = "config",
headers = select({
"DEFAULT": select({
# @oss-only: "DEFAULT": {"mvfst-config.h": "//:mvfst-config.h"},
"DEFAULT": {"mvfst-config.h": "facebook/mvfst-config.h"}, # @fb-only
"ovr_config//runtime:fbcode": {"mvfst-config.h": "facebook/mvfst-config.h"}, # @fb-only
}),
"fbsource//xplat/quic:config-mobile": {"mvfst-config.h": "facebook/mvfst-config-mobile.h"}, # @fb-only
}),
exported_deps = select({
"DEFAULT": [
"//folly/container:f14_hash",
],
"fbsource//xplat/quic:config-mobile": [],
}),
)
mvfst_cpp_library(
name = "constants",
srcs = [

View File

@@ -3,6 +3,16 @@
# This source code is licensed under the MIT license found in the
# LICENSE file in the root directory of this source tree.
set(MVFST_OSS_CONFIG_INPUT_FILE "${PROJECT_SOURCE_DIR}/quic/mvfst-oss-config.h")
set(MVFST_OSS_CONFIG_OUTPUT_FILE "${PROJECT_SOURCE_DIR}/quic/mvfst-config.h")
add_custom_target(mvfst_config
${CMAKE_COMMAND} -E copy ${MVFST_OSS_CONFIG_INPUT_FILE}
${MVFST_OSS_CONFIG_OUTPUT_FILE}
)
install(FILES mvfst-config.h DESTINATION include/quic/)
add_library(
mvfst_constants
QuicConstants.cpp

View File

@@ -21,6 +21,7 @@ mvfst_cpp_library(
"//folly:network_address",
"//folly:portability",
"//folly/io:iobuf",
"//quic:config",
"//quic:constants",
"//quic/common/events:eventbase",
"//quic/common/udpsocket:quic_async_udp_socket",

View File

@@ -28,6 +28,7 @@ add_dependencies(
mvfst_batch_writer
mvfst_async_udp_socket
mvfst_events
mvfst_config
mvfst_constants
mvfst_state_machine
)

View File

@@ -7,6 +7,8 @@
#pragma once
#include <quic/mvfst-config.h>
#include <quic/api/QuicBatchWriter.h>
#include <quic/common/udpsocket/QuicAsyncUDPSocket.h>
@@ -131,7 +133,7 @@ class SendmmsgGSOPacketBatchWriter : public BatchWriter {
int idx_ = -1;
};
folly::F14FastMap<folly::SocketAddress, Index> addrMap_;
UnorderedMap<folly::SocketAddress, Index> addrMap_;
};
class SendmmsgGSOInplacePacketBatchWriter : public BatchWriter {
@@ -183,7 +185,7 @@ class SendmmsgGSOInplacePacketBatchWriter : public BatchWriter {
// have a size of 1000, whereas the third has a size of 1200.
// The first two would have the same index, with GSO enabled, while the third
// would have a different index, with GSO disabled.
folly::F14FastMap<folly::SocketAddress, uint32_t> addrToMostRecentIndex_;
UnorderedMap<folly::SocketAddress, uint32_t> addrToMostRecentIndex_;
};
} // namespace quic

View File

@@ -225,8 +225,9 @@ class QuicTransportBase : public QuicSocket,
std::optional<QuicStreamGroupRetransmissionPolicy> policy) noexcept
override;
[[nodiscard]] const folly::
F14FastMap<StreamGroupId, QuicStreamGroupRetransmissionPolicy>&
[[nodiscard]] const UnorderedMap<
StreamGroupId,
QuicStreamGroupRetransmissionPolicy>&
getStreamGroupRetransmissionPolicies() const {
return conn_->retransmissionPolicies;
}

View File

@@ -813,8 +813,8 @@ class QuicTransportBaseLite : virtual public QuicSocketLite,
DatagramCallback* datagramCallback_{nullptr};
folly::F14FastMap<StreamId, ReadCallbackData> readCallbacks_;
folly::F14FastMap<StreamId, PeekCallbackData> peekCallbacks_;
UnorderedMap<StreamId, ReadCallbackData> readCallbacks_;
UnorderedMap<StreamId, PeekCallbackData> peekCallbacks_;
ConnectionWriteCallback* connWriteCallback_{nullptr};
std::map<StreamId, StreamWriteCallback*> pendingWriteCallbacks_;
@@ -827,7 +827,7 @@ class QuicTransportBaseLite : virtual public QuicSocketLite,
ByteEventCallback* callback;
};
using ByteEventMap = folly::F14FastMap<StreamId, std::deque<ByteEventDetail>>;
using ByteEventMap = UnorderedMap<StreamId, std::deque<ByteEventDetail>>;
ByteEventMap& getByteEventMap(const ByteEvent::Type type);
[[nodiscard]] const ByteEventMap& getByteEventMapConst(
const ByteEvent::Type type) const;

View File

@@ -178,11 +178,11 @@ mvfst_cpp_test(
deps = [
":mocks",
"//folly:move_wrapper",
"//folly/container:f14_hash",
"//folly/futures:core",
"//folly/io/async/test:mocks",
"//folly/portability:gmock",
"//folly/portability:gtest",
"//quic:config",
"//quic/api:stream_async_transport",
"//quic/client:client",
"//quic/common/events:folly_eventbase",

View File

@@ -21,6 +21,7 @@
#include <quic/common/udpsocket/FollyQuicAsyncUDPSocket.h>
#include <quic/fizz/client/handshake/FizzClientHandshake.h>
#include <quic/fizz/client/handshake/FizzClientQuicHandshakeContext.h>
#include <quic/mvfst-config.h>
#include <quic/server/QuicServer.h>
#include <quic/server/QuicServerTransport.h>
#include <quic/server/test/Mocks.h>
@@ -194,7 +195,7 @@ class QuicStreamAsyncTransportTest : public Test {
NiceMock<MockConnectionSetupCallback> serverConnectionSetupCB_;
NiceMock<MockConnectionCallback> serverConnectionCB_;
std::shared_ptr<quic::QuicSocket> serverSocket_;
folly::F14FastMap<quic::StreamId, std::unique_ptr<Stream>> streams_;
UnorderedMap<quic::StreamId, std::unique_ptr<Stream>> streams_;
std::shared_ptr<QuicClientTransport> client_;
folly::EventBase clientEvb_;

22
quic/mvfst-oss-config.h Normal file
View File

@@ -0,0 +1,22 @@
/*
* Copyright (c) Meta Platforms, Inc. and affiliates.
*
* This source code is licensed under the MIT license found in the
* LICENSE file in the root directory of this source tree.
*/
#pragma once
#include <folly/container/F14Map.h>
#include <folly/container/F14Set.h>
namespace quic {
template <class... Args>
struct UnorderedMap : folly::F14FastMap<Args...> {};
template <class... Args>
struct ValueMap : folly::F14ValueMap<Args...> {};
template <class... Args>
struct UnorderedSet : folly::F14FastSet<Args...> {};
} // namespace quic

View File

@@ -27,7 +27,7 @@ mvfst_cpp_library(
],
exported_deps = [
":priority_queue",
"//folly/container:f14_hash",
"//quic:config",
"//quic/common:optional",
],
)
@@ -43,6 +43,6 @@ mvfst_cpp_library(
exported_deps = [
":priority_queue",
":round_robin",
"//folly/container:f14_hash",
"//quic:config",
],
)

View File

@@ -8,6 +8,8 @@ add_library(
RoundRobin.cpp
)
add_dependencies(mvfst_round_robin mvfst_config)
target_include_directories(
mvfst_round_robin PUBLIC
$<BUILD_INTERFACE:${QUIC_FBCODE_ROOT}>
@@ -47,6 +49,8 @@ add_library(
HTTPPriorityQueue.cpp
)
add_dependencies(mvfst_http_priority_queue mvfst_config)
target_include_directories(
mvfst_http_priority_queue PUBLIC
$<BUILD_INTERFACE:${QUIC_FBCODE_ROOT}>

View File

@@ -7,7 +7,8 @@
#pragma once
#include <folly/container/F14Map.h>
#include <quic/mvfst-config.h>
#include <quic/priority/PriorityQueue.h>
#include <quic/priority/RoundRobin.h>
@@ -23,8 +24,7 @@ class HTTPPriorityQueue : public quic::PriorityQueue {
uint64_t index : 63;
};
using IndexMap =
folly::F14ValueMap<Identifier, IndexMapElem, Identifier::hash>;
using IndexMap = ValueMap<Identifier, IndexMapElem, Identifier::hash>;
public:
class Priority : public quic::PriorityQueue::Priority {

View File

@@ -7,7 +7,8 @@
#pragma once
#include <folly/container/F14Map.h>
#include <quic/mvfst-config.h>
#include <quic/common/Optional.h>
#include <quic/priority/PriorityQueue.h>
#include <list>
@@ -37,7 +38,7 @@ class RoundRobin {
ListType list_;
ListType::iterator nextIt_{list_.end()};
folly::F14ValueMap<
ValueMap<
PriorityQueue::Identifier,
ListType::iterator,
PriorityQueue::Identifier::hash>

View File

@@ -2,7 +2,7 @@
Generated by xplat/cross_plat_devx/somerge_maps/compute_merge_maps.py
@generated SignedSource<<19e6026e2e5bdf22adb128b276f33027>>
@generated SignedSource<<32163b678d1b9d8c208f6dd522f9b5c7>>
"""
# Entry Points:
@@ -123,6 +123,7 @@ QUIC_NATIVE_LIBRARY_MERGE_MAP = [
"fbsource//xplat/quic/state:stats_callbackAndroid",
"fbsource//xplat/quic/state:stream_functionsAndroid",
"fbsource//xplat/quic/state:transport_settingsAndroid",
"fbsource//xplat/quic:configAndroid",
"fbsource//xplat/quic:constantsAndroid",
"fbsource//xplat/quic:exceptionAndroid",
]

View File

@@ -234,7 +234,7 @@ folly::Expected<AckEvent, QuicError> processAckFrame(
}
// Store any (new) Rx timestamps reported by the peer.
folly::F14FastMap<PacketNum, uint64_t> packetReceiveTimeStamps;
UnorderedMap<PacketNum, uint64_t> packetReceiveTimeStamps;
if (pnSpace == PacketNumberSpace::AppData) {
parseAckReceiveTimestamps(
conn, frame, packetReceiveTimeStamps, firstPacketNum);
@@ -482,7 +482,7 @@ void clearOldOutstandingPackets(
void parseAckReceiveTimestamps(
const QuicConnectionStateBase& conn,
const quic::ReadAckFrame& frame,
folly::F14FastMap<PacketNum, uint64_t>& packetReceiveTimeStamps,
UnorderedMap<PacketNum, uint64_t>& packetReceiveTimeStamps,
Optional<PacketNum> firstPacketNum) {
// Ignore if we didn't request packet receive timestamps from the peer.
if (!conn.transportSettings.maybeAckReceiveTimestampsConfigSentToPeer

View File

@@ -68,14 +68,14 @@ void commonAckVisitorForAckFrame(
const WriteAckFrame& frame);
/**
* Parse Receive timestamps from ACK frame into a folly::F14FastMap of packet
* Parse Receive timestamps from ACK frame into a UnorderedMap of packet
* number to timestamps and return the latest received packet with timestamp if
* any.
*/
void parseAckReceiveTimestamps(
const QuicConnectionStateBase& conn,
const quic::ReadAckFrame& frame,
folly::F14FastMap<PacketNum, uint64_t>& packetReceiveTimeStamps,
UnorderedMap<PacketNum, uint64_t>& packetReceiveTimeStamps,
Optional<PacketNum> firstPacketNum);
/**

View File

@@ -86,7 +86,7 @@ mvfst_cpp_library(
srcs = ["QuicPriorityQueue.cpp"],
headers = ["QuicPriorityQueue.h"],
exported_deps = [
"//folly/container:f14_hash",
"//quic:config",
"//quic/codec:types",
],
exported_external_deps = [
@@ -122,9 +122,9 @@ mvfst_cpp_library(
":retransmission_policy",
":stats_callback",
":transport_settings",
"//folly/container:f14_hash",
"//folly/io:iobuf",
"//folly/io/async:delayed_destruction",
"//quic:config",
"//quic:constants",
"//quic/codec:codec",
"//quic/codec:types",

View File

@@ -37,6 +37,7 @@ add_dependencies(
mvfst_codec_types
mvfst_dsr_sender
mvfst_handshake
mvfst_config
)
target_link_libraries(

View File

@@ -7,11 +7,11 @@
#pragma once
#include <folly/container/F14Map.h>
#include <glog/logging.h>
#include <set>
#include <glog/logging.h>
#include <quic/codec/Types.h>
#include <quic/mvfst-config.h>
namespace quic::deprecated {
@@ -181,7 +181,7 @@ struct PriorityQueue {
}
private:
folly::F14FastMap<StreamId, OrderId> streamToOrderId;
UnorderedMap<StreamId, OrderId> streamToOrderId;
};
std::vector<Level> levels;
@@ -310,7 +310,7 @@ struct PriorityQueue {
}
private:
folly::F14FastMap<StreamId, uint8_t> writableStreamsToLevel_;
UnorderedMap<StreamId, uint8_t> writableStreamsToLevel_;
using WSIterator = decltype(writableStreamsToLevel_)::iterator;
void eraseFromLevel(uint8_t levelIndex, StreamId id) {

View File

@@ -7,8 +7,8 @@
#pragma once
#include <folly/container/F14Map.h>
#include <folly/container/F14Set.h>
#include <quic/mvfst-config.h>
#include <quic/QuicConstants.h>
#include <quic/codec/Types.h>
#include <quic/priority/PriorityQueue.h>
@@ -894,38 +894,38 @@ class QuicStreamManager {
StreamIdSet openBidirectionalLocalStreamGroups_;
StreamIdSet openUnidirectionalLocalStreamGroups_;
folly::F14FastMap<StreamId, QuicStreamState> streams_;
UnorderedMap<StreamId, QuicStreamState> streams_;
std::vector<StreamId> newPeerStreams_;
std::vector<StreamId> newGroupedPeerStreams_;
folly::F14FastSet<StreamGroupId> newPeerStreamGroups_;
UnorderedSet<StreamGroupId> newPeerStreamGroups_;
StreamIdSet peerUnidirectionalStreamGroupsSeen_;
StreamIdSet peerBidirectionalStreamGroupsSeen_;
folly::F14FastMap<StreamId, StreamDataBlockedFrame> blockedStreams_;
folly::F14FastMap<StreamId, ApplicationErrorCode> stopSendingStreams_;
folly::F14FastSet<StreamId> windowUpdates_;
folly::F14FastSet<StreamId> flowControlUpdated_;
UnorderedMap<StreamId, StreamDataBlockedFrame> blockedStreams_;
UnorderedMap<StreamId, ApplicationErrorCode> stopSendingStreams_;
UnorderedSet<StreamId> windowUpdates_;
UnorderedSet<StreamId> flowControlUpdated_;
// Streams that were removed from the write queue because they are blocked
// on connection flow control.
folly::F14FastSet<StreamId> connFlowControlBlocked_;
UnorderedSet<StreamId> connFlowControlBlocked_;
// Streams that have bytes in loss buffer
folly::F14FastSet<StreamId> lossStreams_;
folly::F14FastSet<StreamId> lossDSRStreams_;
folly::F14FastSet<StreamId> readableStreams_;
folly::F14FastSet<StreamId> unidirectionalReadableStreams_;
folly::F14FastSet<StreamId> peekableStreams_;
UnorderedSet<StreamId> lossStreams_;
UnorderedSet<StreamId> lossDSRStreams_;
UnorderedSet<StreamId> readableStreams_;
UnorderedSet<StreamId> unidirectionalReadableStreams_;
UnorderedSet<StreamId> peekableStreams_;
std::unique_ptr<PriorityQueue> writeQueue_;
std::unique_ptr<deprecated::PriorityQueue> oldWriteQueue_;
std::set<StreamId> controlWriteQueue_;
folly::F14FastSet<StreamId> writableStreams_;
folly::F14FastSet<StreamId> writableDSRStreams_;
folly::F14FastSet<StreamId> txStreams_;
folly::F14FastSet<StreamId> deliverableStreams_;
folly::F14FastSet<StreamId> closedStreams_;
UnorderedSet<StreamId> writableStreams_;
UnorderedSet<StreamId> writableDSRStreams_;
UnorderedSet<StreamId> txStreams_;
UnorderedSet<StreamId> deliverableStreams_;
UnorderedSet<StreamId> closedStreams_;
bool isAppIdle_{false};
const TransportSettings* FOLLY_NONNULL transportSettings_;

View File

@@ -51,7 +51,7 @@ struct OutstandingsInfo {
// All PacketEvents of this connection. If a OutstandingPacketWrapper doesn't
// have an maybeClonedPacketIdentifier or if it's not in this set, there is no
// need to process its frames upon ack or loss.
folly::F14FastSet<ClonedPacketIdentifier, ClonedPacketIdentifierHash>
UnorderedSet<ClonedPacketIdentifier, ClonedPacketIdentifierHash>
clonedPacketIdentifiers;
// Number of outstanding packets not including cloned
@@ -271,7 +271,7 @@ struct RstStreamEvent {
ApplicationErrorCode errorCode;
};
using Resets = folly::F14FastMap<StreamId, RstStreamFrame>;
using Resets = UnorderedMap<StreamId, RstStreamFrame>;
using FrameList = std::vector<QuicSimpleFrame>;
@@ -713,7 +713,7 @@ struct QuicConnectionStateBase : public folly::DelayedDestruction {
ExtendedAckFeatureMaskType negotiatedExtendedAckFeatures{0};
// Retransmission policies map.
folly::F14FastMap<StreamGroupId, QuicStreamGroupRetransmissionPolicy>
UnorderedMap<StreamGroupId, QuicStreamGroupRetransmissionPolicy>
retransmissionPolicies;
struct SocketCmsgsState {

View File

@@ -7,12 +7,13 @@
#pragma once
#include <folly/container/F14Map.h>
#include <folly/container/F14Set.h>
#include <quic/mvfst-config.h>
#include <quic/QuicConstants.h>
#include <quic/codec/Types.h>
#include <quic/common/SmallCollections.h>
#include <quic/dsr/DSRPacketizationRequestSender.h>
#include <quic/mvfst-config.h>
#include <quic/priority/PriorityQueue.h>
namespace quic {
@@ -136,7 +137,7 @@ struct QuicStreamLike {
// are currently un-acked. Each one represents one StreamFrame that was
// written. We need to buffer these because these might be retransmitted in
// the future. These are associated with the starting offset of the buffer.
folly::F14FastMap<uint64_t, std::unique_ptr<WriteStreamBuffer>>
UnorderedMap<uint64_t, std::unique_ptr<WriteStreamBuffer>>
retransmissionBuffer;
// Tracks intervals which we have received ACKs for. E.g. in the case of all
@@ -312,7 +313,7 @@ struct QuicStreamLike {
}
void removeFromRetransmissionBufStartingAtOffset(uint64_t startingOffset) {
folly::F14FastSet<uint64_t> offsetsToRemove;
UnorderedSet<uint64_t> offsetsToRemove;
for (auto& [offset, buf] : retransmissionBuffer) {
if (offset >= startingOffset) {
@@ -613,7 +614,7 @@ struct QuicStreamState : public QuicStreamLike {
void removeFromRetransmissionBufMetasStartingAtOffset(
uint64_t startingOffset) {
folly::F14FastSet<uint64_t> offsetsToRemove;
UnorderedSet<uint64_t> offsetsToRemove;
for (auto& [offset, buf] : retransmissionBufMetas) {
if (offset >= startingOffset) {
@@ -672,7 +673,7 @@ struct QuicStreamState : public QuicStreamLike {
WriteBufferMeta writeBufMeta;
// A map to store sent WriteBufferMetas for potential retransmission.
folly::F14FastMap<uint64_t, WriteBufferMeta> retransmissionBufMetas;
UnorderedMap<uint64_t, WriteBufferMeta> retransmissionBufMetas;
// WriteBufferMetas that's already marked lost. They will be retransmitted.
CircularDeque<WriteBufferMeta> lossBufMetas;

View File

@@ -14,6 +14,7 @@
#include <quic/common/test/TestUtils.h>
#include <quic/fizz/server/handshake/FizzServerQuicHandshakeContext.h>
#include <quic/logging/test/Mocks.h>
#include <quic/mvfst-config.h>
#include <quic/server/state/ServerStateMachine.h>
#include <quic/state/AckHandlers.h>
#include <quic/state/OutstandingPacket.h>
@@ -101,11 +102,11 @@ auto getAckPacketMatcher(
auto testAckEventReceiveTimestampsAll(
const AckEvent& ackEvent,
const folly::F14FastMap<PacketNum, uint64_t>& expectedReceiveTimestamps) {
const UnorderedMap<PacketNum, uint64_t>& expectedReceiveTimestamps) {
// Lambda function to create a map from ackedPackets
auto createReceiveTimestampsMap =
[](const std::vector<AckEvent::AckPacket>& ackedPackets) {
folly::F14FastMap<PacketNum, uint64_t> receiveTimestampsMap;
UnorderedMap<PacketNum, uint64_t> receiveTimestampsMap;
for (const auto& packet : ackedPackets) {
if (packet.receiveRelativeTimeStampUsec.has_value()) {
receiveTimestampsMap.emplace(
@@ -136,7 +137,7 @@ auto getNumAckReceiveTimestamps(const AckEvent& ackEvent) {
// given timestamp range for later matching.
uint64_t buildExpectedReceiveTimestamps(
const RecvdPacketsTimestampsRange& timestampsRange,
folly::F14FastMap<PacketNum, uint64_t>& expectedReceiveTimestamps,
UnorderedMap<PacketNum, uint64_t>& expectedReceiveTimestamps,
quic::PacketNum latestReceivedUdpPacketWithAddedGap,
uint64_t lastReceiveTimestamp,
uint64_t maxTimestamps) {
@@ -2756,7 +2757,7 @@ TEST_P(AckHandlersTest, AckEventReceiveTimestamps) {
ackFrame.ackDelay = 5ms;
const auto ackTime = startTime + 10ms + ackFrame.ackDelay;
folly::F14FastMap<PacketNum, uint64_t> expectedReceiveTimestamps;
UnorderedMap<PacketNum, uint64_t> expectedReceiveTimestamps;
if (GetParam().frameType == FrameType::ACK_RECEIVE_TIMESTAMPS) {
conn.transportSettings.maybeAckReceiveTimestampsConfigSentToPeer.assign(
AckReceiveTimestampsConfig{
@@ -2821,7 +2822,7 @@ TEST_P(AckHandlersTest, AckEventReceiveTimestampsGaps) {
ackFrame.ackBlocks.emplace_back(4, 6);
ackFrame.ackBlocks.emplace_back(0, 1);
ackFrame.ackDelay = 5ms;
folly::F14FastMap<PacketNum, uint64_t> expectedReceiveTimestamps;
UnorderedMap<PacketNum, uint64_t> expectedReceiveTimestamps;
const auto ackTime = startTime + 10ms + ackFrame.ackDelay;
if (GetParam().frameType == FrameType::ACK_RECEIVE_TIMESTAMPS) {
@@ -2920,7 +2921,7 @@ TEST_P(AckHandlersTest, AckEventReceiveTimestampsDuplicatesAll) {
auto ackTime = startTime + 10ms + ackFrame.ackDelay;
// Build the expected received timestamps map.
folly::F14FastMap<PacketNum, uint64_t> expectedReceiveTimestamps;
UnorderedMap<PacketNum, uint64_t> expectedReceiveTimestamps;
if (GetParam().frameType == FrameType::ACK_RECEIVE_TIMESTAMPS) {
conn.transportSettings.maybeAckReceiveTimestampsConfigSentToPeer.assign(
AckReceiveTimestampsConfig{
@@ -3005,7 +3006,7 @@ TEST_P(AckHandlersTest, AckEventReceiveTimestampsPartialDuplicates) {
// send 10 packets
emplacePackets(conn, 10, startTime, GetParam().pnSpace);
// Build the expected received timestamps map.
folly::F14FastMap<PacketNum, uint64_t> expectedReceiveTimestamps;
UnorderedMap<PacketNum, uint64_t> expectedReceiveTimestamps;
{
ReadAckFrame ackFrame;
ackFrame.frameType = GetParam().frameType;
@@ -3090,7 +3091,7 @@ TEST_P(AckHandlersTest, AckEventReceiveTimestampsPartialDuplicates) {
4);
}
folly::F14FastMap<PacketNum, uint64_t> receivedTimestamps;
UnorderedMap<PacketNum, uint64_t> receivedTimestamps;
parseAckReceiveTimestamps(conn, ackFrame2, receivedTimestamps, 6);
// Ack Event will not have the old packets anyway so to unit-test
// duplicate detection, we will directly call parseAckReceiveTimestamps
@@ -3119,7 +3120,7 @@ TEST_P(AckHandlersTest, AckEventReceiveTimestampsOutOfOrderAcks) {
// send 10 packets
emplacePackets(conn, 10, startTime, GetParam().pnSpace);
// Build the expected received timestamps map.
folly::F14FastMap<PacketNum, uint64_t> expectedReceiveTimestamps;
UnorderedMap<PacketNum, uint64_t> expectedReceiveTimestamps;
{
// First send an ACK for (6-9) and not (0-5) with 6-9 timestamps. This is
@@ -3260,7 +3261,7 @@ TEST_P(AckHandlersTest, AckEventReceiveTimestampsMaxCheck) {
ackFrame.ackDelay = 5ms;
const auto ackTime = startTime + 10ms + ackFrame.ackDelay;
// Build the expected received timestamps map for all ACK frame types.
folly::F14FastMap<PacketNum, uint64_t> expectedReceiveTimestamps;
UnorderedMap<PacketNum, uint64_t> expectedReceiveTimestamps;
if (GetParam().frameType == FrameType::ACK_RECEIVE_TIMESTAMPS) {
// Set max requested receive timestamps to 5 and send more than that.
conn.transportSettings.maybeAckReceiveTimestampsConfigSentToPeer.assign(
@@ -3348,7 +3349,7 @@ TEST_P(AckHandlersTest, AckEventReceiveTimestampsInvalidCases) {
ackFrame.maybeLatestRecvdPacketTime = 100ms;
}
folly::F14FastMap<PacketNum, uint64_t> expectedReceiveTimestamps;
UnorderedMap<PacketNum, uint64_t> expectedReceiveTimestamps;
parseAckReceiveTimestamps(
conn, ackFrame, expectedReceiveTimestamps, firstPacketNum);
@@ -3372,7 +3373,7 @@ TEST_P(AckHandlersTest, AckEventReceiveTimestampsInvalidCases) {
ackFrame.recvdPacketsTimestampRanges = {recvdPacketsTimestampsRange1};
}
folly::F14FastMap<PacketNum, uint64_t> expectedReceiveTimestamps;
UnorderedMap<PacketNum, uint64_t> expectedReceiveTimestamps;
parseAckReceiveTimestamps(
conn, ackFrame, expectedReceiveTimestamps, firstPacketNum);
// No packets should be stored.