SF: Move MessageQueue to Scheduler
...as a first step in removing the ISchedulerCallback::scheduleComposite
roundtrip, and extracting scheduling logic from SF::{commit,composite}.
Bug: 185535769
Test: libsurfaceflinger_unittest
Change-Id: I7fb38a1dd7b917e5639b9d58a0d44b32983b689e
diff --git a/services/surfaceflinger/Scheduler/Scheduler.cpp b/services/surfaceflinger/Scheduler/Scheduler.cpp
index f201996..4d72798 100644
--- a/services/surfaceflinger/Scheduler/Scheduler.cpp
+++ b/services/surfaceflinger/Scheduler/Scheduler.cpp
@@ -117,8 +117,8 @@
}
};
-Scheduler::Scheduler(ISchedulerCallback& callback, Options options)
- : mOptions(options), mSchedulerCallback(callback) {}
+Scheduler::Scheduler(ICompositor& compositor, ISchedulerCallback& callback, Options options)
+ : impl::MessageQueue(compositor), mOptions(options), mSchedulerCallback(callback) {}
void Scheduler::startTimers() {
using namespace sysprop;
@@ -148,6 +148,12 @@
mRefreshRateConfigs.reset();
}
+void Scheduler::run() {
+ while (true) {
+ waitMessage();
+ }
+}
+
void Scheduler::createVsyncSchedule(bool supportKernelTimer) {
auto clock = std::make_unique<scheduler::SystemClock>();
auto tracker = createVSyncTracker();
diff --git a/services/surfaceflinger/Scheduler/Scheduler.h b/services/surfaceflinger/Scheduler/Scheduler.h
index 8204abc..6d45b5d 100644
--- a/services/surfaceflinger/Scheduler/Scheduler.h
+++ b/services/surfaceflinger/Scheduler/Scheduler.h
@@ -18,6 +18,7 @@
#include <atomic>
#include <functional>
+#include <future>
#include <memory>
#include <mutex>
#include <optional>
@@ -32,6 +33,7 @@
#include "EventThread.h"
#include "LayerHistory.h"
+#include "MessageQueue.h"
#include "OneShotTimer.h"
#include "RefreshRateConfigs.h"
#include "SchedulerUtils.h"
@@ -70,21 +72,41 @@
~ISchedulerCallback() = default;
};
-class Scheduler {
+class Scheduler : impl::MessageQueue {
+ using Impl = impl::MessageQueue;
+
public:
using RefreshRate = scheduler::RefreshRateConfigs::RefreshRate;
using ModeEvent = scheduler::RefreshRateConfigEvent;
struct Options {
// Whether to use content detection at all.
- bool useContentDetection = false;
+ bool useContentDetection;
};
- Scheduler(ISchedulerCallback&, Options);
+ Scheduler(ICompositor&, ISchedulerCallback&, Options);
~Scheduler();
void createVsyncSchedule(bool supportKernelIdleTimer);
void startTimers();
+ void run();
+
+ using Impl::initVsync;
+ using Impl::setInjector;
+
+ using Impl::getScheduledFrameTime;
+ using Impl::setDuration;
+
+ using Impl::scheduleCommit;
+ using Impl::scheduleComposite;
+
+ // Schedule an asynchronous or synchronous task on the main thread.
+ template <typename F, typename T = std::invoke_result_t<F>>
+ [[nodiscard]] std::future<T> schedule(F&& f) {
+ auto [task, future] = makeTask(std::move(f));
+ postMessage(std::move(task));
+ return std::move(future);
+ }
using ConnectionHandle = scheduler::ConnectionHandle;
ConnectionHandle createConnection(const char* connectionName, frametimeline::TokenManager*,
diff --git a/services/surfaceflinger/SurfaceFlinger.cpp b/services/surfaceflinger/SurfaceFlinger.cpp
index d1e24d9..48bd54f 100644
--- a/services/surfaceflinger/SurfaceFlinger.cpp
+++ b/services/surfaceflinger/SurfaceFlinger.cpp
@@ -125,7 +125,6 @@
#include "Scheduler/DispSyncSource.h"
#include "Scheduler/EventThread.h"
#include "Scheduler/LayerHistory.h"
-#include "Scheduler/MessageQueue.h"
#include "Scheduler/Scheduler.h"
#include "Scheduler/VsyncConfiguration.h"
#include "Scheduler/VsyncController.h"
@@ -362,7 +361,6 @@
mTimeStats(std::make_shared<impl::TimeStats>()),
mFrameTracer(mFactory.createFrameTracer()),
mFrameTimeline(mFactory.createFrameTimeline(mTimeStats, getpid())),
- mEventQueue(mFactory.createMessageQueue(*this)),
mCompositionEngine(mFactory.createCompositionEngine()),
mHwcServiceName(base::GetProperty("debug.sf.hwc_service_name"s, "default"s)),
mTunnelModeEnabledReporter(new TunnelModeEnabledReporter()),
@@ -509,8 +507,8 @@
// the window manager died on us. prepare its eulogy.
mBootFinished = false;
- // Sever the link to inputflinger since its gone as well.
- static_cast<void>(schedule([=] { mInputFlinger = nullptr; }));
+ // Sever the link to inputflinger since it's gone as well.
+ static_cast<void>(mScheduler->schedule([=] { mInputFlinger = nullptr; }));
// restore initial conditions (default device unblank, etc)
initializeDisplays();
@@ -520,16 +518,7 @@
}
void SurfaceFlinger::run() {
- while (true) {
- mEventQueue->waitMessage();
- }
-}
-
-template <typename F, typename T>
-inline std::future<T> SurfaceFlinger::schedule(F&& f) {
- auto [task, future] = makeTask(std::move(f));
- mEventQueue->postMessage(std::move(task));
- return std::move(future);
+ mScheduler->run();
}
sp<ISurfaceComposerClient> SurfaceFlinger::createConnection() {
@@ -730,7 +719,7 @@
sp<IBinder> input(defaultServiceManager()->getService(String16("inputflinger")));
- static_cast<void>(schedule([=] {
+ static_cast<void>(mScheduler->schedule([=] {
if (input == nullptr) {
ALOGE("Failed to link to input service");
} else {
@@ -771,7 +760,7 @@
if (std::this_thread::get_id() == mMainThreadId) {
return genTextures();
} else {
- return schedule(genTextures).get();
+ return mScheduler->schedule(genTextures).get();
}
}
@@ -1120,7 +1109,7 @@
return BAD_VALUE;
}
- auto future = schedule([=]() -> status_t {
+ auto future = mScheduler->schedule([=]() -> status_t {
const auto display = ON_MAIN_THREAD(getDisplayDeviceLocked(displayToken));
if (!display) {
ALOGE("Attempt to set allowed display modes for invalid display token %p",
@@ -1284,7 +1273,7 @@
}
void SurfaceFlinger::disableExpensiveRendering() {
- schedule([=]() MAIN_THREAD {
+ auto future = mScheduler->schedule([=]() MAIN_THREAD {
ATRACE_CALL();
if (mPowerAdvisor.isUsingExpensiveRendering()) {
const auto& displays = ON_MAIN_THREAD(mDisplays);
@@ -1293,7 +1282,9 @@
mPowerAdvisor.setExpensiveRenderingExpected(display->getId(), kDisable);
}
}
- }).wait();
+ });
+
+ future.wait();
}
std::vector<ColorMode> SurfaceFlinger::getDisplayColorModes(const DisplayDevice& display) {
@@ -1332,7 +1323,7 @@
return BAD_VALUE;
}
- auto future = schedule([=]() MAIN_THREAD -> status_t {
+ auto future = mScheduler->schedule([=]() MAIN_THREAD -> status_t {
const auto display = getDisplayDeviceLocked(displayToken);
if (!display) {
ALOGE("Attempt to set active color mode %s (%d) for invalid display token %p",
@@ -1367,22 +1358,24 @@
}
void SurfaceFlinger::setAutoLowLatencyMode(const sp<IBinder>& displayToken, bool on) {
- static_cast<void>(schedule([=]() MAIN_THREAD {
+ const char* const whence = __func__;
+ static_cast<void>(mScheduler->schedule([=]() MAIN_THREAD {
if (const auto displayId = getPhysicalDisplayIdLocked(displayToken)) {
getHwComposer().setAutoLowLatencyMode(*displayId, on);
} else {
- ALOGE("%s: Invalid display token %p", __FUNCTION__, displayToken.get());
+ ALOGE("%s: Invalid display token %p", whence, displayToken.get());
}
}));
}
void SurfaceFlinger::setGameContentType(const sp<IBinder>& displayToken, bool on) {
- static_cast<void>(schedule([=]() MAIN_THREAD {
+ const char* const whence = __func__;
+ static_cast<void>(mScheduler->schedule([=]() MAIN_THREAD {
if (const auto displayId = getPhysicalDisplayIdLocked(displayToken)) {
const auto type = on ? hal::ContentType::GAME : hal::ContentType::NONE;
getHwComposer().setContentType(*displayId, type);
} else {
- ALOGE("%s: Invalid display token %p", __FUNCTION__, displayToken.get());
+ ALOGE("%s: Invalid display token %p", whence, displayToken.get());
}
}));
}
@@ -1441,17 +1434,18 @@
status_t SurfaceFlinger::setDisplayContentSamplingEnabled(const sp<IBinder>& displayToken,
bool enable, uint8_t componentMask,
uint64_t maxFrames) {
- return schedule([=]() MAIN_THREAD -> status_t {
- if (const auto displayId = getPhysicalDisplayIdLocked(displayToken)) {
- return getHwComposer().setDisplayContentSamplingEnabled(*displayId, enable,
- componentMask,
- maxFrames);
- } else {
- ALOGE("%s: Invalid display token %p", __FUNCTION__, displayToken.get());
- return NAME_NOT_FOUND;
- }
- })
- .get();
+ const char* const whence = __func__;
+ auto future = mScheduler->schedule([=]() MAIN_THREAD -> status_t {
+ if (const auto displayId = getPhysicalDisplayIdLocked(displayToken)) {
+ return getHwComposer().setDisplayContentSamplingEnabled(*displayId, enable,
+ componentMask, maxFrames);
+ } else {
+ ALOGE("%s: Invalid display token %p", whence, displayToken.get());
+ return NAME_NOT_FOUND;
+ }
+ });
+
+ return future.get();
}
status_t SurfaceFlinger::getDisplayedContentSample(const sp<IBinder>& displayToken,
@@ -1493,14 +1487,15 @@
}
status_t SurfaceFlinger::enableVSyncInjections(bool enable) {
- schedule([=] {
+ auto future = mScheduler->schedule([=] {
Mutex::Autolock lock(mStateLock);
if (const auto handle = mScheduler->enableVSyncInjection(enable)) {
- mEventQueue->setInjector(enable ? mScheduler->getEventConnection(handle) : nullptr);
+ mScheduler->setInjector(enable ? mScheduler->getEventConnection(handle) : nullptr);
}
- }).wait();
+ });
+ future.wait();
return NO_ERROR;
}
@@ -1516,12 +1511,14 @@
status_t SurfaceFlinger::getLayerDebugInfo(std::vector<LayerDebugInfo>* outLayers) {
outLayers->clear();
- schedule([=] {
+ auto future = mScheduler->schedule([=] {
const auto display = ON_MAIN_THREAD(getDefaultDisplayDeviceLocked());
mDrawingState.traverseInZOrder([&](Layer* layer) {
outLayers->push_back(layer->getLayerDebugInfo(display.get()));
});
- }).wait();
+ });
+
+ future.wait();
return NO_ERROR;
}
@@ -1616,7 +1613,8 @@
return BAD_VALUE;
}
- return ftl::chain(schedule([=]() MAIN_THREAD {
+ const char* const whence = __func__;
+ return ftl::chain(mScheduler->schedule([=]() MAIN_THREAD {
if (const auto display = getDisplayDeviceLocked(displayToken)) {
if (enableSdrDimming) {
display->getCompositionDisplay()
@@ -1626,7 +1624,7 @@
return getHwComposer().setDisplayBrightness(display->getPhysicalId(),
brightness.displayBrightness);
} else {
- ALOGE("%s: Invalid display token %p", __FUNCTION__, displayToken.get());
+ ALOGE("%s: Invalid display token %p", whence, displayToken.get());
return ftl::yield<status_t>(NAME_NOT_FOUND);
}
}))
@@ -1704,7 +1702,7 @@
mScheduler->resetIdleTimer();
}
mPowerAdvisor.notifyDisplayUpdateImminent();
- mEventQueue->scheduleCommit();
+ mScheduler->scheduleCommit();
}
void SurfaceFlinger::scheduleComposite(FrameHint hint) {
@@ -1718,7 +1716,7 @@
}
void SurfaceFlinger::scheduleSample() {
- static_cast<void>(schedule([this] { sample(); }));
+ static_cast<void>(mScheduler->schedule([this] { sample(); }));
}
nsecs_t SurfaceFlinger::getVsyncPeriodFromHWC() const {
@@ -1813,7 +1811,7 @@
ATRACE_CALL();
// On main thread to avoid race conditions with display power state.
- static_cast<void>(schedule([=]() MAIN_THREAD {
+ static_cast<void>(mScheduler->schedule([=]() MAIN_THREAD {
mHWCVsyncPendingState = enabled ? hal::Vsync::ENABLE : hal::Vsync::DISABLE;
if (const auto display = getDefaultDisplayDeviceLocked();
@@ -1930,7 +1928,7 @@
// fired yet just wait for the next commit.
if (mSetActiveModePending) {
if (framePending) {
- mEventQueue->scheduleCommit();
+ mScheduler->scheduleCommit();
return false;
}
@@ -2058,7 +2056,7 @@
const auto hwcMinWorkDuration = mVsyncConfiguration->getCurrentConfigs().hwcMinWorkDuration;
refreshArgs.earliestPresentTime = prevVsyncTime - hwcMinWorkDuration;
refreshArgs.previousPresentFence = mPreviousPresentFences[0].fenceTime;
- refreshArgs.scheduledFrameTime = mEventQueue->getScheduledFrameTime();
+ refreshArgs.scheduledFrameTime = mScheduler->getScheduledFrameTime();
// Store the present time just before calling to the composition engine so we could notify
// the scheduler.
@@ -2785,7 +2783,7 @@
mDisplays.erase(displayToken);
if (display && display->isVirtual()) {
- static_cast<void>(schedule([display = std::move(display)] {
+ static_cast<void>(mScheduler->schedule([display = std::move(display)] {
// Destroy the display without holding the mStateLock.
// This is a temporary solution until we can manage transaction queues without
// holding the mStateLock.
@@ -3128,7 +3126,8 @@
const Scheduler::Options options = {
.useContentDetection = sysprop::use_content_detection_for_refresh_rate(false)};
- mScheduler = std::make_unique<Scheduler>(static_cast<ISchedulerCallback&>(*this), options);
+ mScheduler = std::make_unique<Scheduler>(static_cast<ICompositor&>(*this),
+ static_cast<ISchedulerCallback&>(*this), options);
{
auto configs = display->holdRefreshRateConfigs();
mScheduler->createVsyncSchedule(configs->supportsKernelIdleTimer());
@@ -3153,8 +3152,8 @@
mInterceptor->saveVSyncEvent(timestamp);
});
- mEventQueue->initVsync(mScheduler->getVsyncDispatch(), *mFrameTimeline->getTokenManager(),
- configs.late.sfWorkDuration);
+ mScheduler->initVsync(mScheduler->getVsyncDispatch(), *mFrameTimeline->getTokenManager(),
+ configs.late.sfWorkDuration);
mRegionSamplingThread =
new RegionSamplingThread(*this, RegionSamplingThread::EnvironmentTimingTunables());
@@ -3188,7 +3187,7 @@
mScheduler->setDuration(mSfConnectionHandle,
/*workDuration=*/std::chrono::nanoseconds(vsyncPeriod),
/*readyDuration=*/config.sfWorkDuration);
- mEventQueue->setDuration(config.sfWorkDuration);
+ mScheduler->setDuration(config.sfWorkDuration);
}
void SurfaceFlinger::doCommitTransactions() {
@@ -4459,7 +4458,7 @@
void SurfaceFlinger::initializeDisplays() {
// Async since we may be called from the main thread.
- static_cast<void>(schedule([this]() MAIN_THREAD { onInitializeDisplays(); }));
+ static_cast<void>(mScheduler->schedule([this]() MAIN_THREAD { onInitializeDisplays(); }));
}
sp<DisplayDevice> SurfaceFlinger::getDisplayWithInputByLayer(Layer* layer) const {
@@ -4578,7 +4577,7 @@
}
void SurfaceFlinger::setPowerMode(const sp<IBinder>& displayToken, int mode) {
- schedule([=]() MAIN_THREAD {
+ auto future = mScheduler->schedule([=]() MAIN_THREAD {
const auto display = getDisplayDeviceLocked(displayToken);
if (!display) {
ALOGE("Attempt to set power mode %d for invalid display token %p", mode,
@@ -4588,7 +4587,9 @@
} else {
setPowerModeInternal(display, static_cast<hal::PowerMode>(mode));
}
- }).wait();
+ });
+
+ future.wait();
}
status_t SurfaceFlinger::doDump(int fd, const DumpArgs& args, bool asProto) {
@@ -4910,21 +4911,21 @@
}
LayersProto SurfaceFlinger::dumpProtoFromMainThread(uint32_t traceFlags) {
- return schedule([=] { return dumpDrawingStateProto(traceFlags); }).get();
+ return mScheduler->schedule([=] { return dumpDrawingStateProto(traceFlags); }).get();
}
void SurfaceFlinger::dumpOffscreenLayers(std::string& result) {
+ auto future = mScheduler->schedule([this] {
+ std::string result;
+ for (Layer* offscreenLayer : mOffscreenLayers) {
+ offscreenLayer->traverse(LayerVector::StateSet::Drawing,
+ [&](Layer* layer) { layer->dumpCallingUidPid(result); });
+ }
+ return result;
+ });
+
result.append("Offscreen Layers:\n");
- result.append(schedule([this] {
- std::string result;
- for (Layer* offscreenLayer : mOffscreenLayers) {
- offscreenLayer->traverse(LayerVector::StateSet::Drawing,
- [&](Layer* layer) {
- layer->dumpCallingUidPid(result);
- });
- }
- return result;
- }).get());
+ result.append(future.get());
}
void SurfaceFlinger::dumpAllLocked(const DumpArgs& args, std::string& result) const {
@@ -5343,7 +5344,7 @@
return NO_ERROR;
}
case 1006: // Force composite immediately.
- mEventQueue->scheduleComposite();
+ mScheduler->scheduleComposite();
return NO_ERROR;
case 1007: // Unused.
return NAME_NOT_FOUND;
@@ -5455,7 +5456,8 @@
}
case 1021: { // Disable HWC virtual displays
const bool enable = data.readInt32() != 0;
- static_cast<void>(schedule([this, enable] { enableHalVirtualDisplays(enable); }));
+ static_cast<void>(
+ mScheduler->schedule([this, enable] { enableHalVirtualDisplays(enable); }));
return NO_ERROR;
}
case 1022: { // Set saturation boost
@@ -5486,7 +5488,8 @@
ALOGD("LayerTracing enabled");
tracingEnabledChanged = mLayerTracing.enable();
if (tracingEnabledChanged) {
- schedule([&]() MAIN_THREAD { mLayerTracing.notify("start"); }).wait();
+ mScheduler->schedule([&]() MAIN_THREAD { mLayerTracing.notify("start"); })
+ .wait();
}
} else {
ALOGD("LayerTracing disabled");
@@ -5588,7 +5591,7 @@
return NO_ERROR;
}
case 1034: {
- schedule([&] {
+ auto future = mScheduler->schedule([&] {
switch (n = data.readInt32()) {
case 0:
case 1:
@@ -5598,7 +5601,9 @@
reply->writeBool(ON_MAIN_THREAD(isRefreshRateOverlayEnabled()));
}
}
- }).get();
+ });
+
+ future.wait();
return NO_ERROR;
}
case 1035: {
@@ -5628,32 +5633,36 @@
// rates.
case 1036: {
if (data.readInt32() > 0) { // turn on
- return schedule([this] {
- const auto display = ON_MAIN_THREAD(getDefaultDisplayDeviceLocked());
+ return mScheduler
+ ->schedule([this] {
+ const auto display =
+ ON_MAIN_THREAD(getDefaultDisplayDeviceLocked());
- // This is a little racy, but not in a way that hurts anything. As we
- // grab the defaultMode from the display manager policy, we could be
- // setting a new display manager policy, leaving us using a stale
- // defaultMode. The defaultMode doesn't matter for the override
- // policy though, since we set allowGroupSwitching to true, so it's
- // not a problem.
- scheduler::RefreshRateConfigs::Policy overridePolicy;
- overridePolicy.defaultMode = display->refreshRateConfigs()
- .getDisplayManagerPolicy()
- .defaultMode;
- overridePolicy.allowGroupSwitching = true;
- constexpr bool kOverridePolicy = true;
- return setDesiredDisplayModeSpecsInternal(display, overridePolicy,
- kOverridePolicy);
- })
+ // This is a little racy, but not in a way that hurts anything. As
+ // we grab the defaultMode from the display manager policy, we could
+ // be setting a new display manager policy, leaving us using a stale
+ // defaultMode. The defaultMode doesn't matter for the override
+ // policy though, since we set allowGroupSwitching to true, so it's
+ // not a problem.
+ scheduler::RefreshRateConfigs::Policy overridePolicy;
+ overridePolicy.defaultMode = display->refreshRateConfigs()
+ .getDisplayManagerPolicy()
+ .defaultMode;
+ overridePolicy.allowGroupSwitching = true;
+ constexpr bool kOverridePolicy = true;
+ return setDesiredDisplayModeSpecsInternal(display, overridePolicy,
+ kOverridePolicy);
+ })
.get();
} else { // turn off
- return schedule([this] {
- const auto display = ON_MAIN_THREAD(getDefaultDisplayDeviceLocked());
- constexpr bool kOverridePolicy = true;
- return setDesiredDisplayModeSpecsInternal(display, {},
- kOverridePolicy);
- })
+ return mScheduler
+ ->schedule([this] {
+ const auto display =
+ ON_MAIN_THREAD(getDefaultDisplayDeviceLocked());
+ constexpr bool kOverridePolicy = true;
+ return setDesiredDisplayModeSpecsInternal(display, {},
+ kOverridePolicy);
+ })
.get();
}
}
@@ -5698,31 +5707,29 @@
// Second argument is an optional uint64 - if present, then limits enabling/disabling
// caching to a particular physical display
case 1040: {
- status_t error =
- schedule([&] {
- n = data.readInt32();
- std::optional<PhysicalDisplayId> inputId = std::nullopt;
- if (uint64_t inputDisplayId;
- data.readUint64(&inputDisplayId) == NO_ERROR) {
- inputId = DisplayId::fromValue<PhysicalDisplayId>(inputDisplayId);
- if (!inputId || getPhysicalDisplayToken(*inputId)) {
- ALOGE("No display with id: %" PRIu64, inputDisplayId);
- return NAME_NOT_FOUND;
- }
+ auto future = mScheduler->schedule([&] {
+ n = data.readInt32();
+ std::optional<PhysicalDisplayId> inputId = std::nullopt;
+ if (uint64_t inputDisplayId; data.readUint64(&inputDisplayId) == NO_ERROR) {
+ inputId = DisplayId::fromValue<PhysicalDisplayId>(inputDisplayId);
+ if (!inputId || getPhysicalDisplayToken(*inputId)) {
+ ALOGE("No display with id: %" PRIu64, inputDisplayId);
+ return NAME_NOT_FOUND;
+ }
+ }
+ {
+ Mutex::Autolock lock(mStateLock);
+ mLayerCachingEnabled = n != 0;
+ for (const auto& [_, display] : mDisplays) {
+ if (!inputId || *inputId == display->getPhysicalId()) {
+ display->enableLayerCaching(mLayerCachingEnabled);
}
- {
- Mutex::Autolock lock(mStateLock);
- mLayerCachingEnabled = n != 0;
- for (const auto& [_, display] : mDisplays) {
- if (!inputId || *inputId == display->getPhysicalId()) {
- display->enableLayerCaching(mLayerCachingEnabled);
- }
- }
- }
- return OK;
- }).get();
+ }
+ }
+ return OK;
+ });
- if (error != OK) {
+ if (const status_t error = future.get(); error != OK) {
return error;
}
scheduleRepaint();
@@ -5741,7 +5748,7 @@
// Update the overlay on the main thread to avoid race conditions with
// mRefreshRateConfigs->getCurrentRefreshRate()
- static_cast<void>(schedule([=] {
+ static_cast<void>(mScheduler->schedule([=] {
const auto display = ON_MAIN_THREAD(getDefaultDisplayDeviceLocked());
if (!display) {
ALOGW("%s: default display is null", __func__);
@@ -5756,7 +5763,7 @@
const bool timerExpired = mKernelIdleTimerEnabled && expired;
if (display->onKernelTimerChanged(desiredModeId, timerExpired)) {
- mEventQueue->scheduleCommit();
+ mScheduler->scheduleCommit();
}
}));
}
@@ -6148,14 +6155,15 @@
const bool supportsProtected = getRenderEngine().supportsProtectedContent();
bool hasProtectedLayer = false;
if (allowProtected && supportsProtected) {
- hasProtectedLayer = schedule([=]() {
- bool protectedLayerFound = false;
- traverseLayers([&](Layer* layer) {
- protectedLayerFound = protectedLayerFound ||
- (layer->isVisible() && layer->isProtected());
- });
- return protectedLayerFound;
- }).get();
+ auto future = mScheduler->schedule([=]() {
+ bool protectedLayerFound = false;
+ traverseLayers([&](Layer* layer) {
+ protectedLayerFound =
+ protectedLayerFound || (layer->isVisible() && layer->isProtected());
+ });
+ return protectedLayerFound;
+ });
+ hasProtectedLayer = future.get();
}
const uint32_t usage = GRALLOC_USAGE_HW_COMPOSER | GRALLOC_USAGE_HW_RENDER |
@@ -6186,9 +6194,11 @@
bool canCaptureBlackoutContent = hasCaptureBlackoutContentPermission();
- auto scheduleResultFuture = schedule([=,
- renderAreaFuture = std::move(renderAreaFuture)]() mutable
- -> std::shared_future<renderengine::RenderEngineResult> {
+ auto scheduleResultFuture = mScheduler->schedule([=,
+ renderAreaFuture =
+ std::move(renderAreaFuture)]() mutable
+ -> std::shared_future<
+ renderengine::RenderEngineResult> {
ScreenCaptureResults captureResults;
std::unique_ptr<RenderArea> renderArea = renderAreaFuture.get();
if (!renderArea) {
@@ -6481,7 +6491,7 @@
return BAD_VALUE;
}
- auto future = schedule([=]() -> status_t {
+ auto future = mScheduler->schedule([=]() -> status_t {
const auto display = ON_MAIN_THREAD(getDisplayDeviceLocked(displayToken));
if (!display) {
ALOGE("Attempt to set desired display modes for invalid display token %p",
@@ -6618,7 +6628,7 @@
return BAD_VALUE;
}
- static_cast<void>(schedule([=] {
+ static_cast<void>(mScheduler->schedule([=] {
Mutex::Autolock lock(mStateLock);
if (authenticateSurfaceTextureLocked(surface)) {
sp<Layer> layer = (static_cast<MonitoredProducer*>(surface.get()))->getLayer();
@@ -6810,7 +6820,7 @@
return;
}
- mRegionSamplingThread->onCompositionComplete(mEventQueue->getScheduledFrameTime());
+ mRegionSamplingThread->onCompositionComplete(mScheduler->getScheduledFrameTime());
}
void SurfaceFlinger::onActiveDisplaySizeChanged(const sp<DisplayDevice>& activeDisplay) {
diff --git a/services/surfaceflinger/SurfaceFlinger.h b/services/surfaceflinger/SurfaceFlinger.h
index 8897858..2deb9cb 100644
--- a/services/surfaceflinger/SurfaceFlinger.h
+++ b/services/surfaceflinger/SurfaceFlinger.h
@@ -56,7 +56,6 @@
#include "Fps.h"
#include "FrameTracker.h"
#include "LayerVector.h"
-#include "Scheduler/MessageQueue.h"
#include "Scheduler/RefreshRateConfigs.h"
#include "Scheduler/RefreshRateStats.h"
#include "Scheduler/Scheduler.h"
@@ -265,10 +264,6 @@
SurfaceFlingerBE& getBE() { return mBE; }
const SurfaceFlingerBE& getBE() const { return mBE; }
- // Schedule an asynchronous or synchronous task on the main thread.
- template <typename F, typename T = std::invoke_result_t<F>>
- [[nodiscard]] std::future<T> schedule(F&&);
-
// Schedule commit of transactions on the main thread ahead of the next VSYNC.
void scheduleCommit(FrameHint);
// As above, but also force composite regardless if transactions were committed.
@@ -1206,14 +1201,9 @@
TransactionCallbackInvoker mTransactionCallbackInvoker;
- // these are thread safe
- std::unique_ptr<MessageQueue> mEventQueue;
+ // Thread-safe.
FrameTracker mAnimFrameTracker;
- // protected by mDestroyedLayerLock;
- mutable Mutex mDestroyedLayerLock;
- Vector<Layer const *> mDestroyedLayers;
-
// We maintain a pool of pre-generated texture names to hand out to avoid
// layer creation needing to run on the main thread (which it would
// otherwise need to do to access RenderEngine).
diff --git a/services/surfaceflinger/SurfaceFlingerDefaultFactory.cpp b/services/surfaceflinger/SurfaceFlingerDefaultFactory.cpp
index ae21fcc..b81b445 100644
--- a/services/surfaceflinger/SurfaceFlingerDefaultFactory.cpp
+++ b/services/surfaceflinger/SurfaceFlingerDefaultFactory.cpp
@@ -38,7 +38,6 @@
#include "SurfaceInterceptor.h"
#include "DisplayHardware/ComposerHal.h"
-#include "Scheduler/MessageQueue.h"
#include "Scheduler/Scheduler.h"
#include "Scheduler/VsyncConfiguration.h"
#include "Scheduler/VsyncController.h"
@@ -51,10 +50,6 @@
return std::make_unique<android::impl::HWComposer>(serviceName);
}
-std::unique_ptr<MessageQueue> DefaultFactory::createMessageQueue(ICompositor& compositor) {
- return std::make_unique<android::impl::MessageQueue>(compositor);
-}
-
std::unique_ptr<scheduler::VsyncConfiguration> DefaultFactory::createVsyncConfiguration(
Fps currentRefreshRate) {
if (property_get_bool("debug.sf.use_phase_offsets_as_durations", false)) {
diff --git a/services/surfaceflinger/SurfaceFlingerDefaultFactory.h b/services/surfaceflinger/SurfaceFlingerDefaultFactory.h
index 4f70979..501629d 100644
--- a/services/surfaceflinger/SurfaceFlingerDefaultFactory.h
+++ b/services/surfaceflinger/SurfaceFlingerDefaultFactory.h
@@ -27,7 +27,6 @@
virtual ~DefaultFactory();
std::unique_ptr<HWComposer> createHWComposer(const std::string& serviceName) override;
- std::unique_ptr<MessageQueue> createMessageQueue(ICompositor&) override;
std::unique_ptr<scheduler::VsyncConfiguration> createVsyncConfiguration(
Fps currentRefreshRate) override;
sp<SurfaceInterceptor> createSurfaceInterceptor() override;
diff --git a/services/surfaceflinger/SurfaceFlingerFactory.h b/services/surfaceflinger/SurfaceFlingerFactory.h
index a1bf9fe..e670f37 100644
--- a/services/surfaceflinger/SurfaceFlingerFactory.h
+++ b/services/surfaceflinger/SurfaceFlingerFactory.h
@@ -77,7 +77,6 @@
class Factory {
public:
virtual std::unique_ptr<HWComposer> createHWComposer(const std::string& serviceName) = 0;
- virtual std::unique_ptr<MessageQueue> createMessageQueue(ICompositor&) = 0;
virtual std::unique_ptr<scheduler::VsyncConfiguration> createVsyncConfiguration(
Fps currentRefreshRate) = 0;
virtual sp<SurfaceInterceptor> createSurfaceInterceptor() = 0;
diff --git a/services/surfaceflinger/tests/unittests/Android.bp b/services/surfaceflinger/tests/unittests/Android.bp
index 1ac5680..3dc6d8b 100644
--- a/services/surfaceflinger/tests/unittests/Android.bp
+++ b/services/surfaceflinger/tests/unittests/Android.bp
@@ -107,7 +107,6 @@
"mock/MockEventThread.cpp",
"mock/MockFrameTimeline.cpp",
"mock/MockFrameTracer.cpp",
- "mock/MockMessageQueue.cpp",
"mock/MockNativeWindowSurface.cpp",
"mock/MockSurfaceInterceptor.cpp",
"mock/MockTimeStats.cpp",
diff --git a/services/surfaceflinger/tests/unittests/CompositionTest.cpp b/services/surfaceflinger/tests/unittests/CompositionTest.cpp
index dffc6c6..0c9e6e1 100644
--- a/services/surfaceflinger/tests/unittests/CompositionTest.cpp
+++ b/services/surfaceflinger/tests/unittests/CompositionTest.cpp
@@ -45,7 +45,6 @@
#include "mock/DisplayHardware/MockComposer.h"
#include "mock/DisplayHardware/MockPowerAdvisor.h"
#include "mock/MockEventThread.h"
-#include "mock/MockMessageQueue.h"
#include "mock/MockTimeStats.h"
#include "mock/MockVsyncController.h"
#include "mock/system/window/MockNativeWindow.h"
@@ -95,7 +94,6 @@
::testing::UnitTest::GetInstance()->current_test_info();
ALOGD("**** Setting up for %s.%s\n", test_info->test_case_name(), test_info->name());
- mFlinger.mutableEventQueue().reset(mMessageQueue);
setupScheduler();
EXPECT_CALL(*mNativeWindow, query(NATIVE_WINDOW_WIDTH, _))
@@ -183,7 +181,6 @@
Hwc2::mock::Composer* mComposer = nullptr;
renderengine::mock::RenderEngine* mRenderEngine = new renderengine::mock::RenderEngine();
mock::TimeStats* mTimeStats = new mock::TimeStats();
- mock::MessageQueue* mMessageQueue = new mock::MessageQueue();
Hwc2::mock::PowerAdvisor mPowerAdvisor;
sp<Fence> mClientTargetAcquireFence = Fence::NO_FENCE;
@@ -539,9 +536,9 @@
ASSERT_EQ(NO_ERROR, err);
Mock::VerifyAndClear(test->mRenderEngine);
- EXPECT_CALL(*test->mMessageQueue, scheduleCommit()).Times(1);
+ EXPECT_CALL(*test->mFlinger.scheduler(), scheduleCommit()).Times(1);
enqueueBuffer(test, layer);
- Mock::VerifyAndClearExpectations(test->mMessageQueue);
+ Mock::VerifyAndClearExpectations(test->mFlinger.scheduler());
bool ignoredRecomputeVisibleRegions;
layer->latchBuffer(ignoredRecomputeVisibleRegions, 0, 0);
@@ -836,7 +833,7 @@
struct BaseLayerVariant {
template <typename L, typename F>
static sp<L> createLayerWithFactory(CompositionTest* test, F factory) {
- EXPECT_CALL(*test->mMessageQueue, postMessage(_)).Times(0);
+ EXPECT_CALL(*test->mFlinger.scheduler(), postMessage(_)).Times(0);
sp<L> layer = factory();
@@ -845,7 +842,7 @@
Mock::VerifyAndClear(test->mComposer);
Mock::VerifyAndClear(test->mRenderEngine);
- Mock::VerifyAndClearExpectations(test->mMessageQueue);
+ Mock::VerifyAndClearExpectations(test->mFlinger.scheduler());
initLayerDrawingStateAndComputeBounds(test, layer);
diff --git a/services/surfaceflinger/tests/unittests/DisplayTransactionTest.cpp b/services/surfaceflinger/tests/unittests/DisplayTransactionTest.cpp
index 6cb3052..b1f704a 100644
--- a/services/surfaceflinger/tests/unittests/DisplayTransactionTest.cpp
+++ b/services/surfaceflinger/tests/unittests/DisplayTransactionTest.cpp
@@ -50,7 +50,6 @@
});
injectMockScheduler();
- mFlinger.mutableEventQueue().reset(mMessageQueue);
mFlinger.setupRenderEngine(std::unique_ptr<renderengine::RenderEngine>(mRenderEngine));
mFlinger.mutableInterceptor() = mSurfaceInterceptor;
diff --git a/services/surfaceflinger/tests/unittests/DisplayTransactionTestHelpers.h b/services/surfaceflinger/tests/unittests/DisplayTransactionTestHelpers.h
index 7746e73..de5e9df 100644
--- a/services/surfaceflinger/tests/unittests/DisplayTransactionTestHelpers.h
+++ b/services/surfaceflinger/tests/unittests/DisplayTransactionTestHelpers.h
@@ -47,7 +47,6 @@
#include "mock/DisplayHardware/MockComposer.h"
#include "mock/DisplayHardware/MockPowerAdvisor.h"
#include "mock/MockEventThread.h"
-#include "mock/MockMessageQueue.h"
#include "mock/MockNativeWindowSurface.h"
#include "mock/MockSchedulerCallback.h"
#include "mock/MockSurfaceInterceptor.h"
@@ -118,7 +117,6 @@
// to keep a reference to them for use in setting up call expectations.
renderengine::mock::RenderEngine* mRenderEngine = new renderengine::mock::RenderEngine();
Hwc2::mock::Composer* mComposer = nullptr;
- mock::MessageQueue* mMessageQueue = new mock::MessageQueue();
sp<mock::SurfaceInterceptor> mSurfaceInterceptor = new mock::SurfaceInterceptor;
mock::VsyncController* mVsyncController = new mock::VsyncController;
diff --git a/services/surfaceflinger/tests/unittests/SetFrameRateTest.cpp b/services/surfaceflinger/tests/unittests/SetFrameRateTest.cpp
index 84c845d..115a44d 100644
--- a/services/surfaceflinger/tests/unittests/SetFrameRateTest.cpp
+++ b/services/surfaceflinger/tests/unittests/SetFrameRateTest.cpp
@@ -33,7 +33,6 @@
#include "TestableSurfaceFlinger.h"
#include "mock/DisplayHardware/MockComposer.h"
#include "mock/MockEventThread.h"
-#include "mock/MockMessageQueue.h"
#include "mock/MockVsyncController.h"
namespace android {
@@ -112,7 +111,6 @@
void commitTransaction();
TestableSurfaceFlinger mFlinger;
- mock::MessageQueue* mMessageQueue = new mock::MessageQueue();
std::vector<sp<Layer>> mLayers;
};
@@ -125,7 +123,6 @@
setupScheduler();
mFlinger.setupComposer(std::make_unique<Hwc2::mock::Composer>());
- mFlinger.mutableEventQueue().reset(mMessageQueue);
}
void SetFrameRateTest::addChild(sp<Layer> layer, sp<Layer> child) {
@@ -172,7 +169,7 @@
namespace {
TEST_P(SetFrameRateTest, SetAndGet) {
- EXPECT_CALL(*mMessageQueue, scheduleCommit()).Times(1);
+ EXPECT_CALL(*mFlinger.scheduler(), scheduleCommit()).Times(1);
const auto& layerFactory = GetParam();
@@ -183,7 +180,7 @@
}
TEST_P(SetFrameRateTest, SetAndGetParent) {
- EXPECT_CALL(*mMessageQueue, scheduleCommit()).Times(1);
+ EXPECT_CALL(*mFlinger.scheduler(), scheduleCommit()).Times(1);
const auto& layerFactory = GetParam();
@@ -208,7 +205,7 @@
}
TEST_P(SetFrameRateTest, SetAndGetParentAllVote) {
- EXPECT_CALL(*mMessageQueue, scheduleCommit()).Times(1);
+ EXPECT_CALL(*mFlinger.scheduler(), scheduleCommit()).Times(1);
const auto& layerFactory = GetParam();
@@ -247,7 +244,7 @@
}
TEST_P(SetFrameRateTest, SetAndGetChild) {
- EXPECT_CALL(*mMessageQueue, scheduleCommit()).Times(1);
+ EXPECT_CALL(*mFlinger.scheduler(), scheduleCommit()).Times(1);
const auto& layerFactory = GetParam();
@@ -272,7 +269,7 @@
}
TEST_P(SetFrameRateTest, SetAndGetChildAllVote) {
- EXPECT_CALL(*mMessageQueue, scheduleCommit()).Times(1);
+ EXPECT_CALL(*mFlinger.scheduler(), scheduleCommit()).Times(1);
const auto& layerFactory = GetParam();
@@ -311,7 +308,7 @@
}
TEST_P(SetFrameRateTest, SetAndGetChildAddAfterVote) {
- EXPECT_CALL(*mMessageQueue, scheduleCommit()).Times(1);
+ EXPECT_CALL(*mFlinger.scheduler(), scheduleCommit()).Times(1);
const auto& layerFactory = GetParam();
@@ -341,7 +338,7 @@
}
TEST_P(SetFrameRateTest, SetAndGetChildRemoveAfterVote) {
- EXPECT_CALL(*mMessageQueue, scheduleCommit()).Times(1);
+ EXPECT_CALL(*mFlinger.scheduler(), scheduleCommit()).Times(1);
const auto& layerFactory = GetParam();
@@ -372,7 +369,7 @@
}
TEST_P(SetFrameRateTest, SetAndGetParentNotInTree) {
- EXPECT_CALL(*mMessageQueue, scheduleCommit()).Times(1);
+ EXPECT_CALL(*mFlinger.scheduler(), scheduleCommit()).Times(1);
const auto& layerFactory = GetParam();
@@ -467,7 +464,7 @@
}
TEST_P(SetFrameRateTest, addChildForParentWithTreeVote) {
- EXPECT_CALL(*mMessageQueue, scheduleCommit()).Times(1);
+ EXPECT_CALL(*mFlinger.scheduler(), scheduleCommit()).Times(1);
const auto& layerFactory = GetParam();
diff --git a/services/surfaceflinger/tests/unittests/SurfaceFlinger_CreateDisplayTest.cpp b/services/surfaceflinger/tests/unittests/SurfaceFlinger_CreateDisplayTest.cpp
index 8c30341..2236db7 100644
--- a/services/surfaceflinger/tests/unittests/SurfaceFlinger_CreateDisplayTest.cpp
+++ b/services/surfaceflinger/tests/unittests/SurfaceFlinger_CreateDisplayTest.cpp
@@ -52,7 +52,7 @@
// Cleanup conditions
// Creating the display commits a display transaction.
- EXPECT_CALL(*mMessageQueue, scheduleCommit()).Times(1);
+ EXPECT_CALL(*mFlinger.scheduler(), scheduleCommit()).Times(1);
}
TEST_F(CreateDisplayTest, createDisplaySetsCurrentStateForSecureDisplay) {
@@ -87,7 +87,7 @@
// Cleanup conditions
// Creating the display commits a display transaction.
- EXPECT_CALL(*mMessageQueue, scheduleCommit()).Times(1);
+ EXPECT_CALL(*mFlinger.scheduler(), scheduleCommit()).Times(1);
}
} // namespace
diff --git a/services/surfaceflinger/tests/unittests/SurfaceFlinger_DestroyDisplayTest.cpp b/services/surfaceflinger/tests/unittests/SurfaceFlinger_DestroyDisplayTest.cpp
index 7087fb6..bcd3222 100644
--- a/services/surfaceflinger/tests/unittests/SurfaceFlinger_DestroyDisplayTest.cpp
+++ b/services/surfaceflinger/tests/unittests/SurfaceFlinger_DestroyDisplayTest.cpp
@@ -41,7 +41,7 @@
EXPECT_CALL(*mSurfaceInterceptor, saveDisplayDeletion(_)).Times(1);
// Destroying the display commits a display transaction.
- EXPECT_CALL(*mMessageQueue, scheduleCommit()).Times(1);
+ EXPECT_CALL(*mFlinger.scheduler(), scheduleCommit()).Times(1);
// --------------------------------------------------------------------
// Invocation
diff --git a/services/surfaceflinger/tests/unittests/SurfaceFlinger_HotplugTest.cpp b/services/surfaceflinger/tests/unittests/SurfaceFlinger_HotplugTest.cpp
index 29ff0cd..cc979c9 100644
--- a/services/surfaceflinger/tests/unittests/SurfaceFlinger_HotplugTest.cpp
+++ b/services/surfaceflinger/tests/unittests/SurfaceFlinger_HotplugTest.cpp
@@ -39,7 +39,7 @@
// Call Expectations
// We expect a scheduled commit for the display transaction.
- EXPECT_CALL(*mMessageQueue, scheduleCommit()).Times(1);
+ EXPECT_CALL(*mFlinger.scheduler(), scheduleCommit()).Times(1);
// --------------------------------------------------------------------
// Invocation
@@ -86,7 +86,7 @@
// Call Expectations
// We expect a scheduled commit for the display transaction.
- EXPECT_CALL(*mMessageQueue, scheduleCommit()).Times(1);
+ EXPECT_CALL(*mFlinger.scheduler(), scheduleCommit()).Times(1);
// --------------------------------------------------------------------
// Invocation
diff --git a/services/surfaceflinger/tests/unittests/SurfaceFlinger_OnInitializeDisplaysTest.cpp b/services/surfaceflinger/tests/unittests/SurfaceFlinger_OnInitializeDisplaysTest.cpp
index e1b44cf..83b150f 100644
--- a/services/surfaceflinger/tests/unittests/SurfaceFlinger_OnInitializeDisplaysTest.cpp
+++ b/services/surfaceflinger/tests/unittests/SurfaceFlinger_OnInitializeDisplaysTest.cpp
@@ -47,7 +47,7 @@
Case::Display::setupHwcGetActiveConfigCallExpectations(this);
// We expect a scheduled commit for the display transaction.
- EXPECT_CALL(*mMessageQueue, scheduleCommit()).Times(1);
+ EXPECT_CALL(*mFlinger.scheduler(), scheduleCommit()).Times(1);
EXPECT_CALL(*mVSyncTracker, nextAnticipatedVSyncTimeFrom(_)).WillRepeatedly(Return(0));
diff --git a/services/surfaceflinger/tests/unittests/SurfaceFlinger_SetPowerModeInternalTest.cpp b/services/surfaceflinger/tests/unittests/SurfaceFlinger_SetPowerModeInternalTest.cpp
index 6edebd4..ad696aa 100644
--- a/services/surfaceflinger/tests/unittests/SurfaceFlinger_SetPowerModeInternalTest.cpp
+++ b/services/surfaceflinger/tests/unittests/SurfaceFlinger_SetPowerModeInternalTest.cpp
@@ -273,7 +273,7 @@
}
static void setupRepaintEverythingCallExpectations(DisplayTransactionTest* test) {
- EXPECT_CALL(*test->mMessageQueue, scheduleCommit()).Times(1);
+ EXPECT_CALL(*test->mFlinger.scheduler(), scheduleCommit()).Times(1);
}
static void setupSurfaceInterceptorCallExpectations(DisplayTransactionTest* test,
diff --git a/services/surfaceflinger/tests/unittests/TestableScheduler.h b/services/surfaceflinger/tests/unittests/TestableScheduler.h
index 1b850fc..32ec848 100644
--- a/services/surfaceflinger/tests/unittests/TestableScheduler.h
+++ b/services/surfaceflinger/tests/unittests/TestableScheduler.h
@@ -30,7 +30,7 @@
namespace android {
-class TestableScheduler : public Scheduler {
+class TestableScheduler : public Scheduler, private ICompositor {
public:
TestableScheduler(std::shared_ptr<scheduler::RefreshRateConfigs> configs,
ISchedulerCallback& callback)
@@ -42,11 +42,19 @@
std::unique_ptr<scheduler::VSyncTracker> vsyncTracker,
std::shared_ptr<scheduler::RefreshRateConfigs> configs,
ISchedulerCallback& callback)
- : Scheduler(callback, {.useContentDetection = true}) {
+ : Scheduler(*this, callback, {.useContentDetection = true}) {
mVsyncSchedule = {std::move(vsyncController), std::move(vsyncTracker), nullptr};
setRefreshRateConfigs(std::move(configs));
+
+ ON_CALL(*this, postMessage).WillByDefault([](sp<MessageHandler>&& handler) {
+ // Execute task to prevent broken promise exception on destruction.
+ handler->handleMessage(Message());
+ });
}
+ MOCK_METHOD(void, scheduleCommit, (), (override));
+ MOCK_METHOD(void, postMessage, (sp<MessageHandler>&&), (override));
+
// Used to inject mock event thread.
ConnectionHandle createConnection(std::unique_ptr<EventThread> eventThread) {
return Scheduler::createConnection(std::move(eventThread));
@@ -104,6 +112,12 @@
mVsyncSchedule.controller.reset();
mConnections.clear();
}
+
+private:
+ // ICompositor overrides:
+ bool commit(nsecs_t, int64_t, nsecs_t) override { return false; }
+ void composite(nsecs_t) override {}
+ void sample() override {}
};
} // namespace android
diff --git a/services/surfaceflinger/tests/unittests/TestableSurfaceFlinger.h b/services/surfaceflinger/tests/unittests/TestableSurfaceFlinger.h
index c2c4a54..4c5789e 100644
--- a/services/surfaceflinger/tests/unittests/TestableSurfaceFlinger.h
+++ b/services/surfaceflinger/tests/unittests/TestableSurfaceFlinger.h
@@ -73,10 +73,6 @@
return nullptr;
}
- std::unique_ptr<MessageQueue> createMessageQueue(ICompositor& compositor) override {
- return std::make_unique<android::impl::MessageQueue>(compositor);
- }
-
std::unique_ptr<scheduler::VsyncConfiguration> createVsyncConfiguration(
Fps /*currentRefreshRate*/) override {
return std::make_unique<scheduler::FakePhaseOffsets>();
@@ -426,7 +422,6 @@
auto& mutableDisplayColorSetting() { return mFlinger->mDisplayColorSetting; }
auto& mutableDisplays() { return mFlinger->mDisplays; }
auto& mutableDrawingState() { return mFlinger->mDrawingState; }
- auto& mutableEventQueue() { return mFlinger->mEventQueue; }
auto& mutableGeometryDirty() { return mFlinger->mGeometryDirty; }
auto& mutableInterceptor() { return mFlinger->mInterceptor; }
auto& mutableMainThreadId() { return mFlinger->mMainThreadId; }
@@ -455,7 +450,6 @@
mutableDisplays().clear();
mutableCurrentState().displays.clear();
mutableDrawingState().displays.clear();
- mutableEventQueue().reset();
mutableInterceptor().clear();
mFlinger->mScheduler.reset();
mFlinger->mCompositionEngine->setHwComposer(std::unique_ptr<HWComposer>());
diff --git a/services/surfaceflinger/tests/unittests/TransactionApplicationTest.cpp b/services/surfaceflinger/tests/unittests/TransactionApplicationTest.cpp
index 8caadfb..b3a6a1b 100644
--- a/services/surfaceflinger/tests/unittests/TransactionApplicationTest.cpp
+++ b/services/surfaceflinger/tests/unittests/TransactionApplicationTest.cpp
@@ -29,7 +29,6 @@
#include "TestableScheduler.h"
#include "TestableSurfaceFlinger.h"
#include "mock/MockEventThread.h"
-#include "mock/MockMessageQueue.h"
#include "mock/MockVsyncController.h"
namespace android {
@@ -46,7 +45,6 @@
::testing::UnitTest::GetInstance()->current_test_info();
ALOGD("**** Setting up for %s.%s\n", test_info->test_case_name(), test_info->name());
- mFlinger.mutableEventQueue().reset(mMessageQueue);
setupScheduler();
}
@@ -92,7 +90,6 @@
std::unique_ptr<mock::EventThread> mEventThread = std::make_unique<mock::EventThread>();
- mock::MessageQueue* mMessageQueue = new mock::MessageQueue();
mock::VsyncController* mVsyncController = new mock::VsyncController();
mock::VSyncTracker* mVSyncTracker = new mock::VSyncTracker();
mock::MockFence* mFenceUnsignaled = new mock::MockFence();
@@ -146,7 +143,7 @@
void NotPlacedOnTransactionQueue(uint32_t flags, bool syncInputWindows) {
ASSERT_EQ(0u, mFlinger.getTransactionQueue().size());
- EXPECT_CALL(*mMessageQueue, scheduleCommit()).Times(1);
+ EXPECT_CALL(*mFlinger.scheduler(), scheduleCommit()).Times(1);
TransactionInfo transaction;
setupSingle(transaction, flags, syncInputWindows,
/*desiredPresentTime*/ systemTime(), /*isAutoTimestamp*/ true,
@@ -176,7 +173,7 @@
void PlaceOnTransactionQueue(uint32_t flags, bool syncInputWindows) {
ASSERT_EQ(0u, mFlinger.getTransactionQueue().size());
- EXPECT_CALL(*mMessageQueue, scheduleCommit()).Times(1);
+ EXPECT_CALL(*mFlinger.scheduler(), scheduleCommit()).Times(1);
// first check will see desired present time has not passed,
// but afterwards it will look like the desired present time has passed
@@ -207,9 +204,9 @@
ASSERT_EQ(0u, mFlinger.getTransactionQueue().size());
nsecs_t time = systemTime();
if (!syncInputWindows) {
- EXPECT_CALL(*mMessageQueue, scheduleCommit()).Times(2);
+ EXPECT_CALL(*mFlinger.scheduler(), scheduleCommit()).Times(2);
} else {
- EXPECT_CALL(*mMessageQueue, scheduleCommit()).Times(1);
+ EXPECT_CALL(*mFlinger.scheduler(), scheduleCommit()).Times(1);
}
// transaction that should go on the pending thread
TransactionInfo transactionA;
@@ -454,7 +451,7 @@
TEST_F(TransactionApplicationTest, Flush_RemovesFromQueue) {
ASSERT_EQ(0u, mFlinger.getTransactionQueue().size());
- EXPECT_CALL(*mMessageQueue, scheduleCommit()).Times(1);
+ EXPECT_CALL(*mFlinger.scheduler(), scheduleCommit()).Times(1);
TransactionInfo transactionA; // transaction to go on pending queue
setupSingle(transactionA, /*flags*/ 0, /*syncInputWindows*/ false,
diff --git a/services/surfaceflinger/tests/unittests/TunnelModeEnabledReporterTest.cpp b/services/surfaceflinger/tests/unittests/TunnelModeEnabledReporterTest.cpp
index ade4fbb..15fea9c 100644
--- a/services/surfaceflinger/tests/unittests/TunnelModeEnabledReporterTest.cpp
+++ b/services/surfaceflinger/tests/unittests/TunnelModeEnabledReporterTest.cpp
@@ -27,7 +27,6 @@
#include "TunnelModeEnabledReporter.h"
#include "mock/DisplayHardware/MockComposer.h"
#include "mock/MockEventThread.h"
-#include "mock/MockMessageQueue.h"
namespace android {
@@ -69,12 +68,12 @@
TestableSurfaceFlinger mFlinger;
Hwc2::mock::Composer* mComposer = nullptr;
- sp<TestableTunnelModeEnabledListener> mTunnelModeEnabledListener =
- new TestableTunnelModeEnabledListener();
- sp<TunnelModeEnabledReporter> mTunnelModeEnabledReporter =
- new TunnelModeEnabledReporter();
- mock::MessageQueue* mMessageQueue = new mock::MessageQueue();
+ sp<TestableTunnelModeEnabledListener> mTunnelModeEnabledListener =
+ sp<TestableTunnelModeEnabledListener>::make();
+
+ sp<TunnelModeEnabledReporter> mTunnelModeEnabledReporter =
+ sp<TunnelModeEnabledReporter>::make();
};
TunnelModeEnabledReporterTest::TunnelModeEnabledReporterTest() {
@@ -82,7 +81,6 @@
::testing::UnitTest::GetInstance()->current_test_info();
ALOGD("**** Setting up for %s.%s\n", test_info->test_case_name(), test_info->name());
- mFlinger.mutableEventQueue().reset(mMessageQueue);
setupScheduler();
mFlinger.setupComposer(std::make_unique<Hwc2::mock::Composer>());
mFlinger.flinger()->mTunnelModeEnabledReporter = mTunnelModeEnabledReporter;
diff --git a/services/surfaceflinger/tests/unittests/mock/MockMessageQueue.cpp b/services/surfaceflinger/tests/unittests/mock/MockMessageQueue.cpp
deleted file mode 100644
index 5fb06fd..0000000
--- a/services/surfaceflinger/tests/unittests/mock/MockMessageQueue.cpp
+++ /dev/null
@@ -1,30 +0,0 @@
-/*
- * Copyright (C) 2018 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.
- */
-
-#include "mock/MockMessageQueue.h"
-
-namespace android::mock {
-
-MessageQueue::MessageQueue() {
- ON_CALL(*this, postMessage).WillByDefault([](sp<MessageHandler>&& handler) {
- // Execute task to prevent broken promise exception on destruction.
- handler->handleMessage(Message());
- });
-}
-
-MessageQueue::~MessageQueue() = default;
-
-} // namespace android::mock
diff --git a/services/surfaceflinger/tests/unittests/mock/MockMessageQueue.h b/services/surfaceflinger/tests/unittests/mock/MockMessageQueue.h
deleted file mode 100644
index d684337..0000000
--- a/services/surfaceflinger/tests/unittests/mock/MockMessageQueue.h
+++ /dev/null
@@ -1,46 +0,0 @@
-/*
- * Copyright (C) 2018 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 <gmock/gmock.h>
-
-#include "FrameTimeline.h"
-#include "Scheduler/EventThread.h"
-#include "Scheduler/MessageQueue.h"
-
-namespace android::mock {
-
-class MessageQueue : public android::MessageQueue {
-public:
- MessageQueue();
- ~MessageQueue() override;
-
- MOCK_METHOD1(setInjector, void(sp<EventThreadConnection>));
- MOCK_METHOD0(waitMessage, void());
- MOCK_METHOD1(postMessage, void(sp<MessageHandler>&&));
- MOCK_METHOD3(initVsync,
- void(scheduler::VSyncDispatch&, frametimeline::TokenManager&,
- std::chrono::nanoseconds));
- MOCK_METHOD1(setDuration, void(std::chrono::nanoseconds workDuration));
-
- MOCK_METHOD(void, scheduleCommit, (), (override));
- MOCK_METHOD(void, scheduleComposite, (), (override));
-
- MOCK_METHOD(std::optional<Clock::time_point>, getScheduledFrameTime, (), (const, override));
-};
-
-} // namespace android::mock