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:
committed by
Facebook GitHub Bot
parent
f62ef489f9
commit
09b9ce567b
38
quic/BUCK
38
quic/BUCK
@@ -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 = [
|
||||
|
@@ -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
|
||||
|
@@ -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",
|
||||
|
@@ -28,6 +28,7 @@ add_dependencies(
|
||||
mvfst_batch_writer
|
||||
mvfst_async_udp_socket
|
||||
mvfst_events
|
||||
mvfst_config
|
||||
mvfst_constants
|
||||
mvfst_state_machine
|
||||
)
|
||||
|
@@ -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
|
||||
|
@@ -225,9 +225,10 @@ class QuicTransportBase : public QuicSocket,
|
||||
std::optional<QuicStreamGroupRetransmissionPolicy> policy) noexcept
|
||||
override;
|
||||
|
||||
[[nodiscard]] const folly::
|
||||
F14FastMap<StreamGroupId, QuicStreamGroupRetransmissionPolicy>&
|
||||
getStreamGroupRetransmissionPolicies() const {
|
||||
[[nodiscard]] const UnorderedMap<
|
||||
StreamGroupId,
|
||||
QuicStreamGroupRetransmissionPolicy>&
|
||||
getStreamGroupRetransmissionPolicies() const {
|
||||
return conn_->retransmissionPolicies;
|
||||
}
|
||||
|
||||
|
@@ -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;
|
||||
|
@@ -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",
|
||||
|
@@ -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
22
quic/mvfst-oss-config.h
Normal 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
|
@@ -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",
|
||||
],
|
||||
)
|
||||
|
@@ -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}>
|
||||
|
@@ -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 {
|
||||
|
@@ -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>
|
||||
|
@@ -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",
|
||||
]
|
||||
|
@@ -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
|
||||
|
@@ -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);
|
||||
|
||||
/**
|
||||
|
@@ -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",
|
||||
|
@@ -37,6 +37,7 @@ add_dependencies(
|
||||
mvfst_codec_types
|
||||
mvfst_dsr_sender
|
||||
mvfst_handshake
|
||||
mvfst_config
|
||||
)
|
||||
|
||||
target_link_libraries(
|
||||
|
@@ -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) {
|
||||
|
@@ -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_;
|
||||
|
@@ -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 {
|
||||
|
@@ -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;
|
||||
|
@@ -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.
|
||||
|
Reference in New Issue
Block a user