Revert "Create a VsyncSchedule per display"
This reverts commit 31d41415101ff3483ce1cc5a9c2ef322490a05bd.
Conflicts:
services/surfaceflinger/Scheduler/EventThread.cpp
services/surfaceflinger/SurfaceFlinger.cpp
Bug: 267562341
Test: ARC Regression Dashboard
Change-Id: I0757a7df540fad316b2db42e4c77f1c73bc49420
diff --git a/services/surfaceflinger/Scheduler/EventThread.cpp b/services/surfaceflinger/Scheduler/EventThread.cpp
index 5e79a5c..eb6d7e4 100644
--- a/services/surfaceflinger/Scheduler/EventThread.cpp
+++ b/services/surfaceflinger/Scheduler/EventThread.cpp
@@ -238,19 +238,29 @@
namespace impl {
-EventThread::EventThread(const char* name, std::shared_ptr<scheduler::VsyncSchedule> vsyncSchedule,
- IEventThreadCallback& eventThreadCallback,
+EventThread::EventThread(const char* name, scheduler::VsyncSchedule& vsyncSchedule,
android::frametimeline::TokenManager* tokenManager,
+ ThrottleVsyncCallback throttleVsyncCallback,
+ GetVsyncPeriodFunction getVsyncPeriodFunction,
std::chrono::nanoseconds workDuration,
std::chrono::nanoseconds readyDuration)
: mThreadName(name),
mVsyncTracer(base::StringPrintf("VSYNC-%s", name), 0),
mWorkDuration(base::StringPrintf("VsyncWorkDuration-%s", name), workDuration),
mReadyDuration(readyDuration),
- mVsyncSchedule(std::move(vsyncSchedule)),
- mVsyncRegistration(mVsyncSchedule->getDispatch(), createDispatchCallback(), name),
+ mVsyncSchedule(vsyncSchedule),
+ mVsyncRegistration(
+ vsyncSchedule.getDispatch(),
+ [this](nsecs_t vsyncTime, nsecs_t wakeupTime, nsecs_t readyTime) {
+ onVsync(vsyncTime, wakeupTime, readyTime);
+ },
+ name),
mTokenManager(tokenManager),
- mEventThreadCallback(eventThreadCallback) {
+ mThrottleVsyncCallback(std::move(throttleVsyncCallback)),
+ mGetVsyncPeriodFunction(std::move(getVsyncPeriodFunction)) {
+ LOG_ALWAYS_FATAL_IF(getVsyncPeriodFunction == nullptr,
+ "getVsyncPeriodFunction must not be null");
+
mThread = std::thread([this]() NO_THREAD_SAFETY_ANALYSIS {
std::unique_lock<std::mutex> lock(mMutex);
threadMain(lock);
@@ -361,16 +371,16 @@
}
VsyncEventData vsyncEventData;
- const Fps frameInterval = mEventThreadCallback.getLeaderRenderFrameRate(connection->mOwnerUid);
- vsyncEventData.frameInterval = frameInterval.getPeriodNsecs();
+ nsecs_t frameInterval = mGetVsyncPeriodFunction(connection->mOwnerUid);
+ vsyncEventData.frameInterval = frameInterval;
const auto [presentTime, deadline] = [&]() -> std::pair<nsecs_t, nsecs_t> {
std::lock_guard<std::mutex> lock(mMutex);
- const auto vsyncTime = mVsyncSchedule->getTracker().nextAnticipatedVSyncTimeFrom(
+ const auto vsyncTime = mVsyncSchedule.getTracker().nextAnticipatedVSyncTimeFrom(
systemTime() + mWorkDuration.get().count() + mReadyDuration.count());
return {vsyncTime, vsyncTime - mReadyDuration.count()};
}();
- generateFrameTimeline(vsyncEventData, frameInterval.getPeriodNsecs(),
- systemTime(SYSTEM_TIME_MONOTONIC), presentTime, deadline);
+ generateFrameTimeline(vsyncEventData, frameInterval, systemTime(SYSTEM_TIME_MONOTONIC),
+ presentTime, deadline);
return vsyncEventData;
}
@@ -533,15 +543,14 @@
const auto throttleVsync = [&] {
const auto& vsyncData = event.vsync.vsyncData;
if (connection->frameRate.isValid()) {
- return !mVsyncSchedule->getTracker()
+ return !mVsyncSchedule.getTracker()
.isVSyncInPhase(vsyncData.preferredExpectedPresentationTime(),
connection->frameRate);
}
- const auto expectedPresentTime =
- TimePoint::fromNs(vsyncData.preferredExpectedPresentationTime());
- return !mEventThreadCallback.isVsyncTargetForUid(expectedPresentTime,
- connection->mOwnerUid);
+ return mThrottleVsyncCallback &&
+ mThrottleVsyncCallback(event.vsync.vsyncData.preferredExpectedPresentationTime(),
+ connection->mOwnerUid);
};
switch (event.header.type) {
@@ -629,11 +638,9 @@
for (const auto& consumer : consumers) {
DisplayEventReceiver::Event copy = event;
if (event.header.type == DisplayEventReceiver::DISPLAY_EVENT_VSYNC) {
- const Fps frameInterval =
- mEventThreadCallback.getLeaderRenderFrameRate(consumer->mOwnerUid);
- copy.vsync.vsyncData.frameInterval = frameInterval.getPeriodNsecs();
- generateFrameTimeline(copy.vsync.vsyncData, frameInterval.getPeriodNsecs(),
- copy.header.timestamp,
+ const int64_t frameInterval = mGetVsyncPeriodFunction(consumer->mOwnerUid);
+ copy.vsync.vsyncData.frameInterval = frameInterval;
+ generateFrameTimeline(copy.vsync.vsyncData, frameInterval, copy.header.timestamp,
event.vsync.vsyncData.preferredExpectedPresentationTime(),
event.vsync.vsyncData.preferredDeadlineTimestamp());
}
@@ -699,26 +706,6 @@
}
}
-void EventThread::onNewVsyncSchedule(std::shared_ptr<scheduler::VsyncSchedule> schedule) {
- std::lock_guard<std::mutex> lock(mMutex);
- const bool reschedule = mVsyncRegistration.cancel() == scheduler::CancelResult::Cancelled;
- mVsyncSchedule = std::move(schedule);
- mVsyncRegistration =
- scheduler::VSyncCallbackRegistration(mVsyncSchedule->getDispatch(),
- createDispatchCallback(), mThreadName);
- if (reschedule) {
- mVsyncRegistration.schedule({.workDuration = mWorkDuration.get().count(),
- .readyDuration = mReadyDuration.count(),
- .earliestVsync = mLastVsyncCallbackTime.ns()});
- }
-}
-
-scheduler::VSyncDispatch::Callback EventThread::createDispatchCallback() {
- return [this](nsecs_t vsyncTime, nsecs_t wakeupTime, nsecs_t readyTime) {
- onVsync(vsyncTime, wakeupTime, readyTime);
- };
-}
-
} // namespace impl
} // namespace android
diff --git a/services/surfaceflinger/Scheduler/EventThread.h b/services/surfaceflinger/Scheduler/EventThread.h
index aa27091..347dc4a 100644
--- a/services/surfaceflinger/Scheduler/EventThread.h
+++ b/services/surfaceflinger/Scheduler/EventThread.h
@@ -23,7 +23,6 @@
#include <sys/types.h>
#include <utils/Errors.h>
-#include <scheduler/Fps.h>
#include <scheduler/FrameRateMode.h>
#include <condition_variable>
#include <cstdint>
@@ -68,15 +67,6 @@
// Subsequent values are periods.
};
-class IEventThreadCallback {
-public:
- virtual ~IEventThreadCallback() = default;
-
- virtual bool isVsyncTargetForUid(TimePoint expectedVsyncTime, uid_t uid) const = 0;
-
- virtual Fps getLeaderRenderFrameRate(uid_t uid) const = 0;
-};
-
class EventThreadConnection : public gui::BnDisplayEventConnection {
public:
EventThreadConnection(EventThread*, uid_t callingUid, ResyncCallback,
@@ -146,17 +136,18 @@
// Retrieves the number of event connections tracked by this EventThread.
virtual size_t getEventThreadConnectionCount() = 0;
-
- virtual void onNewVsyncSchedule(std::shared_ptr<scheduler::VsyncSchedule>) = 0;
};
namespace impl {
class EventThread : public android::EventThread {
public:
- EventThread(const char* name, std::shared_ptr<scheduler::VsyncSchedule>, IEventThreadCallback&,
- frametimeline::TokenManager*, std::chrono::nanoseconds workDuration,
- std::chrono::nanoseconds readyDuration);
+ using ThrottleVsyncCallback = std::function<bool(nsecs_t, uid_t)>;
+ using GetVsyncPeriodFunction = std::function<nsecs_t(uid_t)>;
+
+ EventThread(const char* name, scheduler::VsyncSchedule&, frametimeline::TokenManager*,
+ ThrottleVsyncCallback, GetVsyncPeriodFunction,
+ std::chrono::nanoseconds workDuration, std::chrono::nanoseconds readyDuration);
~EventThread();
sp<EventThreadConnection> createEventConnection(
@@ -188,8 +179,6 @@
size_t getEventThreadConnectionCount() override;
- void onNewVsyncSchedule(std::shared_ptr<scheduler::VsyncSchedule>) override;
-
private:
friend EventThreadTest;
@@ -213,19 +202,17 @@
nsecs_t timestamp, nsecs_t preferredExpectedPresentationTime,
nsecs_t preferredDeadlineTimestamp) const;
- scheduler::VSyncDispatch::Callback createDispatchCallback();
-
const char* const mThreadName;
TracedOrdinal<int> mVsyncTracer;
TracedOrdinal<std::chrono::nanoseconds> mWorkDuration GUARDED_BY(mMutex);
std::chrono::nanoseconds mReadyDuration GUARDED_BY(mMutex);
- std::shared_ptr<scheduler::VsyncSchedule> mVsyncSchedule;
+ scheduler::VsyncSchedule& mVsyncSchedule;
TimePoint mLastVsyncCallbackTime GUARDED_BY(mMutex) = TimePoint::now();
scheduler::VSyncCallbackRegistration mVsyncRegistration GUARDED_BY(mMutex);
frametimeline::TokenManager* const mTokenManager;
- // mEventThreadCallback will outlive the EventThread.
- IEventThreadCallback& mEventThreadCallback;
+ const ThrottleVsyncCallback mThrottleVsyncCallback;
+ const GetVsyncPeriodFunction mGetVsyncPeriodFunction;
std::thread mThread;
mutable std::mutex mMutex;
diff --git a/services/surfaceflinger/Scheduler/ISchedulerCallback.h b/services/surfaceflinger/Scheduler/ISchedulerCallback.h
deleted file mode 100644
index 92c2189..0000000
--- a/services/surfaceflinger/Scheduler/ISchedulerCallback.h
+++ /dev/null
@@ -1,37 +0,0 @@
-/*
- * Copyright 2023 The Android Open Source Project
- *
- * Licensed under the Apache License, Version 2.0 (the "License");
- * you may not use this file except in compliance with the License.
- * You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-#pragma once
-
-#include <vector>
-
-#include <ui/DisplayId.h>
-
-#include "Display/DisplayModeRequest.h"
-
-namespace android::scheduler {
-
-struct ISchedulerCallback {
- virtual void setVsyncEnabled(PhysicalDisplayId, bool) = 0;
- virtual void requestDisplayModes(std::vector<display::DisplayModeRequest>) = 0;
- virtual void kernelTimerChanged(bool expired) = 0;
- virtual void triggerOnFrameRateOverridesChanged() = 0;
-
-protected:
- ~ISchedulerCallback() = default;
-};
-
-} // namespace android::scheduler
diff --git a/services/surfaceflinger/Scheduler/MessageQueue.cpp b/services/surfaceflinger/Scheduler/MessageQueue.cpp
index 925f739..dec8f59 100644
--- a/services/surfaceflinger/Scheduler/MessageQueue.cpp
+++ b/services/surfaceflinger/Scheduler/MessageQueue.cpp
@@ -75,37 +75,19 @@
mHandler->dispatchFrame(vsyncId, expectedVsyncTime);
}
-void MessageQueue::initVsync(std::shared_ptr<scheduler::VSyncDispatch> dispatch,
+void MessageQueue::initVsync(scheduler::VSyncDispatch& dispatch,
frametimeline::TokenManager& tokenManager,
std::chrono::nanoseconds workDuration) {
std::lock_guard lock(mVsync.mutex);
mVsync.workDuration = workDuration;
mVsync.tokenManager = &tokenManager;
- updateVsyncRegistrationLocked(std::move(dispatch));
-}
-
-void MessageQueue::updateVsyncRegistration(std::shared_ptr<scheduler::VSyncDispatch> dispatch) {
- std::lock_guard lock(mVsync.mutex);
- updateVsyncRegistrationLocked(std::move(dispatch));
-}
-
-void MessageQueue::updateVsyncRegistrationLocked(
- std::shared_ptr<scheduler::VSyncDispatch> dispatch) {
- const bool reschedule = mVsync.registration &&
- mVsync.registration->cancel() == scheduler::CancelResult::Cancelled;
mVsync.registration = std::make_unique<
- scheduler::VSyncCallbackRegistration>(std::move(dispatch),
+ scheduler::VSyncCallbackRegistration>(dispatch,
std::bind(&MessageQueue::vsyncCallback, this,
std::placeholders::_1,
std::placeholders::_2,
std::placeholders::_3),
"sf");
- if (reschedule) {
- mVsync.scheduledFrameTime =
- mVsync.registration->schedule({.workDuration = mVsync.workDuration.get().count(),
- .readyDuration = 0,
- .earliestVsync = mVsync.lastCallbackTime.ns()});
- }
}
void MessageQueue::destroyVsync() {
diff --git a/services/surfaceflinger/Scheduler/MessageQueue.h b/services/surfaceflinger/Scheduler/MessageQueue.h
index ecb237d..0d59337 100644
--- a/services/surfaceflinger/Scheduler/MessageQueue.h
+++ b/services/surfaceflinger/Scheduler/MessageQueue.h
@@ -65,7 +65,7 @@
public:
virtual ~MessageQueue() = default;
- virtual void initVsync(std::shared_ptr<scheduler::VSyncDispatch>, frametimeline::TokenManager&,
+ virtual void initVsync(scheduler::VSyncDispatch&, frametimeline::TokenManager&,
std::chrono::nanoseconds workDuration) = 0;
virtual void destroyVsync() = 0;
virtual void setDuration(std::chrono::nanoseconds workDuration) = 0;
@@ -106,8 +106,6 @@
void vsyncCallback(nsecs_t vsyncTime, nsecs_t targetWakeupTime, nsecs_t readyTime);
- void updateVsyncRegistration(std::shared_ptr<scheduler::VSyncDispatch>) EXCLUDES(mVsync.mutex);
-
private:
virtual void onFrameSignal(ICompositor&, VsyncId, TimePoint expectedVsyncTime) = 0;
@@ -129,13 +127,10 @@
Vsync mVsync;
- void updateVsyncRegistrationLocked(std::shared_ptr<scheduler::VSyncDispatch>)
- REQUIRES(mVsync.mutex);
-
public:
explicit MessageQueue(ICompositor&);
- void initVsync(std::shared_ptr<scheduler::VSyncDispatch>, frametimeline::TokenManager&,
+ void initVsync(scheduler::VSyncDispatch&, frametimeline::TokenManager&,
std::chrono::nanoseconds workDuration) override;
void destroyVsync() override;
void setDuration(std::chrono::nanoseconds workDuration) override;
diff --git a/services/surfaceflinger/Scheduler/OneShotTimer.h b/services/surfaceflinger/Scheduler/OneShotTimer.h
index 02e8719..f95646c 100644
--- a/services/surfaceflinger/Scheduler/OneShotTimer.h
+++ b/services/surfaceflinger/Scheduler/OneShotTimer.h
@@ -40,7 +40,7 @@
OneShotTimer(std::string name, const Interval& interval, const ResetCallback& resetCallback,
const TimeoutCallback& timeoutCallback,
- std::unique_ptr<android::Clock> clock = std::make_unique<SteadyClock>());
+ std::unique_ptr<Clock> clock = std::make_unique<SteadyClock>());
~OneShotTimer();
Duration interval() const { return mInterval; }
@@ -82,7 +82,7 @@
std::thread mThread;
// Clock object for the timer. Mocked in unit tests.
- std::unique_ptr<android::Clock> mClock;
+ std::unique_ptr<Clock> mClock;
// Semaphore to keep mThread synchronized.
sem_t mSemaphore;
diff --git a/services/surfaceflinger/Scheduler/Scheduler.cpp b/services/surfaceflinger/Scheduler/Scheduler.cpp
index eed57ef..c314b5c 100644
--- a/services/surfaceflinger/Scheduler/Scheduler.cpp
+++ b/services/surfaceflinger/Scheduler/Scheduler.cpp
@@ -114,18 +114,10 @@
}
void Scheduler::registerDisplay(PhysicalDisplayId displayId, RefreshRateSelectorPtr selectorPtr) {
- registerDisplayInternal(displayId, std::move(selectorPtr),
- std::make_shared<VsyncSchedule>(displayId, mFeatures));
-}
-
-void Scheduler::registerDisplayInternal(PhysicalDisplayId displayId,
- RefreshRateSelectorPtr selectorPtr,
- std::shared_ptr<VsyncSchedule> vsyncSchedule) {
demoteLeaderDisplay();
std::scoped_lock lock(mDisplayLock);
mRefreshRateSelectors.emplace_or_replace(displayId, std::move(selectorPtr));
- mVsyncSchedules.emplace_or_replace(displayId, std::move(vsyncSchedule));
promoteLeaderDisplay();
}
@@ -135,7 +127,6 @@
std::scoped_lock lock(mDisplayLock);
mRefreshRateSelectors.erase(displayId);
- mVsyncSchedules.erase(displayId);
// Do not allow removing the final display. Code in the scheduler expects
// there to be at least one display. (This may be relaxed in the future with
@@ -163,49 +154,52 @@
compositor.sample();
}
-std::optional<Fps> Scheduler::getFrameRateOverride(uid_t uid) const {
- std::scoped_lock lock(mDisplayLock);
- return getFrameRateOverrideLocked(uid);
+void Scheduler::createVsyncSchedule(FeatureFlags features) {
+ mVsyncSchedule.emplace(features);
}
-std::optional<Fps> Scheduler::getFrameRateOverrideLocked(uid_t uid) const {
+std::optional<Fps> Scheduler::getFrameRateOverride(uid_t uid) const {
const bool supportsFrameRateOverrideByContent =
- leaderSelectorPtrLocked()->supportsAppFrameRateOverrideByContent();
+ leaderSelectorPtr()->supportsAppFrameRateOverrideByContent();
return mFrameRateOverrideMappings
.getFrameRateOverrideForUid(uid, supportsFrameRateOverrideByContent);
}
-bool Scheduler::isVsyncTargetForUid(TimePoint expectedVsyncTime, uid_t uid) const {
+bool Scheduler::isVsyncValid(TimePoint expectedVsyncTimestamp, uid_t uid) const {
const auto frameRate = getFrameRateOverride(uid);
if (!frameRate.has_value()) {
return true;
}
- return isVsyncInPhase(expectedVsyncTime, *frameRate);
+ return mVsyncSchedule->getTracker().isVSyncInPhase(expectedVsyncTimestamp.ns(), *frameRate);
}
-bool Scheduler::isVsyncInPhase(TimePoint expectedVsyncTime, Fps frameRate) const {
- return getVsyncSchedule()->getTracker().isVSyncInPhase(expectedVsyncTime.ns(), frameRate);
+bool Scheduler::isVsyncInPhase(TimePoint timePoint, const Fps frameRate) const {
+ return mVsyncSchedule->getTracker().isVSyncInPhase(timePoint.ns(), frameRate);
}
-Fps Scheduler::getLeaderRenderFrameRate(uid_t uid) const {
- std::scoped_lock lock(mDisplayLock);
- ftl::FakeGuard guard(kMainThreadContext);
- auto vsyncSchedule = getVsyncScheduleLocked();
+impl::EventThread::ThrottleVsyncCallback Scheduler::makeThrottleVsyncCallback() const {
+ return [this](nsecs_t expectedVsyncTimestamp, uid_t uid) {
+ return !isVsyncValid(TimePoint::fromNs(expectedVsyncTimestamp), uid);
+ };
+}
- const Fps refreshRate = leaderSelectorPtrLocked()->getActiveMode().fps;
- const nsecs_t currentPeriod = vsyncSchedule->period().ns() ?: refreshRate.getPeriodNsecs();
+impl::EventThread::GetVsyncPeriodFunction Scheduler::makeGetVsyncPeriodFunction() const {
+ return [this](uid_t uid) {
+ const Fps refreshRate = leaderSelectorPtr()->getActiveMode().fps;
+ const nsecs_t currentPeriod = mVsyncSchedule->period().ns() ?: refreshRate.getPeriodNsecs();
- const auto frameRate = getFrameRateOverrideLocked(uid);
- if (!frameRate.has_value()) {
- return Fps::fromPeriodNsecs(currentPeriod);
- }
+ const auto frameRate = getFrameRateOverride(uid);
+ if (!frameRate.has_value()) {
+ return currentPeriod;
+ }
- const auto divisor = RefreshRateSelector::getFrameRateDivisor(refreshRate, *frameRate);
- if (divisor <= 1) {
- return Fps::fromPeriodNsecs(currentPeriod);
- }
- return Fps::fromPeriodNsecs(currentPeriod * divisor);
+ const auto divisor = RefreshRateSelector::getFrameRateDivisor(refreshRate, *frameRate);
+ if (divisor <= 1) {
+ return currentPeriod;
+ }
+ return currentPeriod * divisor;
+ };
}
ConnectionHandle Scheduler::createEventThread(Cycle cycle,
@@ -213,7 +207,9 @@
std::chrono::nanoseconds workDuration,
std::chrono::nanoseconds readyDuration) {
auto eventThread = std::make_unique<impl::EventThread>(cycle == Cycle::Render ? "app" : "appSf",
- getVsyncSchedule(), *this, tokenManager,
+ *mVsyncSchedule, tokenManager,
+ makeThrottleVsyncCallback(),
+ makeGetVsyncPeriodFunction(),
workDuration, readyDuration);
auto& handle = cycle == Cycle::Render ? mAppConnectionHandle : mSfConnectionHandle;
@@ -276,6 +272,7 @@
thread = mConnections[handle].thread.get();
}
thread->onScreenAcquired();
+ mScreenAcquired = true;
}
void Scheduler::onScreenReleased(ConnectionHandle handle) {
@@ -286,6 +283,7 @@
thread = mConnections[handle].thread.get();
}
thread->onScreenReleased();
+ mScreenAcquired = false;
}
void Scheduler::onFrameRateOverridesChanged(ConnectionHandle handle, PhysicalDisplayId displayId) {
@@ -396,57 +394,48 @@
setDuration(config.sfWorkDuration);
}
-void Scheduler::enableHardwareVsync(PhysicalDisplayId id) {
- auto schedule = getVsyncSchedule(id);
- schedule->enableHardwareVsync(mSchedulerCallback);
-}
-
-void Scheduler::disableHardwareVsync(PhysicalDisplayId id, bool disallow) {
- auto schedule = getVsyncSchedule(id);
- schedule->disableHardwareVsync(mSchedulerCallback, disallow);
-}
-
-void Scheduler::resyncAllToHardwareVsync(bool allowToEnable) {
- std::scoped_lock lock(mDisplayLock);
- ftl::FakeGuard guard(kMainThreadContext);
-
- for (const auto& [id, _] : mRefreshRateSelectors) {
- resyncToHardwareVsyncLocked(id, allowToEnable);
+void Scheduler::enableHardwareVsync() {
+ std::lock_guard<std::mutex> lock(mHWVsyncLock);
+ if (!mPrimaryHWVsyncEnabled && mHWVsyncAvailable) {
+ mVsyncSchedule->getTracker().resetModel();
+ mSchedulerCallback.setVsyncEnabled(true);
+ mPrimaryHWVsyncEnabled = true;
}
}
-void Scheduler::resyncToHardwareVsyncLocked(PhysicalDisplayId id, bool allowToEnable,
- std::optional<Fps> refreshRate) {
- if (!refreshRate) {
- auto selectorPtr = mRefreshRateSelectors.get(id);
- LOG_ALWAYS_FATAL_IF(!selectorPtr);
- refreshRate = selectorPtr->get()->getActiveMode().modePtr->getFps();
+void Scheduler::disableHardwareVsync(bool makeUnavailable) {
+ std::lock_guard<std::mutex> lock(mHWVsyncLock);
+ if (mPrimaryHWVsyncEnabled) {
+ mSchedulerCallback.setVsyncEnabled(false);
+ mPrimaryHWVsyncEnabled = false;
}
- auto schedule = getVsyncScheduleLocked(id);
- if (allowToEnable) {
- schedule->allowHardwareVsync();
- } else if (!schedule->isHardwareVsyncAllowed()) {
- // Hardware vsync is not currently allowed, so abort the resync
- // attempt for now.
- return;
+ if (makeUnavailable) {
+ mHWVsyncAvailable = false;
}
-
- setVsyncPeriod(schedule, refreshRate->getPeriodNsecs(), false /* force */);
}
-void Scheduler::setRenderRate(PhysicalDisplayId id, Fps renderFrameRate) {
- std::scoped_lock lock(mDisplayLock);
- ftl::FakeGuard guard(kMainThreadContext);
+void Scheduler::resyncToHardwareVsync(bool makeAvailable, Fps refreshRate) {
+ {
+ std::lock_guard<std::mutex> lock(mHWVsyncLock);
+ if (makeAvailable) {
+ mHWVsyncAvailable = makeAvailable;
+ } else if (!mHWVsyncAvailable) {
+ // Hardware vsync is not currently available, so abort the resync
+ // attempt for now
+ return;
+ }
+ }
- auto selectorPtr = mRefreshRateSelectors.get(id);
- LOG_ALWAYS_FATAL_IF(!selectorPtr);
- const auto mode = selectorPtr->get()->getActiveMode();
+ setVsyncPeriod(refreshRate.getPeriodNsecs());
+}
+
+void Scheduler::setRenderRate(Fps renderFrameRate) {
+ const auto mode = leaderSelectorPtr()->getActiveMode();
using fps_approx_ops::operator!=;
LOG_ALWAYS_FATAL_IF(renderFrameRate != mode.fps,
- "Mismatch in render frame rates. Selector: %s, Scheduler: %s, Display: "
- "%" PRIu64,
- to_string(mode.fps).c_str(), to_string(renderFrameRate).c_str(), id.value);
+ "Mismatch in render frame rates. Selector: %s, Scheduler: %s",
+ to_string(mode.fps).c_str(), to_string(renderFrameRate).c_str());
ALOGV("%s %s (%s)", __func__, to_string(mode.fps).c_str(),
to_string(mode.modePtr->getFps()).c_str());
@@ -455,7 +444,7 @@
LOG_ALWAYS_FATAL_IF(divisor == 0, "%s <> %s -- not divisors", to_string(mode.fps).c_str(),
to_string(mode.fps).c_str());
- getVsyncScheduleLocked(id)->getTracker().setDivisor(static_cast<unsigned>(divisor));
+ mVsyncSchedule->getTracker().setDivisor(static_cast<unsigned>(divisor));
}
void Scheduler::resync() {
@@ -465,43 +454,49 @@
const nsecs_t last = mLastResyncTime.exchange(now);
if (now - last > kIgnoreDelay) {
- resyncAllToHardwareVsync(false /* allowToEnable */);
+ const auto refreshRate = leaderSelectorPtr()->getActiveMode().modePtr->getFps();
+ resyncToHardwareVsync(false, refreshRate);
}
}
-void Scheduler::setVsyncPeriod(const std::shared_ptr<VsyncSchedule>& schedule, nsecs_t period,
- bool force) {
- ALOGD("Scheduler::setVsyncPeriod");
+void Scheduler::setVsyncPeriod(nsecs_t period) {
if (period <= 0) return;
- // TODO (b/266712910):The old code held mHWVsyncLock before calling
- // startPeriodTransition. Move these into a new method on VsyncSchedule that
- // encapsulates this behavior there and allows holding the lock the whole
- // time.
- schedule->getController().startPeriodTransition(period, force);
- schedule->enableHardwareVsync(mSchedulerCallback);
+ std::lock_guard<std::mutex> lock(mHWVsyncLock);
+ mVsyncSchedule->getController().startPeriodTransition(period);
+
+ if (!mPrimaryHWVsyncEnabled) {
+ mVsyncSchedule->getTracker().resetModel();
+ mSchedulerCallback.setVsyncEnabled(true);
+ mPrimaryHWVsyncEnabled = true;
+ }
}
-bool Scheduler::addResyncSample(PhysicalDisplayId id, nsecs_t timestamp,
- std::optional<nsecs_t> hwcVsyncPeriod) {
- bool periodFlushed = false;
- auto schedule = getVsyncSchedule(id);
- if (schedule->getController().addHwVsyncTimestamp(timestamp, hwcVsyncPeriod, &periodFlushed)) {
- schedule->enableHardwareVsync(mSchedulerCallback);
- } else {
- schedule->disableHardwareVsync(mSchedulerCallback, false /* disallow */);
+void Scheduler::addResyncSample(nsecs_t timestamp, std::optional<nsecs_t> hwcVsyncPeriod,
+ bool* periodFlushed) {
+ bool needsHwVsync = false;
+ *periodFlushed = false;
+ { // Scope for the lock
+ std::lock_guard<std::mutex> lock(mHWVsyncLock);
+ if (mPrimaryHWVsyncEnabled) {
+ needsHwVsync =
+ mVsyncSchedule->getController().addHwVsyncTimestamp(timestamp, hwcVsyncPeriod,
+ periodFlushed);
+ }
}
- return periodFlushed;
+ if (needsHwVsync) {
+ enableHardwareVsync();
+ } else {
+ disableHardwareVsync(false);
+ }
}
-void Scheduler::addPresentFence(PhysicalDisplayId id, std::shared_ptr<FenceTime> fence) {
- auto schedule = getVsyncSchedule(id);
- const bool needMoreSignals = schedule->getController().addPresentFence(std::move(fence));
- if (needMoreSignals) {
- schedule->enableHardwareVsync(mSchedulerCallback);
+void Scheduler::addPresentFence(std::shared_ptr<FenceTime> fence) {
+ if (mVsyncSchedule->getController().addPresentFence(std::move(fence))) {
+ enableHardwareVsync();
} else {
- schedule->disableHardwareVsync(mSchedulerCallback, false /* disallow */);
+ disableHardwareVsync(false);
}
}
@@ -553,22 +548,12 @@
}
}
-void Scheduler::setDisplayPowerMode(PhysicalDisplayId id, hal::PowerMode powerMode) {
- const bool isLeader = [this, id]() REQUIRES(kMainThreadContext) {
- ftl::FakeGuard guard(mDisplayLock);
- return id == mLeaderDisplayId;
- }();
- if (isLeader) {
- // TODO (b/255657128): This needs to be handled per display.
+void Scheduler::setDisplayPowerMode(hal::PowerMode powerMode) {
+ {
std::lock_guard<std::mutex> lock(mPolicyLock);
mPolicy.displayPowerMode = powerMode;
}
- {
- std::scoped_lock lock(mDisplayLock);
- auto vsyncSchedule = getVsyncScheduleLocked(id);
- vsyncSchedule->getController().setDisplayPowerMode(powerMode);
- }
- if (!isLeader) return;
+ mVsyncSchedule->getController().setDisplayPowerMode(powerMode);
if (mDisplayPowerTimer) {
mDisplayPowerTimer->reset();
@@ -579,24 +564,6 @@
mLayerHistory.clear();
}
-std::shared_ptr<const VsyncSchedule> Scheduler::getVsyncSchedule(
- std::optional<PhysicalDisplayId> idOpt) const {
- std::scoped_lock lock(mDisplayLock);
- return getVsyncScheduleLocked(idOpt);
-}
-
-std::shared_ptr<const VsyncSchedule> Scheduler::getVsyncScheduleLocked(
- std::optional<PhysicalDisplayId> idOpt) const {
- ftl::FakeGuard guard(kMainThreadContext);
- if (!idOpt) {
- LOG_ALWAYS_FATAL_IF(!mLeaderDisplayId, "Missing a leader!");
- idOpt = mLeaderDisplayId;
- }
- auto scheduleOpt = mVsyncSchedules.get(*idOpt);
- LOG_ALWAYS_FATAL_IF(!scheduleOpt);
- return std::const_pointer_cast<const VsyncSchedule>(scheduleOpt->get());
-}
-
void Scheduler::kernelIdleTimerCallback(TimerState state) {
ATRACE_INT("ExpiredKernelIdleTimer", static_cast<int>(state));
@@ -611,17 +578,12 @@
// If we're not in performance mode then the kernel timer shouldn't do
// anything, as the refresh rate during DPU power collapse will be the
// same.
- resyncAllToHardwareVsync(true /* allowToEnable */);
+ resyncToHardwareVsync(true /* makeAvailable */, refreshRate);
} else if (state == TimerState::Expired && refreshRate <= FPS_THRESHOLD_FOR_KERNEL_TIMER) {
// Disable HW VSYNC if the timer expired, as we don't need it enabled if
// we're not pushing frames, and if we're in PERFORMANCE mode then we'll
// need to update the VsyncController model anyway.
- std::scoped_lock lock(mDisplayLock);
- ftl::FakeGuard guard(kMainThreadContext);
- constexpr bool disallow = false;
- for (auto& [_, schedule] : mVsyncSchedules) {
- schedule->disableHardwareVsync(mSchedulerCallback, disallow);
- }
+ disableHardwareVsync(false /* makeUnavailable */);
}
mSchedulerCallback.kernelTimerChanged(state == TimerState::Expired);
@@ -675,23 +637,19 @@
mFrameRateOverrideMappings.dump(dumper);
dumper.eol();
+
+ {
+ utils::Dumper::Section section(dumper, "Hardware VSYNC"sv);
+
+ std::lock_guard lock(mHWVsyncLock);
+ dumper.dump("screenAcquired"sv, mScreenAcquired.load());
+ dumper.dump("hwVsyncAvailable"sv, mHWVsyncAvailable);
+ dumper.dump("hwVsyncEnabled"sv, mPrimaryHWVsyncEnabled);
+ }
}
void Scheduler::dumpVsync(std::string& out) const {
- std::scoped_lock lock(mDisplayLock);
- ftl::FakeGuard guard(kMainThreadContext);
- if (mLeaderDisplayId) {
- base::StringAppendF(&out, "VsyncSchedule for leader %s:\n",
- to_string(*mLeaderDisplayId).c_str());
- getVsyncScheduleLocked()->dump(out);
- }
- for (auto& [id, vsyncSchedule] : mVsyncSchedules) {
- if (id == mLeaderDisplayId) {
- continue;
- }
- base::StringAppendF(&out, "VsyncSchedule for follower %s:\n", to_string(id).c_str());
- vsyncSchedule->dump(out);
- }
+ mVsyncSchedule->dump(out);
}
bool Scheduler::updateFrameRateOverrides(GlobalSignals consideredSignals, Fps displayRefreshRate) {
@@ -711,7 +669,6 @@
mLeaderDisplayId = leaderIdOpt.value_or(mRefreshRateSelectors.begin()->first);
ALOGI("Display %s is the leader", to_string(*mLeaderDisplayId).c_str());
- auto vsyncSchedule = getVsyncScheduleLocked(*mLeaderDisplayId);
if (const auto leaderPtr = leaderSelectorPtrLocked()) {
leaderPtr->setIdleTimerCallbacks(
{.platform = {.onReset = [this] { idleTimerCallback(TimerState::Reset); },
@@ -721,17 +678,6 @@
[this] { kernelIdleTimerCallback(TimerState::Expired); }}});
leaderPtr->startIdleTimer();
-
- const Fps refreshRate = leaderPtr->getActiveMode().modePtr->getFps();
- setVsyncPeriod(vsyncSchedule, refreshRate.getPeriodNsecs(), true /* force */);
- }
-
- updateVsyncRegistration(vsyncSchedule->getDispatch());
- {
- std::lock_guard<std::mutex> lock(mConnectionsLock);
- for (auto& [_, connection] : mConnections) {
- connection.thread->onNewVsyncSchedule(vsyncSchedule);
- }
}
}
diff --git a/services/surfaceflinger/Scheduler/Scheduler.h b/services/surfaceflinger/Scheduler/Scheduler.h
index 8c8fc21..796c785 100644
--- a/services/surfaceflinger/Scheduler/Scheduler.h
+++ b/services/surfaceflinger/Scheduler/Scheduler.h
@@ -43,7 +43,6 @@
#include "Display/DisplayModeRequest.h"
#include "EventThread.h"
#include "FrameRateOverrideMappings.h"
-#include "ISchedulerCallback.h"
#include "LayerHistory.h"
#include "MessageQueue.h"
#include "OneShotTimer.h"
@@ -93,7 +92,17 @@
using GlobalSignals = RefreshRateSelector::GlobalSignals;
-class Scheduler : android::impl::MessageQueue, public IEventThreadCallback {
+struct ISchedulerCallback {
+ virtual void setVsyncEnabled(bool) = 0;
+ virtual void requestDisplayModes(std::vector<display::DisplayModeRequest>) = 0;
+ virtual void kernelTimerChanged(bool expired) = 0;
+ virtual void triggerOnFrameRateOverridesChanged() = 0;
+
+protected:
+ ~ISchedulerCallback() = default;
+};
+
+class Scheduler : android::impl::MessageQueue {
using Impl = android::impl::MessageQueue;
public:
@@ -114,6 +123,8 @@
void run();
+ void createVsyncSchedule(FeatureFlags);
+
using Impl::initVsync;
using Impl::getScheduledFrameTime;
@@ -167,21 +178,9 @@
const VsyncModulator& vsyncModulator() const { return *mVsyncModulator; }
- // In some cases, we should only modulate for the leader display. In those
- // cases, the caller should pass in the relevant display, and the method
- // will no-op if it's not the leader. Other cases are not specific to a
- // display.
template <typename... Args,
typename Handler = std::optional<VsyncConfig> (VsyncModulator::*)(Args...)>
- void modulateVsync(std::optional<PhysicalDisplayId> id, Handler handler, Args... args) {
- if (id) {
- std::scoped_lock lock(mDisplayLock);
- ftl::FakeGuard guard(kMainThreadContext);
- if (id != mLeaderDisplayId) {
- return;
- }
- }
-
+ void modulateVsync(Handler handler, Args... args) {
if (const auto config = (*mVsyncModulator.*handler)(args...)) {
setVsyncConfig(*config, getLeaderVsyncPeriod());
}
@@ -190,32 +189,24 @@
void setVsyncConfigSet(const VsyncConfigSet&, Period vsyncPeriod);
// Sets the render rate for the scheduler to run at.
- void setRenderRate(PhysicalDisplayId, Fps);
+ void setRenderRate(Fps);
- void enableHardwareVsync(PhysicalDisplayId);
- void disableHardwareVsync(PhysicalDisplayId, bool makeUnavailable);
+ void enableHardwareVsync();
+ void disableHardwareVsync(bool makeUnavailable);
// Resyncs the scheduler to hardware vsync.
- // If allowToEnable is true, then hardware vsync will be turned on.
+ // If makeAvailable is true, then hardware vsync will be turned on.
// Otherwise, if hardware vsync is not already enabled then this method will
// no-op.
- // If refreshRate is nullopt, use the existing refresh rate of the display.
- void resyncToHardwareVsync(PhysicalDisplayId id, bool allowToEnable,
- std::optional<Fps> refreshRate = std::nullopt)
- EXCLUDES(mDisplayLock) {
- std::scoped_lock lock(mDisplayLock);
- ftl::FakeGuard guard(kMainThreadContext);
- resyncToHardwareVsyncLocked(id, allowToEnable, refreshRate);
- }
+ void resyncToHardwareVsync(bool makeAvailable, Fps refreshRate);
void resync() EXCLUDES(mDisplayLock);
void forceNextResync() { mLastResyncTime = 0; }
- // Passes a vsync sample to VsyncController. Returns true if
- // VsyncController detected that the vsync period changed and false
- // otherwise.
- bool addResyncSample(PhysicalDisplayId, nsecs_t timestamp,
- std::optional<nsecs_t> hwcVsyncPeriod);
- void addPresentFence(PhysicalDisplayId, std::shared_ptr<FenceTime>) EXCLUDES(mDisplayLock);
+ // Passes a vsync sample to VsyncController. periodFlushed will be true if
+ // VsyncController detected that the vsync period changed, and false otherwise.
+ void addResyncSample(nsecs_t timestamp, std::optional<nsecs_t> hwcVsyncPeriod,
+ bool* periodFlushed);
+ void addPresentFence(std::shared_ptr<FenceTime>);
// Layers are registered on creation, and unregistered when the weak reference expires.
void registerLayer(Layer*);
@@ -233,22 +224,20 @@
// Indicates that touch interaction is taking place.
void onTouchHint();
- void setDisplayPowerMode(PhysicalDisplayId, hal::PowerMode powerMode)
- REQUIRES(kMainThreadContext);
+ void setDisplayPowerMode(hal::PowerMode powerMode);
- std::shared_ptr<const VsyncSchedule> getVsyncSchedule(
- std::optional<PhysicalDisplayId> idOpt = std::nullopt) const EXCLUDES(mDisplayLock);
- std::shared_ptr<VsyncSchedule> getVsyncSchedule(
- std::optional<PhysicalDisplayId> idOpt = std::nullopt) EXCLUDES(mDisplayLock) {
- return std::const_pointer_cast<VsyncSchedule>(
- static_cast<const Scheduler*>(this)->getVsyncSchedule(idOpt));
- }
+ VsyncSchedule& getVsyncSchedule() { return *mVsyncSchedule; }
- bool isVsyncInPhase(TimePoint expectedVsyncTime, Fps frameRate) const;
+ // Returns true if a given vsync timestamp is considered valid vsync
+ // for a given uid
+ bool isVsyncValid(TimePoint expectedVsyncTimestamp, uid_t uid) const;
+
+ // Checks if a vsync timestamp is in phase for a frame rate
+ bool isVsyncInPhase(TimePoint timePoint, const Fps frameRate) const;
void dump(utils::Dumper&) const;
void dump(ConnectionHandle, std::string&) const;
- void dumpVsync(std::string&) const EXCLUDES(mDisplayLock);
+ void dumpVsync(std::string&) const;
// Returns the preferred refresh rate and frame rate for the leader display.
FrameRateMode getPreferredDisplayMode();
@@ -286,10 +275,6 @@
return mLayerHistory.getLayerFramerate(now, id);
}
- // IEventThreadCallback overrides:
- bool isVsyncTargetForUid(TimePoint expectedVsyncTime, uid_t uid) const override;
- Fps getLeaderRenderFrameRate(uid_t uid) const override;
-
private:
friend class TestableScheduler;
@@ -312,12 +297,7 @@
void touchTimerCallback(TimerState);
void displayPowerTimerCallback(TimerState);
- void resyncToHardwareVsyncLocked(PhysicalDisplayId, bool allowToEnable,
- std::optional<Fps> refreshRate = std::nullopt)
- REQUIRES(kMainThreadContext, mDisplayLock);
- void resyncAllToHardwareVsync(bool allowToEnable) EXCLUDES(mDisplayLock);
- void setVsyncPeriod(const std::shared_ptr<VsyncSchedule>&, nsecs_t period, bool force)
- REQUIRES(mDisplayLock);
+ void setVsyncPeriod(nsecs_t period);
void setVsyncConfig(const VsyncConfig&, Period vsyncPeriod);
// Chooses a leader among the registered displays, unless `leaderIdOpt` is specified. The new
@@ -329,12 +309,6 @@
// caller on the main thread to avoid deadlock, since the timer thread locks it before exit.
void demoteLeaderDisplay() REQUIRES(kMainThreadContext) EXCLUDES(mDisplayLock, mPolicyLock);
- void registerDisplayInternal(PhysicalDisplayId, RefreshRateSelectorPtr,
- std::shared_ptr<VsyncSchedule>) REQUIRES(kMainThreadContext)
- EXCLUDES(mDisplayLock);
-
- std::optional<Fps> getFrameRateOverrideLocked(uid_t) const REQUIRES(mDisplayLock);
-
struct Policy;
// Sets the S state of the policy to the T value under mPolicyLock, and chooses a display mode
@@ -372,6 +346,9 @@
void dispatchCachedReportedMode() REQUIRES(mPolicyLock) EXCLUDES(mDisplayLock);
+ android::impl::EventThread::ThrottleVsyncCallback makeThrottleVsyncCallback() const;
+ android::impl::EventThread::GetVsyncPeriodFunction makeGetVsyncPeriodFunction() const;
+
// Stores EventThread associated with a given VSyncSource, and an initial EventThreadConnection.
struct Connection {
sp<EventThreadConnection> connection;
@@ -385,9 +362,14 @@
ConnectionHandle mAppConnectionHandle;
ConnectionHandle mSfConnectionHandle;
+ mutable std::mutex mHWVsyncLock;
+ bool mPrimaryHWVsyncEnabled GUARDED_BY(mHWVsyncLock) = false;
+ bool mHWVsyncAvailable GUARDED_BY(mHWVsyncLock) = false;
+
std::atomic<nsecs_t> mLastResyncTime = 0;
const FeatureFlags mFeatures;
+ std::optional<VsyncSchedule> mVsyncSchedule;
// Shifts the VSYNC phase during certain transactions and refresh rate changes.
const sp<VsyncModulator> mVsyncModulator;
@@ -412,10 +394,6 @@
display::PhysicalDisplayMap<PhysicalDisplayId, RefreshRateSelectorPtr> mRefreshRateSelectors
GUARDED_BY(mDisplayLock) GUARDED_BY(kMainThreadContext);
- // TODO (b/266715559): Store in the same map as mRefreshRateSelectors.
- display::PhysicalDisplayMap<PhysicalDisplayId, std::shared_ptr<VsyncSchedule>> mVsyncSchedules
- GUARDED_BY(mDisplayLock) GUARDED_BY(kMainThreadContext);
-
ftl::Optional<PhysicalDisplayId> mLeaderDisplayId GUARDED_BY(mDisplayLock)
GUARDED_BY(kMainThreadContext);
@@ -435,14 +413,6 @@
.value_or(std::cref(noLeader));
}
- std::shared_ptr<const VsyncSchedule> getVsyncScheduleLocked(
- std::optional<PhysicalDisplayId> idOpt = std::nullopt) const REQUIRES(mDisplayLock);
- std::shared_ptr<VsyncSchedule> getVsyncScheduleLocked(
- std::optional<PhysicalDisplayId> idOpt = std::nullopt) REQUIRES(mDisplayLock) {
- return std::const_pointer_cast<VsyncSchedule>(
- static_cast<const Scheduler*>(this)->getVsyncScheduleLocked(idOpt));
- }
-
struct Policy {
// Policy for choosing the display mode.
LayerHistory::Summary contentRequirements;
@@ -469,6 +439,9 @@
static constexpr std::chrono::nanoseconds MAX_VSYNC_APPLIED_TIME = 200ms;
FrameRateOverrideMappings mFrameRateOverrideMappings;
+
+ // Keeps track of whether the screen is acquired for debug
+ std::atomic<bool> mScreenAcquired = false;
};
} // namespace scheduler
diff --git a/services/surfaceflinger/Scheduler/VSyncDispatch.h b/services/surfaceflinger/Scheduler/VSyncDispatch.h
index 77875e3..9520131 100644
--- a/services/surfaceflinger/Scheduler/VSyncDispatch.h
+++ b/services/surfaceflinger/Scheduler/VSyncDispatch.h
@@ -161,8 +161,7 @@
*/
class VSyncCallbackRegistration {
public:
- VSyncCallbackRegistration(std::shared_ptr<VSyncDispatch>, VSyncDispatch::Callback,
- std::string callbackName);
+ VSyncCallbackRegistration(VSyncDispatch&, VSyncDispatch::Callback, std::string callbackName);
~VSyncCallbackRegistration();
VSyncCallbackRegistration(VSyncCallbackRegistration&&);
@@ -178,7 +177,7 @@
CancelResult cancel();
private:
- std::shared_ptr<VSyncDispatch> mDispatch;
+ std::reference_wrapper<VSyncDispatch> mDispatch;
VSyncDispatch::CallbackToken mToken;
bool mValidToken;
};
diff --git a/services/surfaceflinger/Scheduler/VSyncDispatchTimerQueue.cpp b/services/surfaceflinger/Scheduler/VSyncDispatchTimerQueue.cpp
index 26389eb..73d52cf 100644
--- a/services/surfaceflinger/Scheduler/VSyncDispatchTimerQueue.cpp
+++ b/services/surfaceflinger/Scheduler/VSyncDispatchTimerQueue.cpp
@@ -215,10 +215,10 @@
}
VSyncDispatchTimerQueue::VSyncDispatchTimerQueue(std::unique_ptr<TimeKeeper> tk,
- VsyncSchedule::TrackerPtr tracker,
- nsecs_t timerSlack, nsecs_t minVsyncDistance)
+ VSyncTracker& tracker, nsecs_t timerSlack,
+ nsecs_t minVsyncDistance)
: mTimeKeeper(std::move(tk)),
- mTracker(std::move(tracker)),
+ mTracker(tracker),
mTimerSlack(timerSlack),
mMinVsyncDistance(minVsyncDistance) {}
@@ -255,7 +255,7 @@
}
if (it != skipUpdateIt) {
- callback->update(*mTracker, now);
+ callback->update(mTracker, now);
}
auto const wakeupTime = *callback->wakeupTime();
if (!min || *min > wakeupTime) {
@@ -365,10 +365,10 @@
auto const rearmImminent = now > mIntendedWakeupTime;
if (CC_UNLIKELY(rearmImminent)) {
callback->addPendingWorkloadUpdate(scheduleTiming);
- return getExpectedCallbackTime(*mTracker, now, scheduleTiming);
+ return getExpectedCallbackTime(mTracker, now, scheduleTiming);
}
- const ScheduleResult result = callback->schedule(scheduleTiming, *mTracker, now);
+ const ScheduleResult result = callback->schedule(scheduleTiming, mTracker, now);
if (!result.has_value()) {
return {};
}
@@ -434,15 +434,15 @@
}
}
-VSyncCallbackRegistration::VSyncCallbackRegistration(std::shared_ptr<VSyncDispatch> dispatch,
+VSyncCallbackRegistration::VSyncCallbackRegistration(VSyncDispatch& dispatch,
VSyncDispatch::Callback callback,
std::string callbackName)
- : mDispatch(std::move(dispatch)),
- mToken(mDispatch->registerCallback(std::move(callback), std::move(callbackName))),
+ : mDispatch(dispatch),
+ mToken(dispatch.registerCallback(std::move(callback), std::move(callbackName))),
mValidToken(true) {}
VSyncCallbackRegistration::VSyncCallbackRegistration(VSyncCallbackRegistration&& other)
- : mDispatch(std::move(other.mDispatch)),
+ : mDispatch(other.mDispatch),
mToken(std::move(other.mToken)),
mValidToken(std::move(other.mValidToken)) {
other.mValidToken = false;
@@ -457,28 +457,28 @@
}
VSyncCallbackRegistration::~VSyncCallbackRegistration() {
- if (mValidToken) mDispatch->unregisterCallback(mToken);
+ if (mValidToken) mDispatch.get().unregisterCallback(mToken);
}
ScheduleResult VSyncCallbackRegistration::schedule(VSyncDispatch::ScheduleTiming scheduleTiming) {
if (!mValidToken) {
return std::nullopt;
}
- return mDispatch->schedule(mToken, scheduleTiming);
+ return mDispatch.get().schedule(mToken, scheduleTiming);
}
ScheduleResult VSyncCallbackRegistration::update(VSyncDispatch::ScheduleTiming scheduleTiming) {
if (!mValidToken) {
return std::nullopt;
}
- return mDispatch->update(mToken, scheduleTiming);
+ return mDispatch.get().update(mToken, scheduleTiming);
}
CancelResult VSyncCallbackRegistration::cancel() {
if (!mValidToken) {
return CancelResult::Error;
}
- return mDispatch->cancel(mToken);
+ return mDispatch.get().cancel(mToken);
}
} // namespace android::scheduler
diff --git a/services/surfaceflinger/Scheduler/VSyncDispatchTimerQueue.h b/services/surfaceflinger/Scheduler/VSyncDispatchTimerQueue.h
index 6499d69..c3af136 100644
--- a/services/surfaceflinger/Scheduler/VSyncDispatchTimerQueue.h
+++ b/services/surfaceflinger/Scheduler/VSyncDispatchTimerQueue.h
@@ -26,11 +26,11 @@
#include <android-base/thread_annotations.h>
#include "VSyncDispatch.h"
-#include "VsyncSchedule.h"
namespace android::scheduler {
class TimeKeeper;
+class VSyncTracker;
// VSyncDispatchTimerQueueEntry is a helper class representing internal state for each entry in
// VSyncDispatchTimerQueue hoisted to public for unit testing.
@@ -120,8 +120,8 @@
// should be grouped into one wakeup.
// \param[in] minVsyncDistance The minimum distance between two vsync estimates before the
// vsyncs are considered the same vsync event.
- VSyncDispatchTimerQueue(std::unique_ptr<TimeKeeper>, VsyncSchedule::TrackerPtr,
- nsecs_t timerSlack, nsecs_t minVsyncDistance);
+ VSyncDispatchTimerQueue(std::unique_ptr<TimeKeeper>, VSyncTracker&, nsecs_t timerSlack,
+ nsecs_t minVsyncDistance);
~VSyncDispatchTimerQueue();
CallbackToken registerCallback(Callback, std::string callbackName) final;
@@ -148,7 +148,7 @@
static constexpr nsecs_t kInvalidTime = std::numeric_limits<int64_t>::max();
std::unique_ptr<TimeKeeper> const mTimeKeeper;
- VsyncSchedule::TrackerPtr mTracker;
+ VSyncTracker& mTracker;
nsecs_t const mTimerSlack;
nsecs_t const mMinVsyncDistance;
diff --git a/services/surfaceflinger/Scheduler/VSyncPredictor.cpp b/services/surfaceflinger/Scheduler/VSyncPredictor.cpp
index a3b8a56..02e12fd 100644
--- a/services/surfaceflinger/Scheduler/VSyncPredictor.cpp
+++ b/services/surfaceflinger/Scheduler/VSyncPredictor.cpp
@@ -31,7 +31,6 @@
#include <android-base/stringprintf.h>
#include <cutils/compiler.h>
#include <cutils/properties.h>
-#include <gui/TraceUtils.h>
#include <utils/Log.h>
#include <utils/Trace.h>
@@ -41,16 +40,14 @@
namespace android::scheduler {
using base::StringAppendF;
-using base::StringPrintf;
static auto constexpr kMaxPercent = 100u;
VSyncPredictor::~VSyncPredictor() = default;
-VSyncPredictor::VSyncPredictor(std::string name, nsecs_t idealPeriod, size_t historySize,
+VSyncPredictor::VSyncPredictor(nsecs_t idealPeriod, size_t historySize,
size_t minimumSamplesForPrediction, uint32_t outlierTolerancePercent)
- : mName(name),
- mTraceOn(property_get_bool("debug.sf.vsp_trace", false)),
+ : mTraceOn(property_get_bool("debug.sf.vsp_trace", false)),
kHistorySize(historySize),
kMinimumSamplesForPrediction(minimumSamplesForPrediction),
kOutlierTolerancePercent(std::min(outlierTolerancePercent, kMaxPercent)),
@@ -60,14 +57,12 @@
inline void VSyncPredictor::traceInt64If(const char* name, int64_t value) const {
if (CC_UNLIKELY(mTraceOn)) {
- traceInt64(name, value);
+ ATRACE_INT64(name, value);
}
}
inline void VSyncPredictor::traceInt64(const char* name, int64_t value) const {
- // TODO (b/266817103): Pass in PhysicalDisplayId and use ftl::Concat to
- // avoid unnecessary allocations.
- ATRACE_INT64(StringPrintf("%s %s", name, mName.c_str()).c_str(), value);
+ ATRACE_INT64(name, value);
}
inline size_t VSyncPredictor::next(size_t i) const {
@@ -219,8 +214,8 @@
it->second = {anticipatedPeriod, intercept};
- ALOGV("model update ts %s: %" PRId64 " slope: %" PRId64 " intercept: %" PRId64, mName.c_str(),
- timestamp, anticipatedPeriod, intercept);
+ ALOGV("model update ts: %" PRId64 " slope: %" PRId64 " intercept: %" PRId64, timestamp,
+ anticipatedPeriod, intercept);
return true;
}
@@ -332,7 +327,7 @@
}
void VSyncPredictor::setDivisor(unsigned divisor) {
- ALOGV("%s %s: %d", __func__, mName.c_str(), divisor);
+ ALOGV("%s: %d", __func__, divisor);
std::lock_guard lock(mMutex);
mDivisor = divisor;
}
@@ -348,7 +343,7 @@
}
void VSyncPredictor::setPeriod(nsecs_t period) {
- ATRACE_FORMAT("%s %s", __func__, mName.c_str());
+ ATRACE_CALL();
traceInt64("VSP-setPeriod", period);
std::lock_guard lock(mMutex);
diff --git a/services/surfaceflinger/Scheduler/VSyncPredictor.h b/services/surfaceflinger/Scheduler/VSyncPredictor.h
index 1ded54f..305cdb0 100644
--- a/services/surfaceflinger/Scheduler/VSyncPredictor.h
+++ b/services/surfaceflinger/Scheduler/VSyncPredictor.h
@@ -29,15 +29,14 @@
class VSyncPredictor : public VSyncTracker {
public:
/*
- * \param [in] name The name of the display this corresponds to.
* \param [in] idealPeriod The initial ideal period to use.
* \param [in] historySize The internal amount of entries to store in the model.
* \param [in] minimumSamplesForPrediction The minimum number of samples to collect before
* predicting. \param [in] outlierTolerancePercent a number 0 to 100 that will be used to filter
* samples that fall outlierTolerancePercent from an anticipated vsync event.
*/
- VSyncPredictor(std::string name, nsecs_t idealPeriod, size_t historySize,
- size_t minimumSamplesForPrediction, uint32_t outlierTolerancePercent);
+ VSyncPredictor(nsecs_t idealPeriod, size_t historySize, size_t minimumSamplesForPrediction,
+ uint32_t outlierTolerancePercent);
~VSyncPredictor();
bool addVsyncTimestamp(nsecs_t timestamp) final EXCLUDES(mMutex);
@@ -77,8 +76,6 @@
VSyncPredictor& operator=(VSyncPredictor const&) = delete;
void clearTimestamps() REQUIRES(mMutex);
- const std::string mName;
-
inline void traceInt64If(const char* name, int64_t value) const;
inline void traceInt64(const char* name, int64_t value) const;
bool const mTraceOn;
diff --git a/services/surfaceflinger/Scheduler/VSyncReactor.cpp b/services/surfaceflinger/Scheduler/VSyncReactor.cpp
index a831f66..b5f212e 100644
--- a/services/surfaceflinger/Scheduler/VSyncReactor.cpp
+++ b/services/surfaceflinger/Scheduler/VSyncReactor.cpp
@@ -21,7 +21,6 @@
#include <assert.h>
#include <cutils/properties.h>
-#include <gui/TraceUtils.h>
#include <log/log.h>
#include <utils/Trace.h>
@@ -33,7 +32,6 @@
namespace android::scheduler {
using base::StringAppendF;
-using base::StringPrintf;
VsyncController::~VsyncController() = default;
@@ -41,12 +39,12 @@
return systemTime(SYSTEM_TIME_MONOTONIC);
}
-VSyncReactor::VSyncReactor(std::string name, std::unique_ptr<Clock> clock, VSyncTracker& tracker,
+VSyncReactor::VSyncReactor(std::unique_ptr<Clock> clock, VSyncTracker& tracker,
size_t pendingFenceLimit, bool supportKernelIdleTimer)
- : mName(name),
- mClock(std::move(clock)),
+ : mClock(std::move(clock)),
mTracker(tracker),
mPendingLimit(pendingFenceLimit),
+ // TODO(adyabr): change mSupportKernelIdleTimer when the active display changes
mSupportKernelIdleTimer(supportKernelIdleTimer) {}
VSyncReactor::~VSyncReactor() = default;
@@ -116,7 +114,7 @@
}
void VSyncReactor::startPeriodTransitionInternal(nsecs_t newPeriod) {
- ATRACE_FORMAT("%s %s", __func__, mName.c_str());
+ ATRACE_CALL();
mPeriodConfirmationInProgress = true;
mPeriodTransitioningTo = newPeriod;
mMoreSamplesNeeded = true;
@@ -124,20 +122,18 @@
}
void VSyncReactor::endPeriodTransition() {
- ATRACE_FORMAT("%s %s", __func__, mName.c_str());
+ ATRACE_CALL();
mPeriodTransitioningTo.reset();
mPeriodConfirmationInProgress = false;
mLastHwVsync.reset();
}
-void VSyncReactor::startPeriodTransition(nsecs_t period, bool force) {
- // TODO (b/266817103): Pass in PhysicalDisplayId and use ftl::Concat to
- // avoid unnecessary allocations.
- ATRACE_INT64(StringPrintf("VSR-startPeriodTransition %s", mName.c_str()).c_str(), period);
+void VSyncReactor::startPeriodTransition(nsecs_t period) {
+ ATRACE_INT64("VSR-startPeriodTransition", period);
std::lock_guard lock(mMutex);
mLastHwVsync.reset();
- if (!mSupportKernelIdleTimer && period == mTracker.currentPeriod() && !force) {
+ if (!mSupportKernelIdleTimer && period == mTracker.currentPeriod()) {
endPeriodTransition();
setIgnorePresentFencesInternal(false);
mMoreSamplesNeeded = false;
@@ -185,7 +181,7 @@
std::lock_guard lock(mMutex);
if (periodConfirmed(timestamp, hwcVsyncPeriod)) {
- ATRACE_FORMAT("VSR %s: period confirmed", mName.c_str());
+ ATRACE_NAME("VSR: period confirmed");
if (mPeriodTransitioningTo) {
mTracker.setPeriod(*mPeriodTransitioningTo);
*periodFlushed = true;
@@ -199,12 +195,12 @@
endPeriodTransition();
mMoreSamplesNeeded = mTracker.needsMoreSamples();
} else if (mPeriodConfirmationInProgress) {
- ATRACE_FORMAT("VSR %s: still confirming period", mName.c_str());
+ ATRACE_NAME("VSR: still confirming period");
mLastHwVsync = timestamp;
mMoreSamplesNeeded = true;
*periodFlushed = false;
} else {
- ATRACE_FORMAT("VSR %s: adding sample", mName.c_str());
+ ATRACE_NAME("VSR: adding sample");
*periodFlushed = false;
mTracker.addVsyncTimestamp(timestamp);
mMoreSamplesNeeded = mTracker.needsMoreSamples();
diff --git a/services/surfaceflinger/Scheduler/VSyncReactor.h b/services/surfaceflinger/Scheduler/VSyncReactor.h
index fd9ca42..4501487 100644
--- a/services/surfaceflinger/Scheduler/VSyncReactor.h
+++ b/services/surfaceflinger/Scheduler/VSyncReactor.h
@@ -22,7 +22,6 @@
#include <vector>
#include <android-base/thread_annotations.h>
-#include <ui/DisplayId.h>
#include <ui/FenceTime.h>
#include <scheduler/TimeKeeper.h>
@@ -38,14 +37,14 @@
// TODO (b/145217110): consider renaming.
class VSyncReactor : public VsyncController {
public:
- VSyncReactor(std::string name, std::unique_ptr<Clock> clock, VSyncTracker& tracker,
- size_t pendingFenceLimit, bool supportKernelIdleTimer);
+ VSyncReactor(std::unique_ptr<Clock> clock, VSyncTracker& tracker, size_t pendingFenceLimit,
+ bool supportKernelIdleTimer);
~VSyncReactor();
bool addPresentFence(std::shared_ptr<FenceTime>) final;
void setIgnorePresentFences(bool ignore) final;
- void startPeriodTransition(nsecs_t period, bool force) final;
+ void startPeriodTransition(nsecs_t period) final;
bool addHwVsyncTimestamp(nsecs_t timestamp, std::optional<nsecs_t> hwcVsyncPeriod,
bool* periodFlushed) final;
@@ -62,7 +61,6 @@
bool periodConfirmed(nsecs_t vsync_timestamp, std::optional<nsecs_t> hwcVsyncPeriod)
REQUIRES(mMutex);
- const std::string mName;
std::unique_ptr<Clock> const mClock;
VSyncTracker& mTracker;
size_t const mPendingLimit;
diff --git a/services/surfaceflinger/Scheduler/VsyncController.h b/services/surfaceflinger/Scheduler/VsyncController.h
index 9177899..726a420 100644
--- a/services/surfaceflinger/Scheduler/VsyncController.h
+++ b/services/surfaceflinger/Scheduler/VsyncController.h
@@ -63,9 +63,8 @@
* itself. The controller will end the period transition internally.
*
* \param [in] period The period that the system is changing into.
- * \param [in] force True to recalibrate even if period matches the existing period.
*/
- virtual void startPeriodTransition(nsecs_t period, bool force) = 0;
+ virtual void startPeriodTransition(nsecs_t period) = 0;
/*
* Tells the tracker to stop using present fences to get a vsync signal.
diff --git a/services/surfaceflinger/Scheduler/VsyncSchedule.cpp b/services/surfaceflinger/Scheduler/VsyncSchedule.cpp
index 951c1ec..95bc31f 100644
--- a/services/surfaceflinger/Scheduler/VsyncSchedule.cpp
+++ b/services/surfaceflinger/Scheduler/VsyncSchedule.cpp
@@ -21,9 +21,6 @@
#include "VsyncSchedule.h"
-#include "ISchedulerCallback.h"
-#include "Scheduler.h"
-#include "Utils/Dumper.h"
#include "VSyncDispatchTimerQueue.h"
#include "VSyncPredictor.h"
#include "VSyncReactor.h"
@@ -42,8 +39,8 @@
}
public:
- explicit PredictedVsyncTracer(std::shared_ptr<VsyncDispatch> dispatch)
- : mRegistration(std::move(dispatch), makeVsyncCallback(), __func__) {
+ explicit PredictedVsyncTracer(VsyncDispatch& dispatch)
+ : mRegistration(dispatch, makeVsyncCallback(), __func__) {
schedule();
}
@@ -54,23 +51,21 @@
VSyncCallbackRegistration mRegistration;
};
-VsyncSchedule::VsyncSchedule(PhysicalDisplayId id, FeatureFlags features)
- : mId(id),
- mTracker(createTracker(id)),
- mDispatch(createDispatch(mTracker)),
- mController(createController(id, *mTracker, features)) {
+VsyncSchedule::VsyncSchedule(FeatureFlags features)
+ : mTracker(createTracker()),
+ mDispatch(createDispatch(*mTracker)),
+ mController(createController(*mTracker, features)) {
if (features.test(Feature::kTracePredictedVsync)) {
- mTracer = std::make_unique<PredictedVsyncTracer>(mDispatch);
+ mTracer = std::make_unique<PredictedVsyncTracer>(*mDispatch);
}
}
-VsyncSchedule::VsyncSchedule(PhysicalDisplayId id, TrackerPtr tracker, DispatchPtr dispatch,
- ControllerPtr controller)
- : mId(id),
- mTracker(std::move(tracker)),
+VsyncSchedule::VsyncSchedule(TrackerPtr tracker, DispatchPtr dispatch, ControllerPtr controller)
+ : mTracker(std::move(tracker)),
mDispatch(std::move(dispatch)),
mController(std::move(controller)) {}
+VsyncSchedule::VsyncSchedule(VsyncSchedule&&) = default;
VsyncSchedule::~VsyncSchedule() = default;
Period VsyncSchedule::period() const {
@@ -82,13 +77,6 @@
}
void VsyncSchedule::dump(std::string& out) const {
- utils::Dumper dumper(out);
- {
- std::lock_guard<std::mutex> lock(mHwVsyncLock);
- dumper.dump("hwVsyncState", ftl::enum_string(mHwVsyncState));
- dumper.dump("lastHwVsyncState", ftl::enum_string(mLastHwVsyncState));
- }
-
out.append("VsyncController:\n");
mController->dump(out);
@@ -96,72 +84,40 @@
mDispatch->dump(out);
}
-VsyncSchedule::TrackerPtr VsyncSchedule::createTracker(PhysicalDisplayId id) {
+VsyncSchedule::TrackerPtr VsyncSchedule::createTracker() {
// TODO(b/144707443): Tune constants.
constexpr nsecs_t kInitialPeriod = (60_Hz).getPeriodNsecs();
constexpr size_t kHistorySize = 20;
constexpr size_t kMinSamplesForPrediction = 6;
constexpr uint32_t kDiscardOutlierPercent = 20;
- return std::make_unique<VSyncPredictor>(to_string(id), kInitialPeriod, kHistorySize,
- kMinSamplesForPrediction, kDiscardOutlierPercent);
+ return std::make_unique<VSyncPredictor>(kInitialPeriod, kHistorySize, kMinSamplesForPrediction,
+ kDiscardOutlierPercent);
}
-VsyncSchedule::DispatchPtr VsyncSchedule::createDispatch(TrackerPtr tracker) {
+VsyncSchedule::DispatchPtr VsyncSchedule::createDispatch(VsyncTracker& tracker) {
using namespace std::chrono_literals;
// TODO(b/144707443): Tune constants.
constexpr std::chrono::nanoseconds kGroupDispatchWithin = 500us;
constexpr std::chrono::nanoseconds kSnapToSameVsyncWithin = 3ms;
- return std::make_unique<VSyncDispatchTimerQueue>(std::make_unique<Timer>(), std::move(tracker),
+ return std::make_unique<VSyncDispatchTimerQueue>(std::make_unique<Timer>(), tracker,
kGroupDispatchWithin.count(),
kSnapToSameVsyncWithin.count());
}
-VsyncSchedule::ControllerPtr VsyncSchedule::createController(PhysicalDisplayId id,
- VsyncTracker& tracker,
+VsyncSchedule::ControllerPtr VsyncSchedule::createController(VsyncTracker& tracker,
FeatureFlags features) {
// TODO(b/144707443): Tune constants.
constexpr size_t kMaxPendingFences = 20;
const bool hasKernelIdleTimer = features.test(Feature::kKernelIdleTimer);
- auto reactor = std::make_unique<VSyncReactor>(to_string(id), std::make_unique<SystemClock>(),
- tracker, kMaxPendingFences, hasKernelIdleTimer);
+ auto reactor = std::make_unique<VSyncReactor>(std::make_unique<SystemClock>(), tracker,
+ kMaxPendingFences, hasKernelIdleTimer);
reactor->setIgnorePresentFences(!features.test(Feature::kPresentFences));
return reactor;
}
-void VsyncSchedule::enableHardwareVsync(ISchedulerCallback& callback) {
- std::lock_guard<std::mutex> lock(mHwVsyncLock);
- if (mHwVsyncState == HwVsyncState::Disabled) {
- getTracker().resetModel();
- callback.setVsyncEnabled(mId, true);
- mHwVsyncState = HwVsyncState::Enabled;
- mLastHwVsyncState = HwVsyncState::Enabled;
- }
-}
-
-void VsyncSchedule::disableHardwareVsync(ISchedulerCallback& callback, bool disallow) {
- std::lock_guard<std::mutex> lock(mHwVsyncLock);
- if (mHwVsyncState == HwVsyncState::Enabled) {
- callback.setVsyncEnabled(mId, false);
- mLastHwVsyncState = HwVsyncState::Disabled;
- }
- mHwVsyncState = disallow ? HwVsyncState::Disallowed : HwVsyncState::Disabled;
-}
-
-bool VsyncSchedule::isHardwareVsyncAllowed() const {
- std::lock_guard<std::mutex> lock(mHwVsyncLock);
- return mHwVsyncState != HwVsyncState::Disallowed;
-}
-
-void VsyncSchedule::allowHardwareVsync() {
- std::lock_guard<std::mutex> lock(mHwVsyncLock);
- if (mHwVsyncState == HwVsyncState::Disallowed) {
- mHwVsyncState = HwVsyncState::Disabled;
- }
-}
-
} // namespace android::scheduler
diff --git a/services/surfaceflinger/Scheduler/VsyncSchedule.h b/services/surfaceflinger/Scheduler/VsyncSchedule.h
index ffb7ad5..173b1d0 100644
--- a/services/surfaceflinger/Scheduler/VsyncSchedule.h
+++ b/services/surfaceflinger/Scheduler/VsyncSchedule.h
@@ -19,10 +19,8 @@
#include <memory>
#include <string>
-#include <ftl/enum.h>
#include <scheduler/Features.h>
#include <scheduler/Time.h>
-#include <ui/DisplayId.h>
namespace android {
class EventThreadTest;
@@ -34,8 +32,6 @@
namespace android::scheduler {
-struct ISchedulerCallback;
-
// TODO(b/185535769): Rename classes, and remove aliases.
class VSyncDispatch;
class VSyncTracker;
@@ -47,7 +43,8 @@
// Schedule that synchronizes to hardware VSYNC of a physical display.
class VsyncSchedule {
public:
- VsyncSchedule(PhysicalDisplayId, FeatureFlags);
+ explicit VsyncSchedule(FeatureFlags);
+ VsyncSchedule(VsyncSchedule&&);
~VsyncSchedule();
Period period() const;
@@ -58,71 +55,30 @@
VsyncTracker& getTracker() { return *mTracker; }
VsyncController& getController() { return *mController; }
- // TODO(b/185535769): Once these are hidden behind the API, they may no
- // longer need to be shared_ptrs.
- using DispatchPtr = std::shared_ptr<VsyncDispatch>;
- using TrackerPtr = std::shared_ptr<VsyncTracker>;
-
// TODO(b/185535769): Remove once VsyncSchedule owns all registrations.
- DispatchPtr getDispatch() { return mDispatch; }
+ VsyncDispatch& getDispatch() { return *mDispatch; }
void dump(std::string&) const;
- // Turn on hardware vsyncs, unless mHwVsyncState is Disallowed, in which
- // case this call is ignored.
- void enableHardwareVsync(ISchedulerCallback&) EXCLUDES(mHwVsyncLock);
-
- // Disable hardware vsyncs. If `disallow` is true, future calls to
- // enableHardwareVsync are ineffective until allowHardwareVsync is called.
- void disableHardwareVsync(ISchedulerCallback&, bool disallow) EXCLUDES(mHwVsyncLock);
-
- // Restore the ability to enable hardware vsync.
- void allowHardwareVsync() EXCLUDES(mHwVsyncLock);
-
- // If true, enableHardwareVsync can enable hardware vsync (if not already
- // enabled). If false, enableHardwareVsync does nothing.
- bool isHardwareVsyncAllowed() const EXCLUDES(mHwVsyncLock);
-
-protected:
- using ControllerPtr = std::unique_ptr<VsyncController>;
-
- // For tests.
- VsyncSchedule(PhysicalDisplayId, TrackerPtr, DispatchPtr, ControllerPtr);
-
private:
friend class TestableScheduler;
friend class android::EventThreadTest;
friend class android::fuzz::SchedulerFuzzer;
- static TrackerPtr createTracker(PhysicalDisplayId);
- static DispatchPtr createDispatch(TrackerPtr);
- static ControllerPtr createController(PhysicalDisplayId, VsyncTracker&, FeatureFlags);
+ using TrackerPtr = std::unique_ptr<VsyncTracker>;
+ using DispatchPtr = std::unique_ptr<VsyncDispatch>;
+ using ControllerPtr = std::unique_ptr<VsyncController>;
- mutable std::mutex mHwVsyncLock;
- enum class HwVsyncState {
- // Hardware vsyncs are currently enabled.
- Enabled,
+ // For tests.
+ VsyncSchedule(TrackerPtr, DispatchPtr, ControllerPtr);
- // Hardware vsyncs are currently disabled. They can be enabled by a call
- // to `enableHardwareVsync`.
- Disabled,
-
- // Hardware vsyncs are not currently allowed (e.g. because the display
- // is off).
- Disallowed,
-
- ftl_last = Disallowed,
- };
- HwVsyncState mHwVsyncState GUARDED_BY(mHwVsyncLock) = HwVsyncState::Disallowed;
-
- // The last state, which may be the current state, or the state prior to setting to Disallowed.
- HwVsyncState mLastHwVsyncState GUARDED_BY(mHwVsyncLock) = HwVsyncState::Disabled;
+ static TrackerPtr createTracker();
+ static DispatchPtr createDispatch(VsyncTracker&);
+ static ControllerPtr createController(VsyncTracker&, FeatureFlags);
class PredictedVsyncTracer;
using TracerPtr = std::unique_ptr<PredictedVsyncTracer>;
- const PhysicalDisplayId mId;
-
// Effectively const except in move constructor.
TrackerPtr mTracker;
DispatchPtr mDispatch;