Update bitrate only when we have incoming packet.
Also cleans up some unused code and makes sure the min bitrate of the BWE can't be set to anything lower than 10 kbps. BUG=webrtc:5474 R=pbos@webrtc.org Review URL: https://codereview.webrtc.org/1699903003 . Cr-Commit-Position: refs/heads/master@{#11636}
This commit is contained in:
parent
58cf5f14ec
commit
62a5ccdb53
@ -10,6 +10,7 @@
|
||||
|
||||
#include "webrtc/call/congestion_controller.h"
|
||||
|
||||
#include <algorithm>
|
||||
#include <vector>
|
||||
|
||||
#include "webrtc/base/checks.h"
|
||||
@ -82,11 +83,6 @@ class WrappingBitrateEstimator : public RemoteBitrateEstimator {
|
||||
return rbe_->LatestEstimate(ssrcs, bitrate_bps);
|
||||
}
|
||||
|
||||
bool GetStats(ReceiveBandwidthEstimatorStats* output) const override {
|
||||
CriticalSectionScoped cs(crit_sect_.get());
|
||||
return rbe_->GetStats(output);
|
||||
}
|
||||
|
||||
void SetMinBitrate(int min_bitrate_bps) {
|
||||
CriticalSectionScoped cs(crit_sect_.get());
|
||||
rbe_->SetMinBitrate(min_bitrate_bps);
|
||||
@ -193,8 +189,18 @@ CongestionController::~CongestionController() {
|
||||
void CongestionController::SetBweBitrates(int min_bitrate_bps,
|
||||
int start_bitrate_bps,
|
||||
int max_bitrate_bps) {
|
||||
if (start_bitrate_bps > 0)
|
||||
// TODO(holmer): We should make sure the default bitrates are set to 10 kbps,
|
||||
// and that we don't try to set the min bitrate to 0 from any applications.
|
||||
// The congestion controller should allow a min bitrate of 0.
|
||||
const int kMinBitrateBps = 10000;
|
||||
if (min_bitrate_bps < kMinBitrateBps)
|
||||
min_bitrate_bps = kMinBitrateBps;
|
||||
if (max_bitrate_bps > 0)
|
||||
max_bitrate_bps = std::max(min_bitrate_bps, max_bitrate_bps);
|
||||
if (start_bitrate_bps > 0) {
|
||||
start_bitrate_bps = std::max(min_bitrate_bps, start_bitrate_bps);
|
||||
bitrate_controller_->SetStartBitrate(start_bitrate_bps);
|
||||
}
|
||||
bitrate_controller_->SetMinMaxBitrate(min_bitrate_bps, max_bitrate_bps);
|
||||
if (remote_bitrate_estimator_.get())
|
||||
remote_bitrate_estimator_->SetMinBitrate(min_bitrate_bps);
|
||||
|
||||
@ -31,7 +31,6 @@ class MockRemoteBitrateEstimator : public RemoteBitrateEstimator {
|
||||
MOCK_METHOD4(IncomingPacket, void(int64_t, size_t, const RTPHeader&, bool));
|
||||
MOCK_METHOD1(RemoveStream, void(uint32_t));
|
||||
MOCK_CONST_METHOD2(LatestEstimate, bool(std::vector<uint32_t>*, uint32_t*));
|
||||
MOCK_CONST_METHOD1(GetStats, bool(ReceiveBandwidthEstimatorStats*));
|
||||
|
||||
// From CallStatsObserver;
|
||||
MOCK_METHOD2(OnRttUpdate, void(int64_t, int64_t));
|
||||
|
||||
@ -38,27 +38,6 @@ class RemoteBitrateObserver {
|
||||
virtual ~RemoteBitrateObserver() {}
|
||||
};
|
||||
|
||||
struct ReceiveBandwidthEstimatorStats {
|
||||
ReceiveBandwidthEstimatorStats() : total_propagation_time_delta_ms(0) {}
|
||||
|
||||
// The "propagation_time_delta" of a frame is defined as (d_arrival - d_sent),
|
||||
// where d_arrival is the delta of the arrival times of the frame and the
|
||||
// previous frame, d_sent is the delta of the sent times of the frame and
|
||||
// the previous frame. The sent time is calculated from the RTP timestamp.
|
||||
|
||||
// |total_propagation_time_delta_ms| is the sum of the propagation_time_deltas
|
||||
// of all received frames, except that it's is adjusted to 0 when it becomes
|
||||
// negative.
|
||||
int total_propagation_time_delta_ms;
|
||||
// The propagation_time_deltas for the frames arrived in the last
|
||||
// kProcessIntervalMs using the clock passed to
|
||||
// RemoteBitrateEstimatorFactory::Create.
|
||||
std::vector<int> recent_propagation_time_delta_ms;
|
||||
// The arrival times for the frames arrived in the last kProcessIntervalMs
|
||||
// using the clock passed to RemoteBitrateEstimatorFactory::Create.
|
||||
std::vector<int64_t> recent_arrival_time_ms;
|
||||
};
|
||||
|
||||
class RemoteBitrateEstimator : public CallStatsObserver, public Module {
|
||||
public:
|
||||
static const int kDefaultMinBitrateBps = 30000;
|
||||
@ -88,9 +67,6 @@ class RemoteBitrateEstimator : public CallStatsObserver, public Module {
|
||||
virtual bool LatestEstimate(std::vector<uint32_t>* ssrcs,
|
||||
uint32_t* bitrate_bps) const = 0;
|
||||
|
||||
// Returns true if the statistics are available.
|
||||
virtual bool GetStats(ReceiveBandwidthEstimatorStats* output) const = 0;
|
||||
|
||||
virtual void SetMinBitrate(int min_bitrate_bps) = 0;
|
||||
|
||||
protected:
|
||||
|
||||
@ -37,24 +37,9 @@ enum {
|
||||
kExpectedNumberOfProbes = 3
|
||||
};
|
||||
|
||||
static const size_t kPropagationDeltaQueueMaxSize = 1000;
|
||||
static const int64_t kPropagationDeltaQueueMaxTimeMs = 1000;
|
||||
static const double kTimestampToMs = 1000.0 /
|
||||
static_cast<double>(1 << kInterArrivalShift);
|
||||
|
||||
// Removes the entries at |index| of |time| and |value|, if time[index] is
|
||||
// smaller than or equal to |deadline|. |time| must be sorted ascendingly.
|
||||
static void RemoveStaleEntries(
|
||||
std::vector<int64_t>* time, std::vector<int>* value, int64_t deadline) {
|
||||
assert(time->size() == value->size());
|
||||
std::vector<int64_t>::iterator end_of_removal = std::upper_bound(
|
||||
time->begin(), time->end(), deadline);
|
||||
size_t end_of_removal_index = end_of_removal - time->begin();
|
||||
|
||||
time->erase(time->begin(), end_of_removal);
|
||||
value->erase(value->begin(), value->begin() + end_of_removal_index);
|
||||
}
|
||||
|
||||
template<typename K, typename V>
|
||||
std::vector<K> Keys(const std::map<K, V>& map) {
|
||||
std::vector<K> keys;
|
||||
@ -94,29 +79,24 @@ bool RemoteBitrateEstimatorAbsSendTime::IsWithinClusterBounds(
|
||||
clusters->push_back(*cluster);
|
||||
}
|
||||
|
||||
int RemoteBitrateEstimatorAbsSendTime::Id() const {
|
||||
return static_cast<int>(reinterpret_cast<uint64_t>(this));
|
||||
}
|
||||
|
||||
RemoteBitrateEstimatorAbsSendTime::RemoteBitrateEstimatorAbsSendTime(
|
||||
RemoteBitrateObserver* observer,
|
||||
Clock* clock)
|
||||
: crit_sect_(CriticalSectionWrapper::CreateCriticalSection()),
|
||||
observer_(observer),
|
||||
clock_(clock),
|
||||
ssrcs_(),
|
||||
: observer_(observer),
|
||||
inter_arrival_(),
|
||||
estimator_(OverUseDetectorOptions()),
|
||||
detector_(OverUseDetectorOptions()),
|
||||
incoming_bitrate_(kBitrateWindowMs, 8000),
|
||||
last_process_time_(-1),
|
||||
process_interval_ms_(kProcessIntervalMs),
|
||||
total_propagation_delta_ms_(0),
|
||||
total_probes_received_(0),
|
||||
first_packet_time_ms_(-1) {
|
||||
assert(observer_);
|
||||
assert(clock_);
|
||||
LOG(LS_INFO) << "RemoteBitrateEstimatorAbsSendTime: Instantiating.";
|
||||
first_packet_time_ms_(-1),
|
||||
last_update_ms_(-1),
|
||||
ssrcs_(),
|
||||
clock_(clock) {
|
||||
RTC_DCHECK(observer_);
|
||||
RTC_DCHECK(clock_);
|
||||
LOG(LS_INFO) << "RemoteBitrateEstimatorAbsSendTime: Instantiating.";
|
||||
network_thread_.DetachFromThread();
|
||||
process_thread_.DetachFromThread();
|
||||
}
|
||||
|
||||
void RemoteBitrateEstimatorAbsSendTime::ComputeClusters(
|
||||
@ -183,7 +163,8 @@ RemoteBitrateEstimatorAbsSendTime::FindBestProbe(
|
||||
return best_it;
|
||||
}
|
||||
|
||||
void RemoteBitrateEstimatorAbsSendTime::ProcessClusters(int64_t now_ms) {
|
||||
RemoteBitrateEstimatorAbsSendTime::ProbeResult
|
||||
RemoteBitrateEstimatorAbsSendTime::ProcessClusters(int64_t now_ms) {
|
||||
std::list<Cluster> clusters;
|
||||
ComputeClusters(&clusters);
|
||||
if (clusters.empty()) {
|
||||
@ -191,7 +172,7 @@ void RemoteBitrateEstimatorAbsSendTime::ProcessClusters(int64_t now_ms) {
|
||||
// we will remove the oldest one.
|
||||
if (probes_.size() >= kMaxProbePackets)
|
||||
probes_.pop_front();
|
||||
return;
|
||||
return ProbeResult::kNoUpdate;
|
||||
}
|
||||
|
||||
std::list<Cluster>::const_iterator best_it = FindBestProbe(clusters);
|
||||
@ -209,6 +190,7 @@ void RemoteBitrateEstimatorAbsSendTime::ProcessClusters(int64_t now_ms) {
|
||||
<< " ms, mean recv delta: " << best_it->recv_mean_ms
|
||||
<< " ms, num probes: " << best_it->count;
|
||||
remote_rate_.SetEstimate(probe_bitrate_bps, now_ms);
|
||||
return ProbeResult::kBitrateUpdated;
|
||||
}
|
||||
}
|
||||
|
||||
@ -216,6 +198,7 @@ void RemoteBitrateEstimatorAbsSendTime::ProcessClusters(int64_t now_ms) {
|
||||
// of probes.
|
||||
if (clusters.size() >= kExpectedNumberOfProbes)
|
||||
probes_.clear();
|
||||
return ProbeResult::kNoUpdate;
|
||||
}
|
||||
|
||||
bool RemoteBitrateEstimatorAbsSendTime::IsBitrateImproving(
|
||||
@ -229,6 +212,7 @@ bool RemoteBitrateEstimatorAbsSendTime::IsBitrateImproving(
|
||||
|
||||
void RemoteBitrateEstimatorAbsSendTime::IncomingPacketFeedbackVector(
|
||||
const std::vector<PacketInfo>& packet_feedback_vector) {
|
||||
RTC_DCHECK(network_thread_.CalledOnValidThread());
|
||||
for (const auto& packet_info : packet_feedback_vector) {
|
||||
IncomingPacketInfo(packet_info.arrival_time_ms,
|
||||
ConvertMsTo24Bits(packet_info.send_time_ms),
|
||||
@ -240,6 +224,7 @@ void RemoteBitrateEstimatorAbsSendTime::IncomingPacket(int64_t arrival_time_ms,
|
||||
size_t payload_size,
|
||||
const RTPHeader& header,
|
||||
bool was_paced) {
|
||||
RTC_DCHECK(network_thread_.CalledOnValidThread());
|
||||
if (!header.extension.hasAbsoluteSendTime) {
|
||||
LOG(LS_WARNING) << "RemoteBitrateEstimatorAbsSendTimeImpl: Incoming packet "
|
||||
"is missing absolute send time extension!";
|
||||
@ -261,13 +246,10 @@ void RemoteBitrateEstimatorAbsSendTime::IncomingPacketInfo(
|
||||
uint32_t timestamp = send_time_24bits << kAbsSendTimeInterArrivalUpshift;
|
||||
int64_t send_time_ms = static_cast<int64_t>(timestamp) * kTimestampToMs;
|
||||
|
||||
CriticalSectionScoped cs(crit_sect_.get());
|
||||
int64_t now_ms = clock_->TimeInMilliseconds();
|
||||
// TODO(holmer): SSRCs are only needed for REMB, should be broken out from
|
||||
// here.
|
||||
ssrcs_[ssrc] = now_ms;
|
||||
incoming_bitrate_.Update(payload_size, now_ms);
|
||||
const BandwidthUsage prior_state = detector_.State();
|
||||
|
||||
if (first_packet_time_ms_ == -1)
|
||||
first_packet_time_ms_ = clock_->TimeInMilliseconds();
|
||||
@ -279,79 +261,88 @@ void RemoteBitrateEstimatorAbsSendTime::IncomingPacketInfo(
|
||||
// make sure the packet was paced. We currently assume that only packets
|
||||
// larger than 200 bytes are paced by the sender.
|
||||
was_paced = was_paced && payload_size > PacedSender::kMinProbePacketSize;
|
||||
if (was_paced &&
|
||||
(!remote_rate_.ValidEstimate() ||
|
||||
now_ms - first_packet_time_ms_ < kInitialProbingIntervalMs)) {
|
||||
// TODO(holmer): Use a map instead to get correct order?
|
||||
if (total_probes_received_ < kMaxProbePackets) {
|
||||
int send_delta_ms = -1;
|
||||
int recv_delta_ms = -1;
|
||||
if (!probes_.empty()) {
|
||||
send_delta_ms = send_time_ms - probes_.back().send_time_ms;
|
||||
recv_delta_ms = arrival_time_ms - probes_.back().recv_time_ms;
|
||||
bool update_estimate = false;
|
||||
uint32_t target_bitrate_bps = 0;
|
||||
std::vector<uint32_t> ssrcs;
|
||||
{
|
||||
rtc::CritScope lock(&crit_);
|
||||
|
||||
TimeoutStreams(now_ms);
|
||||
ssrcs_[ssrc] = now_ms;
|
||||
|
||||
if (was_paced &&
|
||||
(!remote_rate_.ValidEstimate() ||
|
||||
now_ms - first_packet_time_ms_ < kInitialProbingIntervalMs)) {
|
||||
// TODO(holmer): Use a map instead to get correct order?
|
||||
if (total_probes_received_ < kMaxProbePackets) {
|
||||
int send_delta_ms = -1;
|
||||
int recv_delta_ms = -1;
|
||||
if (!probes_.empty()) {
|
||||
send_delta_ms = send_time_ms - probes_.back().send_time_ms;
|
||||
recv_delta_ms = arrival_time_ms - probes_.back().recv_time_ms;
|
||||
}
|
||||
LOG(LS_INFO) << "Probe packet received: send time=" << send_time_ms
|
||||
<< " ms, recv time=" << arrival_time_ms
|
||||
<< " ms, send delta=" << send_delta_ms
|
||||
<< " ms, recv delta=" << recv_delta_ms << " ms.";
|
||||
}
|
||||
LOG(LS_INFO) << "Probe packet received: send time=" << send_time_ms
|
||||
<< " ms, recv time=" << arrival_time_ms
|
||||
<< " ms, send delta=" << send_delta_ms
|
||||
<< " ms, recv delta=" << recv_delta_ms << " ms.";
|
||||
probes_.push_back(Probe(send_time_ms, arrival_time_ms, payload_size));
|
||||
++total_probes_received_;
|
||||
// Make sure that a probe which updated the bitrate immediately has an
|
||||
// effect by calling the OnReceiveBitrateChanged callback.
|
||||
if (ProcessClusters(now_ms) == ProbeResult::kBitrateUpdated)
|
||||
update_estimate = true;
|
||||
}
|
||||
probes_.push_back(Probe(send_time_ms, arrival_time_ms, payload_size));
|
||||
++total_probes_received_;
|
||||
ProcessClusters(now_ms);
|
||||
}
|
||||
if (!inter_arrival_.get()) {
|
||||
inter_arrival_.reset(
|
||||
new InterArrival((kTimestampGroupLengthMs << kInterArrivalShift) / 1000,
|
||||
kTimestampToMs, true));
|
||||
}
|
||||
if (inter_arrival_->ComputeDeltas(timestamp, arrival_time_ms, payload_size,
|
||||
&ts_delta, &t_delta, &size_delta)) {
|
||||
double ts_delta_ms = (1000.0 * ts_delta) / (1 << kInterArrivalShift);
|
||||
estimator_.Update(t_delta, ts_delta_ms, size_delta, detector_.State());
|
||||
detector_.Detect(estimator_.offset(), ts_delta_ms,
|
||||
estimator_.num_of_deltas(), arrival_time_ms);
|
||||
UpdateStats(static_cast<int>(t_delta - ts_delta_ms), now_ms);
|
||||
}
|
||||
if (detector_.State() == kBwOverusing) {
|
||||
uint32_t incoming_bitrate_bps = incoming_bitrate_.Rate(now_ms);
|
||||
if (prior_state != kBwOverusing ||
|
||||
remote_rate_.TimeToReduceFurther(now_ms, incoming_bitrate_bps)) {
|
||||
if (inter_arrival_->ComputeDeltas(timestamp, arrival_time_ms, payload_size,
|
||||
&ts_delta, &t_delta, &size_delta)) {
|
||||
double ts_delta_ms = (1000.0 * ts_delta) / (1 << kInterArrivalShift);
|
||||
estimator_.Update(t_delta, ts_delta_ms, size_delta, detector_.State());
|
||||
detector_.Detect(estimator_.offset(), ts_delta_ms,
|
||||
estimator_.num_of_deltas(), arrival_time_ms);
|
||||
}
|
||||
|
||||
if (!update_estimate) {
|
||||
// Check if it's time for a periodic update or if we should update because
|
||||
// of an over-use.
|
||||
if (last_update_ms_ == -1 ||
|
||||
now_ms - last_update_ms_ > remote_rate_.GetFeedbackInterval()) {
|
||||
update_estimate = true;
|
||||
} else if (detector_.State() == kBwOverusing &&
|
||||
remote_rate_.TimeToReduceFurther(
|
||||
now_ms, incoming_bitrate_.Rate(now_ms))) {
|
||||
update_estimate = true;
|
||||
}
|
||||
}
|
||||
|
||||
if (update_estimate) {
|
||||
// The first overuse should immediately trigger a new estimate.
|
||||
// We also have to update the estimate immediately if we are overusing
|
||||
// and the target bitrate is too high compared to what we are receiving.
|
||||
UpdateEstimate(now_ms);
|
||||
const RateControlInput input(detector_.State(),
|
||||
incoming_bitrate_.Rate(now_ms),
|
||||
estimator_.var_noise());
|
||||
remote_rate_.Update(&input, now_ms);
|
||||
target_bitrate_bps = remote_rate_.UpdateBandwidthEstimate(now_ms);
|
||||
update_estimate = remote_rate_.ValidEstimate();
|
||||
ssrcs = Keys(ssrcs_);
|
||||
}
|
||||
}
|
||||
if (update_estimate) {
|
||||
last_update_ms_ = now_ms;
|
||||
observer_->OnReceiveBitrateChanged(ssrcs, target_bitrate_bps);
|
||||
}
|
||||
}
|
||||
|
||||
int32_t RemoteBitrateEstimatorAbsSendTime::Process() {
|
||||
if (TimeUntilNextProcess() > 0) {
|
||||
return 0;
|
||||
}
|
||||
{
|
||||
CriticalSectionScoped cs(crit_sect_.get());
|
||||
UpdateEstimate(clock_->TimeInMilliseconds());
|
||||
}
|
||||
last_process_time_ = clock_->TimeInMilliseconds();
|
||||
return 0;
|
||||
}
|
||||
|
||||
int64_t RemoteBitrateEstimatorAbsSendTime::TimeUntilNextProcess() {
|
||||
if (last_process_time_ < 0) {
|
||||
return 0;
|
||||
}
|
||||
{
|
||||
CriticalSectionScoped cs(crit_sect_.get());
|
||||
return last_process_time_ + process_interval_ms_ -
|
||||
clock_->TimeInMilliseconds();
|
||||
}
|
||||
const int64_t kDisabledModuleTime = 1000;
|
||||
return kDisabledModuleTime;
|
||||
}
|
||||
|
||||
void RemoteBitrateEstimatorAbsSendTime::UpdateEstimate(int64_t now_ms) {
|
||||
if (!inter_arrival_.get()) {
|
||||
// No packets have been received on the active streams.
|
||||
return;
|
||||
}
|
||||
void RemoteBitrateEstimatorAbsSendTime::TimeoutStreams(int64_t now_ms) {
|
||||
for (Ssrcs::iterator it = ssrcs_.begin(); it != ssrcs_.end();) {
|
||||
if ((now_ms - it->second) > kStreamTimeOutMs) {
|
||||
ssrcs_.erase(it++);
|
||||
@ -361,40 +352,36 @@ void RemoteBitrateEstimatorAbsSendTime::UpdateEstimate(int64_t now_ms) {
|
||||
}
|
||||
if (ssrcs_.empty()) {
|
||||
// We can't update the estimate if we don't have any active streams.
|
||||
inter_arrival_.reset();
|
||||
inter_arrival_.reset(
|
||||
new InterArrival((kTimestampGroupLengthMs << kInterArrivalShift) / 1000,
|
||||
kTimestampToMs, true));
|
||||
// We deliberately don't reset the first_packet_time_ms_ here for now since
|
||||
// we only probe for bandwidth in the beginning of a call right now.
|
||||
return;
|
||||
}
|
||||
|
||||
const RateControlInput input(detector_.State(),
|
||||
incoming_bitrate_.Rate(now_ms),
|
||||
estimator_.var_noise());
|
||||
remote_rate_.Update(&input, now_ms);
|
||||
uint32_t target_bitrate = remote_rate_.UpdateBandwidthEstimate(now_ms);
|
||||
if (remote_rate_.ValidEstimate()) {
|
||||
process_interval_ms_ = remote_rate_.GetFeedbackInterval();
|
||||
observer_->OnReceiveBitrateChanged(Keys(ssrcs_), target_bitrate);
|
||||
}
|
||||
}
|
||||
|
||||
void RemoteBitrateEstimatorAbsSendTime::OnRttUpdate(int64_t avg_rtt_ms,
|
||||
int64_t max_rtt_ms) {
|
||||
CriticalSectionScoped cs(crit_sect_.get());
|
||||
RTC_DCHECK(process_thread_.CalledOnValidThread());
|
||||
rtc::CritScope lock(&crit_);
|
||||
remote_rate_.SetRtt(avg_rtt_ms);
|
||||
}
|
||||
|
||||
void RemoteBitrateEstimatorAbsSendTime::RemoveStream(uint32_t ssrc) {
|
||||
CriticalSectionScoped cs(crit_sect_.get());
|
||||
rtc::CritScope lock(&crit_);
|
||||
ssrcs_.erase(ssrc);
|
||||
}
|
||||
|
||||
bool RemoteBitrateEstimatorAbsSendTime::LatestEstimate(
|
||||
std::vector<uint32_t>* ssrcs,
|
||||
uint32_t* bitrate_bps) const {
|
||||
CriticalSectionScoped cs(crit_sect_.get());
|
||||
assert(ssrcs);
|
||||
assert(bitrate_bps);
|
||||
// Currently accessed from both the process thread (see
|
||||
// ModuleRtpRtcpImpl::Process()) and the configuration thread (see
|
||||
// Call::GetStats()). Should in the future only be accessed from a single
|
||||
// thread.
|
||||
RTC_DCHECK(ssrcs);
|
||||
RTC_DCHECK(bitrate_bps);
|
||||
rtc::CritScope lock(&crit_);
|
||||
if (!remote_rate_.ValidEstimate()) {
|
||||
return false;
|
||||
}
|
||||
@ -407,45 +394,10 @@ bool RemoteBitrateEstimatorAbsSendTime::LatestEstimate(
|
||||
return true;
|
||||
}
|
||||
|
||||
bool RemoteBitrateEstimatorAbsSendTime::GetStats(
|
||||
ReceiveBandwidthEstimatorStats* output) const {
|
||||
{
|
||||
CriticalSectionScoped cs(crit_sect_.get());
|
||||
output->recent_propagation_time_delta_ms = recent_propagation_delta_ms_;
|
||||
output->recent_arrival_time_ms = recent_update_time_ms_;
|
||||
output->total_propagation_time_delta_ms = total_propagation_delta_ms_;
|
||||
}
|
||||
RemoveStaleEntries(
|
||||
&output->recent_arrival_time_ms,
|
||||
&output->recent_propagation_time_delta_ms,
|
||||
clock_->TimeInMilliseconds() - kPropagationDeltaQueueMaxTimeMs);
|
||||
return true;
|
||||
}
|
||||
|
||||
void RemoteBitrateEstimatorAbsSendTime::UpdateStats(int propagation_delta_ms,
|
||||
int64_t now_ms) {
|
||||
// The caller must enter crit_sect_ before the call.
|
||||
|
||||
// Remove the oldest entry if the size limit is reached.
|
||||
if (recent_update_time_ms_.size() == kPropagationDeltaQueueMaxSize) {
|
||||
recent_update_time_ms_.erase(recent_update_time_ms_.begin());
|
||||
recent_propagation_delta_ms_.erase(recent_propagation_delta_ms_.begin());
|
||||
}
|
||||
|
||||
recent_propagation_delta_ms_.push_back(propagation_delta_ms);
|
||||
recent_update_time_ms_.push_back(now_ms);
|
||||
|
||||
RemoveStaleEntries(
|
||||
&recent_update_time_ms_,
|
||||
&recent_propagation_delta_ms_,
|
||||
now_ms - kPropagationDeltaQueueMaxTimeMs);
|
||||
|
||||
total_propagation_delta_ms_ =
|
||||
std::max(total_propagation_delta_ms_ + propagation_delta_ms, 0);
|
||||
}
|
||||
|
||||
void RemoteBitrateEstimatorAbsSendTime::SetMinBitrate(int min_bitrate_bps) {
|
||||
CriticalSectionScoped cs(crit_sect_.get());
|
||||
// Called from both the configuration thread and the network thread. Shouldn't
|
||||
// be called from the network thread in the future.
|
||||
rtc::CritScope lock(&crit_);
|
||||
remote_rate_.SetMinBitrate(min_bitrate_bps);
|
||||
}
|
||||
} // namespace webrtc
|
||||
|
||||
@ -16,7 +16,9 @@
|
||||
#include <vector>
|
||||
|
||||
#include "webrtc/base/checks.h"
|
||||
#include "webrtc/base/criticalsection.h"
|
||||
#include "webrtc/base/scoped_ptr.h"
|
||||
#include "webrtc/base/thread_checker.h"
|
||||
#include "webrtc/modules/remote_bitrate_estimator/aimd_rate_control.h"
|
||||
#include "webrtc/modules/remote_bitrate_estimator/include/remote_bitrate_estimator.h"
|
||||
#include "webrtc/modules/remote_bitrate_estimator/inter_arrival.h"
|
||||
@ -86,65 +88,54 @@ class RemoteBitrateEstimatorAbsSendTime : public RemoteBitrateEstimator {
|
||||
void RemoveStream(uint32_t ssrc) override;
|
||||
bool LatestEstimate(std::vector<uint32_t>* ssrcs,
|
||||
uint32_t* bitrate_bps) const override;
|
||||
bool GetStats(ReceiveBandwidthEstimatorStats* output) const override;
|
||||
void SetMinBitrate(int min_bitrate_bps) override;
|
||||
|
||||
private:
|
||||
typedef std::map<uint32_t, int64_t> Ssrcs;
|
||||
enum class ProbeResult { kBitrateUpdated, kNoUpdate };
|
||||
|
||||
static bool IsWithinClusterBounds(int send_delta_ms,
|
||||
const Cluster& cluster_aggregate);
|
||||
|
||||
static void AddCluster(std::list<Cluster>* clusters, Cluster* cluster);
|
||||
|
||||
int Id() const;
|
||||
|
||||
void IncomingPacketInfo(int64_t arrival_time_ms,
|
||||
uint32_t send_time_24bits,
|
||||
size_t payload_size,
|
||||
uint32_t ssrc,
|
||||
bool was_paced);
|
||||
|
||||
bool IsProbe(int64_t send_time_ms, int payload_size) const
|
||||
EXCLUSIVE_LOCKS_REQUIRED(crit_sect_.get());
|
||||
|
||||
// Triggers a new estimate calculation.
|
||||
void UpdateEstimate(int64_t now_ms)
|
||||
EXCLUSIVE_LOCKS_REQUIRED(crit_sect_.get());
|
||||
|
||||
void UpdateStats(int propagation_delta_ms, int64_t now_ms)
|
||||
EXCLUSIVE_LOCKS_REQUIRED(crit_sect_.get());
|
||||
|
||||
void ComputeClusters(std::list<Cluster>* clusters) const;
|
||||
|
||||
std::list<Cluster>::const_iterator FindBestProbe(
|
||||
const std::list<Cluster>& clusters) const
|
||||
EXCLUSIVE_LOCKS_REQUIRED(crit_sect_.get());
|
||||
const std::list<Cluster>& clusters) const;
|
||||
|
||||
void ProcessClusters(int64_t now_ms)
|
||||
EXCLUSIVE_LOCKS_REQUIRED(crit_sect_.get());
|
||||
// Returns true if a probe which changed the estimate was detected.
|
||||
ProbeResult ProcessClusters(int64_t now_ms) EXCLUSIVE_LOCKS_REQUIRED(&crit_);
|
||||
|
||||
bool IsBitrateImproving(int probe_bitrate_bps) const
|
||||
EXCLUSIVE_LOCKS_REQUIRED(crit_sect_.get());
|
||||
EXCLUSIVE_LOCKS_REQUIRED(&crit_);
|
||||
|
||||
rtc::scoped_ptr<CriticalSectionWrapper> crit_sect_;
|
||||
RemoteBitrateObserver* observer_ GUARDED_BY(crit_sect_.get());
|
||||
Clock* clock_;
|
||||
Ssrcs ssrcs_ GUARDED_BY(crit_sect_.get());
|
||||
rtc::scoped_ptr<InterArrival> inter_arrival_ GUARDED_BY(crit_sect_.get());
|
||||
OveruseEstimator estimator_ GUARDED_BY(crit_sect_.get());
|
||||
OveruseDetector detector_ GUARDED_BY(crit_sect_.get());
|
||||
RateStatistics incoming_bitrate_ GUARDED_BY(crit_sect_.get());
|
||||
AimdRateControl remote_rate_ GUARDED_BY(crit_sect_.get());
|
||||
int64_t last_process_time_;
|
||||
std::vector<int> recent_propagation_delta_ms_ GUARDED_BY(crit_sect_.get());
|
||||
std::vector<int64_t> recent_update_time_ms_ GUARDED_BY(crit_sect_.get());
|
||||
int64_t process_interval_ms_ GUARDED_BY(crit_sect_.get());
|
||||
int total_propagation_delta_ms_ GUARDED_BY(crit_sect_.get());
|
||||
void TimeoutStreams(int64_t now_ms) EXCLUSIVE_LOCKS_REQUIRED(&crit_);
|
||||
|
||||
rtc::ThreadChecker network_thread_;
|
||||
RemoteBitrateObserver* const observer_;
|
||||
rtc::scoped_ptr<InterArrival> inter_arrival_;
|
||||
OveruseEstimator estimator_;
|
||||
OveruseDetector detector_;
|
||||
RateStatistics incoming_bitrate_;
|
||||
std::vector<int> recent_propagation_delta_ms_;
|
||||
std::vector<int64_t> recent_update_time_ms_;
|
||||
std::list<Probe> probes_;
|
||||
size_t total_probes_received_;
|
||||
int64_t first_packet_time_ms_;
|
||||
int64_t last_update_ms_;
|
||||
|
||||
rtc::ThreadChecker process_thread_;
|
||||
rtc::CriticalSection crit_;
|
||||
Ssrcs ssrcs_ GUARDED_BY(&crit_);
|
||||
AimdRateControl remote_rate_ GUARDED_BY(&crit_);
|
||||
Clock* const clock_;
|
||||
|
||||
RTC_DISALLOW_IMPLICIT_CONSTRUCTORS(RemoteBitrateEstimatorAbsSendTime);
|
||||
};
|
||||
|
||||
@ -27,31 +27,31 @@ class RemoteBitrateEstimatorAbsSendTimeTest :
|
||||
};
|
||||
|
||||
TEST_F(RemoteBitrateEstimatorAbsSendTimeTest, InitialBehavior) {
|
||||
InitialBehaviorTestHelper(508017);
|
||||
InitialBehaviorTestHelper(674840);
|
||||
}
|
||||
|
||||
TEST_F(RemoteBitrateEstimatorAbsSendTimeTest, RateIncreaseReordering) {
|
||||
RateIncreaseReorderingTestHelper(506422);
|
||||
RateIncreaseReorderingTestHelper(674840);
|
||||
}
|
||||
|
||||
TEST_F(RemoteBitrateEstimatorAbsSendTimeTest, RateIncreaseRtpTimestamps) {
|
||||
RateIncreaseRtpTimestampsTestHelper(1240);
|
||||
RateIncreaseRtpTimestampsTestHelper(1232);
|
||||
}
|
||||
|
||||
TEST_F(RemoteBitrateEstimatorAbsSendTimeTest, CapacityDropOneStream) {
|
||||
CapacityDropTestHelper(1, false, 600);
|
||||
CapacityDropTestHelper(1, false, 567);
|
||||
}
|
||||
|
||||
TEST_F(RemoteBitrateEstimatorAbsSendTimeTest, CapacityDropOneStreamWrap) {
|
||||
CapacityDropTestHelper(1, true, 600);
|
||||
CapacityDropTestHelper(1, true, 567);
|
||||
}
|
||||
|
||||
TEST_F(RemoteBitrateEstimatorAbsSendTimeTest, CapacityDropTwoStreamsWrap) {
|
||||
CapacityDropTestHelper(2, true, 533);
|
||||
CapacityDropTestHelper(2, true, 600);
|
||||
}
|
||||
|
||||
TEST_F(RemoteBitrateEstimatorAbsSendTimeTest, CapacityDropThreeStreamsWrap) {
|
||||
CapacityDropTestHelper(3, true, 700);
|
||||
CapacityDropTestHelper(3, true, 567);
|
||||
}
|
||||
|
||||
TEST_F(RemoteBitrateEstimatorAbsSendTimeTest, CapacityDropThirteenStreamsWrap) {
|
||||
@ -59,21 +59,17 @@ TEST_F(RemoteBitrateEstimatorAbsSendTimeTest, CapacityDropThirteenStreamsWrap) {
|
||||
}
|
||||
|
||||
TEST_F(RemoteBitrateEstimatorAbsSendTimeTest, CapacityDropNineteenStreamsWrap) {
|
||||
CapacityDropTestHelper(19, true, 700);
|
||||
CapacityDropTestHelper(19, true, 633);
|
||||
}
|
||||
|
||||
TEST_F(RemoteBitrateEstimatorAbsSendTimeTest, CapacityDropThirtyStreamsWrap) {
|
||||
CapacityDropTestHelper(30, true, 700);
|
||||
CapacityDropTestHelper(30, true, 600);
|
||||
}
|
||||
|
||||
TEST_F(RemoteBitrateEstimatorAbsSendTimeTest, TestTimestampGrouping) {
|
||||
TestTimestampGroupingTestHelper();
|
||||
}
|
||||
|
||||
TEST_F(RemoteBitrateEstimatorAbsSendTimeTest, TestGetStats) {
|
||||
TestGetStatsHelper();
|
||||
}
|
||||
|
||||
TEST_F(RemoteBitrateEstimatorAbsSendTimeTest, TestShortTimeoutAndWrap) {
|
||||
// Simulate a client leaving and rejoining the call after 35 seconds. This
|
||||
// will make abs send time wrap, so if streams aren't timed out properly
|
||||
|
||||
@ -215,12 +215,6 @@ bool RemoteBitrateEstimatorSingleStream::LatestEstimate(
|
||||
return true;
|
||||
}
|
||||
|
||||
bool RemoteBitrateEstimatorSingleStream::GetStats(
|
||||
ReceiveBandwidthEstimatorStats* output) const {
|
||||
// Not implemented.
|
||||
return false;
|
||||
}
|
||||
|
||||
void RemoteBitrateEstimatorSingleStream::GetSsrcs(
|
||||
std::vector<uint32_t>* ssrcs) const {
|
||||
assert(ssrcs);
|
||||
|
||||
@ -37,7 +37,6 @@ class RemoteBitrateEstimatorSingleStream : public RemoteBitrateEstimator {
|
||||
void RemoveStream(uint32_t ssrc) override;
|
||||
bool LatestEstimate(std::vector<uint32_t>* ssrcs,
|
||||
uint32_t* bitrate_bps) const override;
|
||||
bool GetStats(ReceiveBandwidthEstimatorStats* output) const override;
|
||||
void SetMinBitrate(int min_bitrate_bps) override;
|
||||
|
||||
private:
|
||||
|
||||
@ -47,7 +47,7 @@ TEST_F(RemoteBitrateEstimatorSingleTest, CapacityDropOneStreamWrap) {
|
||||
}
|
||||
|
||||
TEST_F(RemoteBitrateEstimatorSingleTest, CapacityDropTwoStreamsWrap) {
|
||||
CapacityDropTestHelper(2, true, 700);
|
||||
CapacityDropTestHelper(2, true, 567);
|
||||
}
|
||||
|
||||
TEST_F(RemoteBitrateEstimatorSingleTest, CapacityDropThreeStreamsWrap) {
|
||||
|
||||
@ -257,10 +257,8 @@ bool RemoteBitrateEstimatorTest::GenerateAndProcessFrame(uint32_t ssrc,
|
||||
(packet->arrival_time + 500) / 1000, packet->rtp_timestamp,
|
||||
AbsSendTime(packet->send_time, 1000000), true);
|
||||
if (bitrate_observer_->updated()) {
|
||||
// Verify that new estimates only are triggered by an overuse and a
|
||||
// rate decrease.
|
||||
overuse = true;
|
||||
EXPECT_LE(bitrate_observer_->latest_bitrate(), bitrate_bps);
|
||||
if (bitrate_observer_->latest_bitrate() < bitrate_bps)
|
||||
overuse = true;
|
||||
}
|
||||
delete packet;
|
||||
packets.pop_front();
|
||||
@ -478,15 +476,11 @@ void RemoteBitrateEstimatorTest::CapacityDropTestHelper(
|
||||
int64_t bitrate_drop_time = -1;
|
||||
for (int i = 0; i < 100 * number_of_streams; ++i) {
|
||||
GenerateAndProcessFrame(kDefaultSsrc, bitrate_bps);
|
||||
// Check for either increase or decrease.
|
||||
if (bitrate_observer_->updated()) {
|
||||
if (bitrate_drop_time == -1 &&
|
||||
bitrate_observer_->latest_bitrate() <= kReducedCapacityBps) {
|
||||
bitrate_drop_time = clock_.TimeInMilliseconds();
|
||||
}
|
||||
bitrate_bps = bitrate_observer_->latest_bitrate();
|
||||
bitrate_observer_->Reset();
|
||||
if (bitrate_drop_time == -1 &&
|
||||
bitrate_observer_->latest_bitrate() <= kReducedCapacityBps) {
|
||||
bitrate_drop_time = clock_.TimeInMilliseconds();
|
||||
}
|
||||
bitrate_bps = bitrate_observer_->latest_bitrate();
|
||||
}
|
||||
|
||||
EXPECT_NEAR(expected_bitrate_drop_delta,
|
||||
@ -563,67 +557,6 @@ void RemoteBitrateEstimatorTest::TestTimestampGroupingTestHelper() {
|
||||
EXPECT_LT(bitrate_observer_->latest_bitrate(), 400000u);
|
||||
}
|
||||
|
||||
void RemoteBitrateEstimatorTest::TestGetStatsHelper() {
|
||||
const int kFramerate = 100;
|
||||
const int kFrameIntervalMs = 1000 / kFramerate;
|
||||
const int kBurstThresholdMs = 5;
|
||||
const uint32_t kFrameIntervalAbsSendTime = AbsSendTime(1, kFramerate);
|
||||
uint32_t timestamp = 0;
|
||||
// Initialize absolute_send_time (24 bits) so that it will definitely wrap
|
||||
// during the test.
|
||||
uint32_t absolute_send_time =
|
||||
AddAbsSendTime((1 << 24),
|
||||
-(50 * static_cast<int>(kFrameIntervalAbsSendTime)));
|
||||
|
||||
// Inject propagation_time_delta of kFrameIntervalMs.
|
||||
for (size_t i = 0; i < 3; ++i) {
|
||||
IncomingPacket(kDefaultSsrc, 1000, clock_.TimeInMilliseconds(), timestamp,
|
||||
absolute_send_time, true);
|
||||
timestamp += kFrameIntervalMs;
|
||||
// Insert a kFrameIntervalMs propagation_time_delta.
|
||||
clock_.AdvanceTimeMilliseconds(kFrameIntervalMs * 2);
|
||||
absolute_send_time = AddAbsSendTime(absolute_send_time,
|
||||
kFrameIntervalAbsSendTime);
|
||||
}
|
||||
ReceiveBandwidthEstimatorStats stats;
|
||||
EXPECT_TRUE(bitrate_estimator_->GetStats(&stats));
|
||||
EXPECT_EQ(1U, stats.recent_propagation_time_delta_ms.size());
|
||||
EXPECT_EQ(kFrameIntervalMs, stats.recent_propagation_time_delta_ms[0]);
|
||||
EXPECT_EQ(1U, stats.recent_arrival_time_ms.size());
|
||||
EXPECT_EQ(kFrameIntervalMs, stats.total_propagation_time_delta_ms);
|
||||
|
||||
// Inject negative propagation_time_deltas. The total propagation_time_delta
|
||||
// should be adjusted to 0.
|
||||
for (size_t i = 0; i < 3; ++i) {
|
||||
IncomingPacket(kDefaultSsrc, 1000, clock_.TimeInMilliseconds(), timestamp,
|
||||
absolute_send_time, true);
|
||||
timestamp += 10 * kFrameIntervalMs;
|
||||
clock_.AdvanceTimeMilliseconds(kBurstThresholdMs + 1);
|
||||
absolute_send_time = AddAbsSendTime(absolute_send_time,
|
||||
10 * kFrameIntervalAbsSendTime);
|
||||
}
|
||||
EXPECT_TRUE(bitrate_estimator_->GetStats(&stats));
|
||||
EXPECT_EQ(0, stats.total_propagation_time_delta_ms);
|
||||
|
||||
// Send more than 1000 frames and make sure the stats queues stays within
|
||||
// limits.
|
||||
for (size_t i = 0; i < 1001; ++i) {
|
||||
IncomingPacket(kDefaultSsrc, 1000, clock_.TimeInMilliseconds(), timestamp,
|
||||
absolute_send_time, true);
|
||||
timestamp += kFrameIntervalMs;
|
||||
absolute_send_time = AddAbsSendTime(absolute_send_time,
|
||||
kFrameIntervalAbsSendTime);
|
||||
}
|
||||
EXPECT_TRUE(bitrate_estimator_->GetStats(&stats));
|
||||
EXPECT_LE(stats.recent_propagation_time_delta_ms.size(), 1000U);
|
||||
EXPECT_LE(stats.recent_arrival_time_ms.size(), 1000U);
|
||||
|
||||
// Move the clock over the 1000ms limit.
|
||||
clock_.AdvanceTimeMilliseconds(2000);
|
||||
EXPECT_TRUE(bitrate_estimator_->GetStats(&stats));
|
||||
EXPECT_EQ(0U, stats.recent_propagation_time_delta_ms.size());
|
||||
}
|
||||
|
||||
void RemoteBitrateEstimatorTest::TestWrappingHelper(
|
||||
int silence_time_s) {
|
||||
const int kFramerate = 100;
|
||||
|
||||
@ -197,8 +197,6 @@ class RemoteBitrateEstimatorTest : public ::testing::Test {
|
||||
|
||||
void TestTimestampGroupingTestHelper();
|
||||
|
||||
void TestGetStatsHelper();
|
||||
|
||||
void TestWrappingHelper(int silence_time_s);
|
||||
|
||||
void InitialBehaviorTestHelper(uint32_t expected_converge_bitrate);
|
||||
|
||||
@ -52,22 +52,15 @@ void RemoteEstimatorProxy::IncomingPacket(int64_t arrival_time_ms,
|
||||
}
|
||||
rtc::CritScope cs(&lock_);
|
||||
media_ssrc_ = header.ssrc;
|
||||
|
||||
OnPacketArrival(header.extension.transportSequenceNumber, arrival_time_ms);
|
||||
}
|
||||
|
||||
void RemoteEstimatorProxy::RemoveStream(unsigned int ssrc) {}
|
||||
|
||||
bool RemoteEstimatorProxy::LatestEstimate(std::vector<unsigned int>* ssrcs,
|
||||
unsigned int* bitrate_bps) const {
|
||||
return false;
|
||||
}
|
||||
|
||||
bool RemoteEstimatorProxy::GetStats(
|
||||
ReceiveBandwidthEstimatorStats* output) const {
|
||||
return false;
|
||||
}
|
||||
|
||||
|
||||
int64_t RemoteEstimatorProxy::TimeUntilNextProcess() {
|
||||
int64_t now = clock_->TimeInMilliseconds();
|
||||
int64_t time_until_next = 0;
|
||||
@ -118,6 +111,7 @@ void RemoteEstimatorProxy::OnPacketArrival(uint16_t sequence_number,
|
||||
}
|
||||
|
||||
RTC_DCHECK(packet_arrival_times_.end() == packet_arrival_times_.find(seq));
|
||||
|
||||
packet_arrival_times_[seq] = arrival_time;
|
||||
}
|
||||
|
||||
|
||||
@ -41,10 +41,9 @@ class RemoteEstimatorProxy : public RemoteBitrateEstimator {
|
||||
size_t payload_size,
|
||||
const RTPHeader& header,
|
||||
bool was_paced) override;
|
||||
void RemoveStream(unsigned int ssrc) override;
|
||||
void RemoveStream(uint32_t ssrc) override {}
|
||||
bool LatestEstimate(std::vector<unsigned int>* ssrcs,
|
||||
unsigned int* bitrate_bps) const override;
|
||||
bool GetStats(ReceiveBandwidthEstimatorStats* output) const override;
|
||||
void OnRttUpdate(int64_t avg_rtt_ms, int64_t max_rtt_ms) override {}
|
||||
void SetMinBitrate(int min_bitrate_bps) override {}
|
||||
int64_t TimeUntilNextProcess() override;
|
||||
|
||||
Loading…
x
Reference in New Issue
Block a user