From 2260067fae5a70fbca7653677c27166050af175b Mon Sep 17 00:00:00 2001 From: Kaushik Iska Date: Mon, 13 Apr 2020 15:25:28 -0700 Subject: [PATCH 1/4] Remove pipeline in favor of layer tree holder go/flutter-pipeline-improvements for more details --- ci/licenses_golden/licenses_flutter | 3 - shell/common/BUILD.gn | 5 +- shell/common/animator.cc | 45 +----- shell/common/animator.h | 10 +- shell/common/layer_tree_holder.cc | 28 ++++ shell/common/layer_tree_holder.h | 35 +++++ shell/common/pipeline.cc | 14 -- shell/common/pipeline.h | 215 ---------------------------- shell/common/pipeline_unittests.cc | 134 ----------------- shell/common/rasterizer.cc | 39 ++--- shell/common/rasterizer.h | 6 +- shell/common/shell.cc | 7 +- shell/common/shell.h | 3 +- 13 files changed, 95 insertions(+), 449 deletions(-) create mode 100644 shell/common/layer_tree_holder.cc create mode 100644 shell/common/layer_tree_holder.h delete mode 100644 shell/common/pipeline.cc delete mode 100644 shell/common/pipeline.h delete mode 100644 shell/common/pipeline_unittests.cc diff --git a/ci/licenses_golden/licenses_flutter b/ci/licenses_golden/licenses_flutter index 65464989fbf6f..00ade5c06a1e7 100755 --- a/ci/licenses_golden/licenses_flutter +++ b/ci/licenses_golden/licenses_flutter @@ -579,9 +579,6 @@ FILE: ../../../flutter/shell/common/isolate_configuration.h FILE: ../../../flutter/shell/common/persistent_cache.cc FILE: ../../../flutter/shell/common/persistent_cache.h FILE: ../../../flutter/shell/common/persistent_cache_unittests.cc -FILE: ../../../flutter/shell/common/pipeline.cc -FILE: ../../../flutter/shell/common/pipeline.h -FILE: ../../../flutter/shell/common/pipeline_unittests.cc FILE: ../../../flutter/shell/common/platform_view.cc FILE: ../../../flutter/shell/common/platform_view.h FILE: ../../../flutter/shell/common/pointer_data_dispatcher.cc diff --git a/shell/common/BUILD.gn b/shell/common/BUILD.gn index e9e5965dffda0..a201ef74c41c1 100644 --- a/shell/common/BUILD.gn +++ b/shell/common/BUILD.gn @@ -87,10 +87,10 @@ source_set("common") { "engine.h", "isolate_configuration.cc", "isolate_configuration.h", + "layer_tree_holder.cc", + "layer_tree_holder.h", "persistent_cache.cc", "persistent_cache.h", - "pipeline.cc", - "pipeline.h", "platform_view.cc", "platform_view.h", "pointer_data_dispatcher.cc", @@ -192,7 +192,6 @@ if (enable_unittests) { "canvas_spy_unittests.cc", "input_events_unittests.cc", "persistent_cache_unittests.cc", - "pipeline_unittests.cc", "renderer_context_manager_unittests.cc", "renderer_context_test.cc", "renderer_context_test.h", diff --git a/shell/common/animator.cc b/shell/common/animator.cc index 004069bd8cfe2..42da0143c34f2 100644 --- a/shell/common/animator.cc +++ b/shell/common/animator.cc @@ -3,6 +3,7 @@ // found in the LICENSE file. #include "flutter/shell/common/animator.h" +#include #include "flutter/fml/trace_event.h" #include "third_party/dart/runtime/include/dart_tools_api.h" @@ -28,18 +29,7 @@ Animator::Animator(Delegate& delegate, last_frame_begin_time_(), last_frame_target_time_(), dart_frame_deadline_(0), -#if FLUTTER_SHELL_ENABLE_METAL - layer_tree_pipeline_(fml::MakeRefCounted(2)), -#else // FLUTTER_SHELL_ENABLE_METAL - // TODO(dnfield): We should remove this logic and set the pipeline depth - // back to 2 in this case. See - // https://github.com/flutter/engine/pull/9132 for discussion. - layer_tree_pipeline_(fml::MakeRefCounted( - task_runners.GetPlatformTaskRunner() == - task_runners.GetRasterTaskRunner() - ? 1 - : 2)), -#endif // FLUTTER_SHELL_ENABLE_METAL + layer_tree_holder_(std::make_shared()), pending_frame_semaphore_(1), frame_number_(1), paused_(false), @@ -47,8 +37,7 @@ Animator::Animator(Delegate& delegate, frame_scheduled_(false), notify_idle_task_id_(0), dimension_change_pending_(false), - weak_factory_(this) { -} + weak_factory_(this) {} Animator::~Animator() = default; @@ -114,25 +103,6 @@ void Animator::BeginFrame(fml::TimePoint frame_start_time, regenerate_layer_tree_ = false; pending_frame_semaphore_.Signal(); - if (!producer_continuation_) { - // We may already have a valid pipeline continuation in case a previous - // begin frame did not result in an Animation::Render. Simply reuse that - // instead of asking the pipeline for a fresh continuation. - producer_continuation_ = layer_tree_pipeline_->Produce(); - - if (!producer_continuation_) { - // If we still don't have valid continuation, the pipeline is currently - // full because the consumer is being too slow. Try again at the next - // frame interval. - RequestFrame(); - return; - } - } - - // We have acquired a valid continuation from the pipeline and are ready - // to service potential frame. - FML_DCHECK(producer_continuation_); - last_frame_begin_time_ = frame_start_time; last_frame_target_time_ = frame_target_time; dart_frame_deadline_ = FxlToDartOrEarlier(frame_target_time); @@ -184,13 +154,8 @@ void Animator::Render(std::unique_ptr layer_tree) { last_frame_target_time_); } - // Commit the pending continuation. - bool result = producer_continuation_.Complete(std::move(layer_tree)); - if (!result) { - FML_DLOG(INFO) << "No pending continuation to commit"; - } - - delegate_.OnAnimatorDraw(layer_tree_pipeline_); + layer_tree_holder_->ReplaceIfNewer(std::move(layer_tree)); + delegate_.OnAnimatorDraw(layer_tree_holder_); } bool Animator::CanReuseLastLayerTree() { diff --git a/shell/common/animator.h b/shell/common/animator.h index f96acb7e9a3a0..1f7876fc69d2d 100644 --- a/shell/common/animator.h +++ b/shell/common/animator.h @@ -6,13 +6,14 @@ #define FLUTTER_SHELL_COMMON_ANIMATOR_H_ #include +#include #include "flutter/common/task_runners.h" #include "flutter/fml/memory/ref_ptr.h" #include "flutter/fml/memory/weak_ptr.h" #include "flutter/fml/synchronization/semaphore.h" #include "flutter/fml/time/time_point.h" -#include "flutter/shell/common/pipeline.h" +#include "flutter/shell/common/layer_tree_holder.h" #include "flutter/shell/common/rasterizer.h" #include "flutter/shell/common/vsync_waiter.h" @@ -35,7 +36,7 @@ class Animator final { virtual void OnAnimatorNotifyIdle(int64_t deadline) = 0; virtual void OnAnimatorDraw( - fml::RefPtr> pipeline) = 0; + std::shared_ptr layer_tree_holder) = 0; virtual void OnAnimatorDrawLastLayerTree() = 0; }; @@ -80,8 +81,6 @@ class Animator final { void EnqueueTraceFlowId(uint64_t trace_flow_id); private: - using LayerTreePipeline = Pipeline; - void BeginFrame(fml::TimePoint frame_start_time, fml::TimePoint frame_target_time); @@ -99,9 +98,8 @@ class Animator final { fml::TimePoint last_frame_begin_time_; fml::TimePoint last_frame_target_time_; int64_t dart_frame_deadline_; - fml::RefPtr layer_tree_pipeline_; + std::shared_ptr layer_tree_holder_; fml::Semaphore pending_frame_semaphore_; - LayerTreePipeline::ProducerContinuation producer_continuation_; int64_t frame_number_; bool paused_; bool regenerate_layer_tree_; diff --git a/shell/common/layer_tree_holder.cc b/shell/common/layer_tree_holder.cc new file mode 100644 index 0000000000000..6d0e7d75106ab --- /dev/null +++ b/shell/common/layer_tree_holder.cc @@ -0,0 +1,28 @@ +// Copyright 2013 The Flutter Authors. All rights reserved. +// Use of this source code is governed by a BSD-style license that can be +// found in the LICENSE file. + +#include "flutter/shell/common/layer_tree_holder.h" + +namespace flutter { + +std::unique_ptr LayerTreeHolder::Get() { + std::scoped_lock lock(layer_tree_mutex); + return std::move(layer_tree_); +} + +void LayerTreeHolder::ReplaceIfNewer(std::unique_ptr layer_tree) { + std::scoped_lock lock(layer_tree_mutex); + if (IsEmpty()) { + layer_tree_ = std::move(layer_tree); + } else if (layer_tree_->target_time() < layer_tree->target_time()) { + layer_tree_ = std::move(layer_tree); + } +} + +bool LayerTreeHolder::IsEmpty() const { + std::scoped_lock lock(layer_tree_mutex); + return !layer_tree_; +} + +}; // namespace flutter diff --git a/shell/common/layer_tree_holder.h b/shell/common/layer_tree_holder.h new file mode 100644 index 0000000000000..3666fd302502a --- /dev/null +++ b/shell/common/layer_tree_holder.h @@ -0,0 +1,35 @@ +// Copyright 2013 The Flutter Authors. All rights reserved. +// Use of this source code is governed by a BSD-style license that can be +// found in the LICENSE file. + +#ifndef FLUTTER_SHELL_COMMON_LAYER_TREE_HOLDER_H_ +#define FLUTTER_SHELL_COMMON_LAYER_TREE_HOLDER_H_ + +#include + +#include "flow/layers/layer_tree.h" + +namespace flutter { + +class LayerTreeHolder { + public: + LayerTreeHolder() = default; + + ~LayerTreeHolder() = default; + + bool IsEmpty() const; + + std::unique_ptr Get(); + + void ReplaceIfNewer(std::unique_ptr layer_tree); + + private: + mutable std::mutex layer_tree_mutex; + std::unique_ptr layer_tree_; + + FML_DISALLOW_COPY_AND_ASSIGN(LayerTreeHolder); +}; + +}; // namespace flutter + +#endif // FLUTTER_SHELL_COMMON_LAYER_TREE_HOLDER_H_ diff --git a/shell/common/pipeline.cc b/shell/common/pipeline.cc deleted file mode 100644 index fb80c187181fd..0000000000000 --- a/shell/common/pipeline.cc +++ /dev/null @@ -1,14 +0,0 @@ -// Copyright 2013 The Flutter Authors. All rights reserved. -// Use of this source code is governed by a BSD-style license that can be -// found in the LICENSE file. - -#include "flutter/shell/common/pipeline.h" - -namespace flutter { - -size_t GetNextPipelineTraceID() { - static std::atomic_size_t PipelineLastTraceID = {0}; - return ++PipelineLastTraceID; -} - -} // namespace flutter diff --git a/shell/common/pipeline.h b/shell/common/pipeline.h deleted file mode 100644 index c225d9a499990..0000000000000 --- a/shell/common/pipeline.h +++ /dev/null @@ -1,215 +0,0 @@ -// Copyright 2013 The Flutter Authors. All rights reserved. -// Use of this source code is governed by a BSD-style license that can be -// found in the LICENSE file. - -#ifndef FLUTTER_SHELL_COMMON_PIPELINE_H_ -#define FLUTTER_SHELL_COMMON_PIPELINE_H_ - -#include "flutter/fml/macros.h" -#include "flutter/fml/memory/ref_counted.h" -#include "flutter/fml/synchronization/semaphore.h" -#include "flutter/fml/trace_event.h" - -#include -#include -#include - -namespace flutter { - -enum class PipelineConsumeResult { - NoneAvailable, - Done, - MoreAvailable, -}; - -size_t GetNextPipelineTraceID(); - -/// A thread-safe queue of resources for a single consumer and a single -/// producer. -template -class Pipeline : public fml::RefCountedThreadSafe> { - public: - using Resource = R; - using ResourcePtr = std::unique_ptr; - - /// Denotes a spot in the pipeline reserved for the producer to finish - /// preparing a completed pipeline resource. - class ProducerContinuation { - public: - ProducerContinuation() : trace_id_(0) {} - - ProducerContinuation(ProducerContinuation&& other) - : continuation_(other.continuation_), trace_id_(other.trace_id_) { - other.continuation_ = nullptr; - other.trace_id_ = 0; - } - - ProducerContinuation& operator=(ProducerContinuation&& other) { - std::swap(continuation_, other.continuation_); - std::swap(trace_id_, other.trace_id_); - return *this; - } - - ~ProducerContinuation() { - if (continuation_) { - continuation_(nullptr, trace_id_); - TRACE_EVENT_ASYNC_END0("flutter", "PipelineProduce", trace_id_); - // The continuation is being dropped on the floor. End the flow. - TRACE_FLOW_END("flutter", "PipelineItem", trace_id_); - TRACE_EVENT_ASYNC_END0("flutter", "PipelineItem", trace_id_); - } - } - - [[nodiscard]] bool Complete(ResourcePtr resource) { - bool result = false; - if (continuation_) { - result = continuation_(std::move(resource), trace_id_); - continuation_ = nullptr; - TRACE_EVENT_ASYNC_END0("flutter", "PipelineProduce", trace_id_); - TRACE_FLOW_STEP("flutter", "PipelineItem", trace_id_); - } - return result; - } - - operator bool() const { return continuation_ != nullptr; } - - private: - friend class Pipeline; - using Continuation = std::function; - - Continuation continuation_; - size_t trace_id_; - - ProducerContinuation(const Continuation& continuation, size_t trace_id) - : continuation_(continuation), trace_id_(trace_id) { - TRACE_FLOW_BEGIN("flutter", "PipelineItem", trace_id_); - TRACE_EVENT_ASYNC_BEGIN0("flutter", "PipelineItem", trace_id_); - TRACE_EVENT_ASYNC_BEGIN0("flutter", "PipelineProduce", trace_id_); - } - - FML_DISALLOW_COPY_AND_ASSIGN(ProducerContinuation); - }; - - explicit Pipeline(uint32_t depth) - : depth_(depth), empty_(depth), available_(0), inflight_(0) {} - - ~Pipeline() = default; - - bool IsValid() const { return empty_.IsValid() && available_.IsValid(); } - - ProducerContinuation Produce() { - if (!empty_.TryWait()) { - return {}; - } - ++inflight_; - FML_TRACE_COUNTER("flutter", "Pipeline Depth", - reinterpret_cast(this), // - "frames in flight", inflight_.load() // - ); - - return ProducerContinuation{ - std::bind(&Pipeline::ProducerCommit, this, std::placeholders::_1, - std::placeholders::_2), // continuation - GetNextPipelineTraceID()}; // trace id - } - - // Create a `ProducerContinuation` that will only push the task if the queue - // is empty. - // Prefer using |Produce|. ProducerContinuation returned by this method - // doesn't guarantee that the frame will be rendered. - ProducerContinuation ProduceIfEmpty() { - if (!empty_.TryWait()) { - return {}; - } - ++inflight_; - FML_TRACE_COUNTER("flutter", "Pipeline Depth", - reinterpret_cast(this), // - "frames in flight", inflight_.load() // - ); - - return ProducerContinuation{ - std::bind(&Pipeline::ProducerCommitIfEmpty, this, std::placeholders::_1, - std::placeholders::_2), // continuation - GetNextPipelineTraceID()}; // trace id - } - - using Consumer = std::function; - - /// @note Procedure doesn't copy all closures. - [[nodiscard]] PipelineConsumeResult Consume(const Consumer& consumer) { - if (consumer == nullptr) { - return PipelineConsumeResult::NoneAvailable; - } - - if (!available_.TryWait()) { - return PipelineConsumeResult::NoneAvailable; - } - - ResourcePtr resource; - size_t trace_id = 0; - size_t items_count = 0; - - { - std::scoped_lock lock(queue_mutex_); - std::tie(resource, trace_id) = std::move(queue_.front()); - queue_.pop_front(); - items_count = queue_.size(); - } - - { - TRACE_EVENT0("flutter", "PipelineConsume"); - consumer(std::move(resource)); - } - - empty_.Signal(); - --inflight_; - - TRACE_FLOW_END("flutter", "PipelineItem", trace_id); - TRACE_EVENT_ASYNC_END0("flutter", "PipelineItem", trace_id); - - return items_count > 0 ? PipelineConsumeResult::MoreAvailable - : PipelineConsumeResult::Done; - } - - private: - const uint32_t depth_; - fml::Semaphore empty_; - fml::Semaphore available_; - std::atomic inflight_; - std::mutex queue_mutex_; - std::deque> queue_; - - bool ProducerCommit(ResourcePtr resource, size_t trace_id) { - { - std::scoped_lock lock(queue_mutex_); - queue_.emplace_back(std::move(resource), trace_id); - } - - // Ensure the queue mutex is not held as that would be a pessimization. - available_.Signal(); - return true; - } - - bool ProducerCommitIfEmpty(ResourcePtr resource, size_t trace_id) { - { - std::scoped_lock lock(queue_mutex_); - if (!queue_.empty()) { - // Bail if the queue is not empty, opens up spaces to produce other - // frames. - empty_.Signal(); - return false; - } - queue_.emplace_back(std::move(resource), trace_id); - } - - // Ensure the queue mutex is not held as that would be a pessimization. - available_.Signal(); - return true; - } - - FML_DISALLOW_COPY_AND_ASSIGN(Pipeline); -}; - -} // namespace flutter - -#endif // FLUTTER_SHELL_COMMON_PIPELINE_H_ diff --git a/shell/common/pipeline_unittests.cc b/shell/common/pipeline_unittests.cc deleted file mode 100644 index d9cce5ac81a7b..0000000000000 --- a/shell/common/pipeline_unittests.cc +++ /dev/null @@ -1,134 +0,0 @@ -// Copyright 2013 The Flutter Authors. All rights reserved. -// Use of this source code is governed by a BSD-style license that can be -// found in the LICENSE file. - -#define FML_USED_ON_EMBEDDER - -#include -#include -#include - -#include "flutter/shell/common/pipeline.h" -#include "gtest/gtest.h" - -namespace flutter { -namespace testing { - -using IntPipeline = Pipeline; -using Continuation = IntPipeline::ProducerContinuation; - -TEST(PipelineTest, ConsumeOneVal) { - fml::RefPtr pipeline = fml::MakeRefCounted(2); - - Continuation continuation = pipeline->Produce(); - - const int test_val = 1; - bool result = continuation.Complete(std::make_unique(test_val)); - ASSERT_EQ(result, true); - - PipelineConsumeResult consume_result = pipeline->Consume( - [&test_val](std::unique_ptr v) { ASSERT_EQ(*v, test_val); }); - - ASSERT_EQ(consume_result, PipelineConsumeResult::Done); -} - -TEST(PipelineTest, ContinuationCanOnlyBeUsedOnce) { - fml::RefPtr pipeline = fml::MakeRefCounted(2); - - Continuation continuation = pipeline->Produce(); - - const int test_val = 1; - bool result = continuation.Complete(std::make_unique(test_val)); - ASSERT_EQ(result, true); - - PipelineConsumeResult consume_result_1 = pipeline->Consume( - [&test_val](std::unique_ptr v) { ASSERT_EQ(*v, test_val); }); - - result = continuation.Complete(std::make_unique(test_val)); - ASSERT_EQ(result, false); - ASSERT_EQ(consume_result_1, PipelineConsumeResult::Done); - - PipelineConsumeResult consume_result_2 = - pipeline->Consume([](std::unique_ptr v) { FAIL(); }); - - result = continuation.Complete(std::make_unique(test_val)); - ASSERT_EQ(result, false); - ASSERT_EQ(consume_result_2, PipelineConsumeResult::NoneAvailable); -} - -TEST(PipelineTest, PushingMoreThanDepthCompletesFirstSubmission) { - const int depth = 1; - fml::RefPtr pipeline = fml::MakeRefCounted(depth); - - Continuation continuation_1 = pipeline->Produce(); - Continuation continuation_2 = pipeline->Produce(); - - const int test_val_1 = 1, test_val_2 = 2; - bool result = continuation_1.Complete(std::make_unique(test_val_1)); - ASSERT_EQ(result, true); - result = continuation_2.Complete(std::make_unique(test_val_2)); - ASSERT_EQ(result, false); - - PipelineConsumeResult consume_result_1 = pipeline->Consume( - [&test_val_1](std::unique_ptr v) { ASSERT_EQ(*v, test_val_1); }); - - ASSERT_EQ(consume_result_1, PipelineConsumeResult::Done); -} - -TEST(PipelineTest, PushingMultiProcessesInOrder) { - const int depth = 2; - fml::RefPtr pipeline = fml::MakeRefCounted(depth); - - Continuation continuation_1 = pipeline->Produce(); - Continuation continuation_2 = pipeline->Produce(); - - const int test_val_1 = 1, test_val_2 = 2; - bool result = continuation_1.Complete(std::make_unique(test_val_1)); - ASSERT_EQ(result, true); - result = continuation_2.Complete(std::make_unique(test_val_2)); - ASSERT_EQ(result, true); - - PipelineConsumeResult consume_result_1 = pipeline->Consume( - [&test_val_1](std::unique_ptr v) { ASSERT_EQ(*v, test_val_1); }); - ASSERT_EQ(consume_result_1, PipelineConsumeResult::MoreAvailable); - - PipelineConsumeResult consume_result_2 = pipeline->Consume( - [&test_val_2](std::unique_ptr v) { ASSERT_EQ(*v, test_val_2); }); - ASSERT_EQ(consume_result_2, PipelineConsumeResult::Done); -} - -TEST(PipelineTest, ProduceIfEmptyDoesNotConsumeWhenQueueIsNotEmpty) { - const int depth = 2; - fml::RefPtr pipeline = fml::MakeRefCounted(depth); - - Continuation continuation_1 = pipeline->Produce(); - Continuation continuation_2 = pipeline->ProduceIfEmpty(); - - const int test_val_1 = 1, test_val_2 = 2; - bool result = continuation_1.Complete(std::make_unique(test_val_1)); - ASSERT_EQ(result, true); - result = continuation_2.Complete(std::make_unique(test_val_2)); - ASSERT_EQ(result, false); - - PipelineConsumeResult consume_result_1 = pipeline->Consume( - [&test_val_1](std::unique_ptr v) { ASSERT_EQ(*v, test_val_1); }); - ASSERT_EQ(consume_result_1, PipelineConsumeResult::Done); -} - -TEST(PipelineTest, ProduceIfEmptySuccessfulIfQueueIsEmpty) { - const int depth = 1; - fml::RefPtr pipeline = fml::MakeRefCounted(depth); - - Continuation continuation_1 = pipeline->ProduceIfEmpty(); - - const int test_val_1 = 1; - bool result = continuation_1.Complete(std::make_unique(test_val_1)); - ASSERT_EQ(result, true); - - PipelineConsumeResult consume_result_1 = pipeline->Consume( - [&test_val_1](std::unique_ptr v) { ASSERT_EQ(*v, test_val_1); }); - ASSERT_EQ(consume_result_1, PipelineConsumeResult::Done); -} - -} // namespace testing -} // namespace flutter diff --git a/shell/common/rasterizer.cc b/shell/common/rasterizer.cc index 74329c941b94a..0181ec6e7b6d1 100644 --- a/shell/common/rasterizer.cc +++ b/shell/common/rasterizer.cc @@ -111,7 +111,7 @@ void Rasterizer::DrawLastLayerTree() { DrawToSurface(*last_layer_tree_); } -void Rasterizer::Draw(fml::RefPtr> pipeline) { +void Rasterizer::Draw(std::shared_ptr layer_tree_holder) { TRACE_EVENT0("flutter", "GPURasterizer::Draw"); if (raster_thread_merger_ && !raster_thread_merger_->IsOnRasterizingThread()) { @@ -120,24 +120,16 @@ void Rasterizer::Draw(fml::RefPtr> pipeline) { } FML_DCHECK(task_runners_.GetRasterTaskRunner()->RunsTasksOnCurrentThread()); + std::unique_ptr layer_tree = layer_tree_holder->Get(); RasterStatus raster_status = RasterStatus::kFailed; - Pipeline::Consumer consumer = - [&](std::unique_ptr layer_tree) { - raster_status = DoDraw(std::move(layer_tree)); - }; + if (layer_tree) { + raster_status = DoDraw(std::move(layer_tree)); + } - PipelineConsumeResult consume_result = pipeline->Consume(consumer); // if the raster status is to resubmit the frame, we push the frame to the // front of the queue and also change the consume status to more available. if (raster_status == RasterStatus::kResubmit) { - auto front_continuation = pipeline->ProduceIfEmpty(); - bool result = - front_continuation.Complete(std::move(resubmitted_layer_tree_)); - if (result) { - consume_result = PipelineConsumeResult::MoreAvailable; - } - } else if (raster_status == RasterStatus::kEnqueuePipeline) { - consume_result = PipelineConsumeResult::MoreAvailable; + layer_tree_holder->ReplaceIfNewer(std::move(resubmitted_layer_tree_)); } // Merging the thread as we know the next `Draw` should be run on the platform @@ -152,18 +144,13 @@ void Rasterizer::Draw(fml::RefPtr> pipeline) { // Consume as many pipeline items as possible. But yield the event loop // between successive tries. - switch (consume_result) { - case PipelineConsumeResult::MoreAvailable: { - task_runners_.GetRasterTaskRunner()->PostTask( - [weak_this = weak_factory_.GetTaskRunnerAffineWeakPtr(), pipeline]() { - if (weak_this) { - weak_this->Draw(pipeline); - } - }); - break; - } - default: - break; + if (!layer_tree_holder->IsEmpty()) { + task_runners_.GetRasterTaskRunner()->PostTask( + [weak_this = weak_factory_.GetWeakPtr(), layer_tree_holder]() { + if (weak_this) { + weak_this->Draw(layer_tree_holder); + } + }); } } diff --git a/shell/common/rasterizer.h b/shell/common/rasterizer.h index 6de1a843bdf5c..487e0ceb9acc5 100644 --- a/shell/common/rasterizer.h +++ b/shell/common/rasterizer.h @@ -19,7 +19,7 @@ #include "flutter/fml/time/time_delta.h" #include "flutter/fml/time/time_point.h" #include "flutter/lib/ui/snapshot_delegate.h" -#include "flutter/shell/common/pipeline.h" +#include "flutter/shell/common/layer_tree_holder.h" #include "flutter/shell/common/surface.h" namespace flutter { @@ -257,8 +257,8 @@ class Rasterizer final : public SnapshotDelegate { /// /// @param[in] pipeline The layer tree pipeline to take the next layer tree /// to render from. - /// - void Draw(fml::RefPtr> pipeline); + /// TODO(kaushikiska) fix docs whereever pipeline is mentioned. + void Draw(std::shared_ptr layer_tree_holder); //---------------------------------------------------------------------------- /// @brief The type of the screenshot to obtain of the previously diff --git a/shell/common/shell.cc b/shell/common/shell.cc index 8d278535c15e9..2c8a33aff046e 100644 --- a/shell/common/shell.cc +++ b/shell/common/shell.cc @@ -952,17 +952,16 @@ void Shell::OnAnimatorNotifyIdle(int64_t deadline) { } // |Animator::Delegate| -void Shell::OnAnimatorDraw(fml::RefPtr> pipeline) { +void Shell::OnAnimatorDraw(std::shared_ptr layer_tree_holder) { FML_DCHECK(is_setup_); task_runners_.GetRasterTaskRunner()->PostTask( [&waiting_for_first_frame = waiting_for_first_frame_, &waiting_for_first_frame_condition = waiting_for_first_frame_condition_, rasterizer = rasterizer_->GetWeakPtr(), - pipeline = std::move(pipeline)]() { + layer_tree_holder = std::move(layer_tree_holder)]() { if (rasterizer) { - rasterizer->Draw(pipeline); - + rasterizer->Draw(std::move(layer_tree_holder)); if (waiting_for_first_frame.load()) { waiting_for_first_frame.store(false); waiting_for_first_frame_condition.notify_all(); diff --git a/shell/common/shell.h b/shell/common/shell.h index 3c904d85c45e3..adaab8a77531b 100644 --- a/shell/common/shell.h +++ b/shell/common/shell.h @@ -30,6 +30,7 @@ #include "flutter/runtime/service_protocol.h" #include "flutter/shell/common/animator.h" #include "flutter/shell/common/engine.h" +#include "flutter/shell/common/layer_tree_holder.h" #include "flutter/shell/common/platform_view.h" #include "flutter/shell/common/rasterizer.h" #include "flutter/shell/common/shell_io_manager.h" @@ -490,7 +491,7 @@ class Shell final : public PlatformView::Delegate, // |Animator::Delegate| void OnAnimatorDraw( - fml::RefPtr> pipeline) override; + std::shared_ptr layer_tree_holder) override; // |Animator::Delegate| void OnAnimatorDrawLastLayerTree() override; From 68ad126971698f10c0a1da45f081abb594b47bd7 Mon Sep 17 00:00:00 2001 From: Kaushik Iska Date: Thu, 23 Apr 2020 11:44:47 -0700 Subject: [PATCH 2/4] revise docs and address CR comments --- ci/licenses_golden/licenses_flutter | 2 ++ flow/compositor_context.h | 4 --- shell/common/engine.h | 21 +++++-------- shell/common/layer_tree_holder.cc | 10 +++---- shell/common/layer_tree_holder.h | 2 +- shell/common/rasterizer.cc | 46 ++++++----------------------- shell/common/rasterizer.h | 39 ++++++++++-------------- shell/common/shell_test.cc | 2 +- 8 files changed, 41 insertions(+), 85 deletions(-) diff --git a/ci/licenses_golden/licenses_flutter b/ci/licenses_golden/licenses_flutter index 00ade5c06a1e7..2753414c4f0aa 100755 --- a/ci/licenses_golden/licenses_flutter +++ b/ci/licenses_golden/licenses_flutter @@ -576,6 +576,8 @@ FILE: ../../../flutter/shell/common/fixtures/shelltest_screenshot.png FILE: ../../../flutter/shell/common/input_events_unittests.cc FILE: ../../../flutter/shell/common/isolate_configuration.cc FILE: ../../../flutter/shell/common/isolate_configuration.h +FILE: ../../../flutter/shell/common/layer_tree_holder.cc +FILE: ../../../flutter/shell/common/layer_tree_holder.h FILE: ../../../flutter/shell/common/persistent_cache.cc FILE: ../../../flutter/shell/common/persistent_cache.h FILE: ../../../flutter/shell/common/persistent_cache_unittests.cc diff --git a/flow/compositor_context.h b/flow/compositor_context.h index b932594be5d71..23bf0be7a7dab 100644 --- a/flow/compositor_context.h +++ b/flow/compositor_context.h @@ -27,10 +27,6 @@ enum class RasterStatus { // Frame needs to be resubmitted for rasterization. This is // currently only called when thread configuration change occurs. kResubmit, - // Frame has been successfully rasterized, but "there are additional items in - // the pipeline waiting to be consumed. This is currently - // only called when thread configuration change occurs. - kEnqueuePipeline, // Failed to rasterize the frame. kFailed }; diff --git a/shell/common/engine.h b/shell/common/engine.h index 0f5f0717220b1..c7b2d24504888 100644 --- a/shell/common/engine.h +++ b/shell/common/engine.h @@ -406,25 +406,18 @@ class Engine final : public RuntimeDelegate, PointerDataDispatcher::Delegate { /// will cause the jank in the Flutter application: /// * The time taken by this method to create a layer-tree exceeds /// on frame interval (for example, 16.66 ms on a 60Hz display). - /// * The time take by this method to generate a new layer-tree - /// causes the current layer-tree pipeline depth to change. To - /// illustrate this point, note that maximum pipeline depth used - /// by layer tree in the engine is 2. If both the UI and GPU - /// task runner tasks finish within one frame interval, the - /// pipeline depth is one. If the UI thread happens to be - /// working on a frame when the raster thread is still not done - /// with the previous frame, the pipeline depth is 2. When the - /// pipeline depth changes from 1 to 2, animations and UI - /// interactions that cause the generation of the new layer tree - /// appropriate for (frame_time + one frame interval) will - /// actually end up at (frame_time + two frame intervals). This - /// is not what code running on the UI thread expected would - /// happen. This causes perceptible jank. + /// * A new layer-tree produced by this method replaces a stale + /// layer tree in `LayerTreeHolder`. See: + /// `LayerTreeHolder::ReplaceIfNewer`. This could happen if + /// rasterizer takes more than one frame interval to rasterize a + /// layer tree. This would cause some frames to be skipped and + /// could result in perceptible jank. /// /// @param[in] frame_time The point at which the current frame interval /// began. May be used by animation interpolators, /// physics simulations, etc.. /// + /// @see `LayerTreeHolder::ReplaceIfNewer` void BeginFrame(fml::TimePoint frame_time); //---------------------------------------------------------------------------- diff --git a/shell/common/layer_tree_holder.cc b/shell/common/layer_tree_holder.cc index 6d0e7d75106ab..5d7a386d9e623 100644 --- a/shell/common/layer_tree_holder.cc +++ b/shell/common/layer_tree_holder.cc @@ -11,12 +11,12 @@ std::unique_ptr LayerTreeHolder::Get() { return std::move(layer_tree_); } -void LayerTreeHolder::ReplaceIfNewer(std::unique_ptr layer_tree) { +void LayerTreeHolder::ReplaceIfNewer( + std::unique_ptr proposed_layer_tree) { std::scoped_lock lock(layer_tree_mutex); - if (IsEmpty()) { - layer_tree_ = std::move(layer_tree); - } else if (layer_tree_->target_time() < layer_tree->target_time()) { - layer_tree_ = std::move(layer_tree); + if (IsEmpty() || + layer_tree_->target_time() < proposed_layer_tree->target_time()) { + layer_tree_ = std::move(proposed_layer_tree); } } diff --git a/shell/common/layer_tree_holder.h b/shell/common/layer_tree_holder.h index 3666fd302502a..0617b3a04a1bd 100644 --- a/shell/common/layer_tree_holder.h +++ b/shell/common/layer_tree_holder.h @@ -21,7 +21,7 @@ class LayerTreeHolder { std::unique_ptr Get(); - void ReplaceIfNewer(std::unique_ptr layer_tree); + void ReplaceIfNewer(std::unique_ptr proposed_layer_tree); private: mutable std::mutex layer_tree_mutex; diff --git a/shell/common/rasterizer.cc b/shell/common/rasterizer.cc index 0181ec6e7b6d1..14652f125af09 100644 --- a/shell/common/rasterizer.cc +++ b/shell/common/rasterizer.cc @@ -121,29 +121,24 @@ void Rasterizer::Draw(std::shared_ptr layer_tree_holder) { FML_DCHECK(task_runners_.GetRasterTaskRunner()->RunsTasksOnCurrentThread()); std::unique_ptr layer_tree = layer_tree_holder->Get(); - RasterStatus raster_status = RasterStatus::kFailed; - if (layer_tree) { - raster_status = DoDraw(std::move(layer_tree)); - } - - // if the raster status is to resubmit the frame, we push the frame to the - // front of the queue and also change the consume status to more available. - if (raster_status == RasterStatus::kResubmit) { - layer_tree_holder->ReplaceIfNewer(std::move(resubmitted_layer_tree_)); - } + RasterStatus raster_status = + layer_tree ? DoDraw(std::move(layer_tree)) : RasterStatus::kFailed; // Merging the thread as we know the next `Draw` should be run on the platform // thread. if (raster_status == RasterStatus::kResubmit) { + layer_tree_holder->ReplaceIfNewer(std::move(resubmitted_layer_tree_)); auto* external_view_embedder = surface_->GetExternalViewEmbedder(); - // We know only the `external_view_embedder` can - // causes|RasterStatus::kResubmit|. Check to make sure. - FML_DCHECK(external_view_embedder != nullptr); + FML_DCHECK(external_view_embedder != nullptr) + << "kResubmit is an invalid raster status without external view " + "embedder."; external_view_embedder->EndFrame(raster_thread_merger_); } - // Consume as many pipeline items as possible. But yield the event loop + // Consume as many layer trees as possible. But yield the event loop // between successive tries. + // Note: This behaviour is left as-is to be inline with the pipeline + // semantics. TODO(kaushikiska): explore removing this block. if (!layer_tree_holder->IsEmpty()) { task_runners_.GetRasterTaskRunner()->PostTask( [weak_this = weak_factory_.GetWeakPtr(), layer_tree_holder]() { @@ -299,29 +294,6 @@ RasterStatus Rasterizer::DoDraw( ); } - // Pipeline pressure is applied from a couple of places: - // rasterizer: When there are more items as of the time of Consume. - // animator (via shell): Frame gets produces every vsync. - // Enqueing here is to account for the following scenario: - // T = 1 - // - one item (A) in the pipeline - // - rasterizer starts (and merges the threads) - // - pipeline consume result says no items to process - // T = 2 - // - animator produces (B) to the pipeline - // - applies pipeline pressure via platform thread. - // T = 3 - // - rasterizes finished (and un-merges the threads) - // - |Draw| for B yields as its on the wrong thread. - // This enqueue ensures that we attempt to consume from the right - // thread one more time after un-merge. - if (raster_thread_merger_) { - if (raster_thread_merger_->DecrementLease() == - fml::RasterThreadStatus::kUnmergedNow) { - return RasterStatus::kEnqueuePipeline; - } - } - return raster_status; } diff --git a/shell/common/rasterizer.h b/shell/common/rasterizer.h index 487e0ceb9acc5..32f16e5d9c2d8 100644 --- a/shell/common/rasterizer.h +++ b/shell/common/rasterizer.h @@ -230,34 +230,26 @@ class Rasterizer final : public SnapshotDelegate { flutter::TextureRegistry* GetTextureRegistry(); //---------------------------------------------------------------------------- - /// @brief Takes the next item from the layer tree pipeline and executes - /// the raster thread frame workload for that pipeline item to - /// render a frame on the on-screen surface. + /// @brief Takes the latest item from the layer tree holder and executes + /// the raster thread frame workload for that item to render a + /// frame on the on-screen surface. /// - /// Why does the draw call take a layer tree pipeline and not the + /// Why does the draw call take a layer tree holder and not the /// layer tree directly? /// - /// The pipeline is the way book-keeping of frame workloads - /// distributed across the multiple threads is managed. The - /// rasterizer deals with the pipelines directly (instead of layer - /// trees which is what it actually renders) because the pipeline - /// consumer's workload must be accounted for within the pipeline - /// itself. If the rasterizer took the layer tree directly, it - /// would have to be taken out of the pipeline. That would signal - /// the end of the frame workload and the pipeline would be ready - /// for new frames. But the last frame has not been rendered by - /// the frame yet! On the other hand, the pipeline must own the - /// layer tree it renders because it keeps a reference to the last - /// layer tree around till a new frame is rendered. So a simple - /// reference wont work either. The `Rasterizer::DoDraw` method - /// actually performs the GPU operations within the layer tree - /// pipeline. + /// The layer tree holder is a thread safe way to produce frame + /// workloads from the UI thread and raster them on the raster + /// thread. To account for scenarious where the UI thread + /// continues to produce the frames while a raster task is queued, + /// `Rasterizer::DoDraw` that gets executed on the raster thread, + /// must pick up the newest layer tree produced by the UI thread. + /// If we were to pass the layer tree as opposed to the holder, it + /// would result in stale frames being rendered. /// /// @see `Rasterizer::DoDraw` /// - /// @param[in] pipeline The layer tree pipeline to take the next layer tree - /// to render from. - /// TODO(kaushikiska) fix docs whereever pipeline is mentioned. + /// @param[in] layer_tree_holder The layer tree holder to take the latest + /// layer tree to render from. void Draw(std::shared_ptr layer_tree_holder); //---------------------------------------------------------------------------- @@ -425,7 +417,8 @@ class Rasterizer final : public SnapshotDelegate { std::unique_ptr last_layer_tree_; // Set when we need attempt to rasterize the layer tree again. This layer_tree // has not successfully rasterized. This can happen due to the change in the - // thread configuration. This will be inserted to the front of the pipeline. + // thread configuration. This layer tree could be rasterized again if there + // are no newer ones. std::unique_ptr resubmitted_layer_tree_; fml::closure next_frame_callback_; bool user_override_resource_cache_bytes_; diff --git a/shell/common/shell_test.cc b/shell/common/shell_test.cc index 8ef045cf0a045..a24c9eb85f5d2 100644 --- a/shell/common/shell_test.cc +++ b/shell/common/shell_test.cc @@ -136,7 +136,7 @@ void ShellTest::PumpOneFrame(Shell* shell, flutter::ViewportMetrics viewport_metrics, LayerTreeBuilder builder) { // Set viewport to nonempty, and call Animator::BeginFrame to make the layer - // tree pipeline nonempty. Without either of this, the layer tree below + // tree holder nonempty. Without either of this, the layer tree below // won't be rasterized. fml::AutoResetWaitableEvent latch; shell->GetTaskRunners().GetUITaskRunner()->PostTask( From 15c02ac9c56e5fbde20175bd62a952820665e1ab Mon Sep 17 00:00:00 2001 From: Kaushik Iska Date: Fri, 1 May 2020 12:40:00 -0700 Subject: [PATCH 3/4] fix deadlock --- shell/common/layer_tree_holder.cc | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/shell/common/layer_tree_holder.cc b/shell/common/layer_tree_holder.cc index 5d7a386d9e623..0cb39b720a075 100644 --- a/shell/common/layer_tree_holder.cc +++ b/shell/common/layer_tree_holder.cc @@ -14,7 +14,7 @@ std::unique_ptr LayerTreeHolder::Get() { void LayerTreeHolder::ReplaceIfNewer( std::unique_ptr proposed_layer_tree) { std::scoped_lock lock(layer_tree_mutex); - if (IsEmpty() || + if (!layer_tree_ || layer_tree_->target_time() < proposed_layer_tree->target_time()) { layer_tree_ = std::move(proposed_layer_tree); } From 55dacacccfc63edfbb34c64087b65173c2302272 Mon Sep 17 00:00:00 2001 From: Kaushik Iska Date: Tue, 5 May 2020 17:31:30 -0700 Subject: [PATCH 4/4] add tests --- ci/licenses_golden/licenses_flutter | 1 + flow/compositor_context.h | 4 ++ shell/common/BUILD.gn | 1 + shell/common/layer_tree_holder_unittests.cc | 76 +++++++++++++++++++++ shell/common/rasterizer.cc | 26 ++++++- shell/common/rasterizer.h | 4 +- 6 files changed, 108 insertions(+), 4 deletions(-) create mode 100644 shell/common/layer_tree_holder_unittests.cc diff --git a/ci/licenses_golden/licenses_flutter b/ci/licenses_golden/licenses_flutter index 2753414c4f0aa..6e29cdd120cae 100755 --- a/ci/licenses_golden/licenses_flutter +++ b/ci/licenses_golden/licenses_flutter @@ -578,6 +578,7 @@ FILE: ../../../flutter/shell/common/isolate_configuration.cc FILE: ../../../flutter/shell/common/isolate_configuration.h FILE: ../../../flutter/shell/common/layer_tree_holder.cc FILE: ../../../flutter/shell/common/layer_tree_holder.h +FILE: ../../../flutter/shell/common/layer_tree_holder_unittests.cc FILE: ../../../flutter/shell/common/persistent_cache.cc FILE: ../../../flutter/shell/common/persistent_cache.h FILE: ../../../flutter/shell/common/persistent_cache_unittests.cc diff --git a/flow/compositor_context.h b/flow/compositor_context.h index 23bf0be7a7dab..b932594be5d71 100644 --- a/flow/compositor_context.h +++ b/flow/compositor_context.h @@ -27,6 +27,10 @@ enum class RasterStatus { // Frame needs to be resubmitted for rasterization. This is // currently only called when thread configuration change occurs. kResubmit, + // Frame has been successfully rasterized, but "there are additional items in + // the pipeline waiting to be consumed. This is currently + // only called when thread configuration change occurs. + kEnqueuePipeline, // Failed to rasterize the frame. kFailed }; diff --git a/shell/common/BUILD.gn b/shell/common/BUILD.gn index a201ef74c41c1..9ada102333c5b 100644 --- a/shell/common/BUILD.gn +++ b/shell/common/BUILD.gn @@ -191,6 +191,7 @@ if (enable_unittests) { "animator_unittests.cc", "canvas_spy_unittests.cc", "input_events_unittests.cc", + "layer_tree_holder_unittests.cc", "persistent_cache_unittests.cc", "renderer_context_manager_unittests.cc", "renderer_context_test.cc", diff --git a/shell/common/layer_tree_holder_unittests.cc b/shell/common/layer_tree_holder_unittests.cc new file mode 100644 index 0000000000000..a82cefa1ba756 --- /dev/null +++ b/shell/common/layer_tree_holder_unittests.cc @@ -0,0 +1,76 @@ +// Copyright 2013 The Flutter Authors. All rights reserved. +// Use of this source code is governed by a BSD-style license that can be +// found in the LICENSE file. + +#define FML_USED_ON_EMBEDDER + +#include +#include +#include + +#include "flutter/shell/common/layer_tree_holder.h" +#include "gtest/gtest.h" + +namespace flutter { +namespace testing { + +TEST(LayerTreeHolder, EmptyOnInit) { + const LayerTreeHolder layer_tree_holder; + ASSERT_TRUE(layer_tree_holder.IsEmpty()); +} + +TEST(LayerTreeHolder, PutOneAndGet) { + LayerTreeHolder layer_tree_holder; + const auto frame_size = SkISize::Make(64, 64); + auto layer_tree = std::make_unique(frame_size, 100.0f, 1.0f); + layer_tree_holder.ReplaceIfNewer(std::move(layer_tree)); + ASSERT_FALSE(layer_tree_holder.IsEmpty()); + const auto stored = layer_tree_holder.Get(); + ASSERT_EQ(stored->frame_size(), frame_size); + ASSERT_TRUE(layer_tree_holder.IsEmpty()); +} + +TEST(LayerTreeHolder, PutMultiGetsLatest) { + const auto build_begin = fml::TimePoint::Now(); + const auto target_time_1 = build_begin + fml::TimeDelta::FromSeconds(2); + const auto target_time_2 = build_begin + fml::TimeDelta::FromSeconds(5); + + LayerTreeHolder layer_tree_holder; + const auto frame_size_1 = SkISize::Make(64, 64); + auto layer_tree_1 = std::make_unique(frame_size_1, 100.0f, 1.0f); + layer_tree_1->RecordBuildTime(build_begin, target_time_1); + layer_tree_holder.ReplaceIfNewer(std::move(layer_tree_1)); + + const auto frame_size_2 = SkISize::Make(128, 128); + auto layer_tree_2 = std::make_unique(frame_size_2, 100.0f, 1.0f); + layer_tree_2->RecordBuildTime(build_begin, target_time_2); + layer_tree_holder.ReplaceIfNewer(std::move(layer_tree_2)); + + const auto stored = layer_tree_holder.Get(); + ASSERT_EQ(stored->frame_size(), frame_size_2); + ASSERT_TRUE(layer_tree_holder.IsEmpty()); +} + +TEST(LayerTreeHolder, RetainsOlderIfNewerFrameHasEarlierTargetTime) { + const auto build_begin = fml::TimePoint::Now(); + const auto target_time_1 = build_begin + fml::TimeDelta::FromSeconds(5); + const auto target_time_2 = build_begin + fml::TimeDelta::FromSeconds(2); + + LayerTreeHolder layer_tree_holder; + const auto frame_size_1 = SkISize::Make(64, 64); + auto layer_tree_1 = std::make_unique(frame_size_1, 100.0f, 1.0f); + layer_tree_1->RecordBuildTime(build_begin, target_time_1); + layer_tree_holder.ReplaceIfNewer(std::move(layer_tree_1)); + + const auto frame_size_2 = SkISize::Make(128, 128); + auto layer_tree_2 = std::make_unique(frame_size_2, 100.0f, 1.0f); + layer_tree_2->RecordBuildTime(build_begin, target_time_2); + layer_tree_holder.ReplaceIfNewer(std::move(layer_tree_2)); + + const auto stored = layer_tree_holder.Get(); + ASSERT_EQ(stored->frame_size(), frame_size_1); + ASSERT_TRUE(layer_tree_holder.IsEmpty()); +} + +} // namespace testing +} // namespace flutter diff --git a/shell/common/rasterizer.cc b/shell/common/rasterizer.cc index 14652f125af09..b1354a0a53bdc 100644 --- a/shell/common/rasterizer.cc +++ b/shell/common/rasterizer.cc @@ -4,12 +4,11 @@ #include "flutter/shell/common/rasterizer.h" -#include "flutter/shell/common/persistent_cache.h" - #include #include "flutter/fml/time/time_delta.h" #include "flutter/fml/time/time_point.h" +#include "flutter/shell/common/persistent_cache.h" #include "third_party/skia/include/core/SkEncodedImageFormat.h" #include "third_party/skia/include/core/SkImageEncoder.h" #include "third_party/skia/include/core/SkPictureRecorder.h" @@ -294,6 +293,29 @@ RasterStatus Rasterizer::DoDraw( ); } + // Pipeline pressure is applied from a couple of places: + // rasterizer: When there are more items as of the time of Consume. + // animator (via shell): Frame gets produces every vsync. + // Enqueing here is to account for the following scenario: + // T = 1 + // - one item (A) in the pipeline + // - rasterizer starts (and merges the threads) + // - pipeline consume result says no items to process + // T = 2 + // - animator produces (B) to the pipeline + // - applies pipeline pressure via platform thread. + // T = 3 + // - rasterizes finished (and un-merges the threads) + // - |Draw| for B yields as its on the wrong thread. + // This enqueue ensures that we attempt to consume from the right + // thread one more time after un-merge. + if (raster_thread_merger_) { + if (raster_thread_merger_->DecrementLease() == + fml::RasterThreadStatus::kUnmergedNow) { + return RasterStatus::kEnqueuePipeline; + } + } + return raster_status; } diff --git a/shell/common/rasterizer.h b/shell/common/rasterizer.h index 32f16e5d9c2d8..1a4a3cf63e5b0 100644 --- a/shell/common/rasterizer.h +++ b/shell/common/rasterizer.h @@ -238,10 +238,10 @@ class Rasterizer final : public SnapshotDelegate { /// layer tree directly? /// /// The layer tree holder is a thread safe way to produce frame - /// workloads from the UI thread and raster them on the raster + /// workloads from the UI thread and rasterize them on the raster /// thread. To account for scenarious where the UI thread /// continues to produce the frames while a raster task is queued, - /// `Rasterizer::DoDraw` that gets executed on the raster thread, + /// `Rasterizer::DoDraw` that gets executed on the raster thread /// must pick up the newest layer tree produced by the UI thread. /// If we were to pass the layer tree as opposed to the holder, it /// would result in stale frames being rendered.