Merge "L2CAP Data pipeline rework"
diff --git a/gd/l2cap/Android.bp b/gd/l2cap/Android.bp
index fcd6604..649f618 100644
--- a/gd/l2cap/Android.bp
+++ b/gd/l2cap/Android.bp
@@ -17,9 +17,11 @@
         "classic/internal/link_manager.cc",
         "classic/internal/signalling_manager.cc",
         "classic/l2cap_classic_module.cc",
-        "internal/reassembler.cc",
+        "internal/basic_mode_channel_data_controller.cc",
+        "internal/enhanced_retransmission_mode_channel_data_controller.cc",
+        "internal/receiver.cc",
         "internal/scheduler_fifo.cc",
-        "internal/segmenter.cc",
+        "internal/sender.cc",
         "le/internal/fixed_channel_impl.cc",
         "le/internal/fixed_channel_service_manager_impl.cc",
         "le/internal/link_manager.cc",
@@ -41,9 +43,9 @@
         "classic/internal/link_manager_test.cc",
         "classic/internal/signalling_manager_test.cc",
         "internal/fixed_channel_allocator_test.cc",
-        "internal/reassembler_test.cc",
+        "internal/receiver_test.cc",
         "internal/scheduler_fifo_test.cc",
-        "internal/segmenter_test.cc",
+        "internal/sender_test.cc",
         "l2cap_packet_test.cc",
         "le/internal/fixed_channel_impl_test.cc",
         "le/internal/fixed_channel_service_manager_test.cc",
diff --git a/gd/l2cap/classic/internal/link.cc b/gd/l2cap/classic/internal/link.cc
index 286ab54..055e77c 100644
--- a/gd/l2cap/classic/internal/link.cc
+++ b/gd/l2cap/classic/internal/link.cc
@@ -35,8 +35,8 @@
            l2cap::internal::ParameterProvider* parameter_provider,
            DynamicChannelServiceManagerImpl* dynamic_service_manager,
            FixedChannelServiceManagerImpl* fixed_service_manager)
-    : l2cap_handler_(l2cap_handler), acl_connection_(std::move(acl_connection)),
-      reassembler_(acl_connection_->GetAclQueueEnd(), l2cap_handler_), scheduler_(std::move(scheduler)),
+    : l2cap_handler_(l2cap_handler), acl_connection_(std::move(acl_connection)), scheduler_(std::move(scheduler)),
+      receiver_(acl_connection_->GetAclQueueEnd(), l2cap_handler_, scheduler_.get()),
       parameter_provider_(parameter_provider), dynamic_service_manager_(dynamic_service_manager),
       fixed_service_manager_(fixed_service_manager),
       signalling_manager_(l2cap_handler_, this, dynamic_service_manager_, &dynamic_channel_allocator_,
@@ -61,7 +61,6 @@
 std::shared_ptr<FixedChannelImpl> Link::AllocateFixedChannel(Cid cid, SecurityPolicy security_policy) {
   auto channel = fixed_channel_allocator_.AllocateChannel(cid, security_policy);
   scheduler_->AttachChannel(cid, channel);
-  reassembler_.AttachChannel(cid, channel);
   return channel;
 }
 
@@ -96,7 +95,6 @@
   auto channel = dynamic_channel_allocator_.AllocateChannel(psm, remote_cid, security_policy);
   if (channel != nullptr) {
     scheduler_->AttachChannel(channel->GetCid(), channel);
-    reassembler_.AttachChannel(channel->GetCid(), channel);
   }
   channel->local_initiated_ = false;
   return channel;
@@ -107,12 +105,19 @@
   auto channel = dynamic_channel_allocator_.AllocateReservedChannel(reserved_cid, psm, remote_cid, security_policy);
   if (channel != nullptr) {
     scheduler_->AttachChannel(channel->GetCid(), channel);
-    reassembler_.AttachChannel(channel->GetCid(), channel);
   }
   channel->local_initiated_ = true;
   return channel;
 }
 
+void Link::SetChannelRetransmissionFlowControlMode(Cid cid, RetransmissionAndFlowControlModeOption mode) {
+  if (dynamic_channel_allocator_.FindChannelByCid(cid) == nullptr) {
+    LOG_ERROR("Channel doesn't exist: %d", cid);
+    return;
+  }
+  scheduler_->SetChannelRetransmissionFlowControlMode(cid, mode);
+}
+
 void Link::FreeDynamicChannel(Cid cid) {
   if (dynamic_channel_allocator_.FindChannelByCid(cid) == nullptr) {
     return;
diff --git a/gd/l2cap/classic/internal/link.h b/gd/l2cap/classic/internal/link.h
index ea33b15..f192e3e 100644
--- a/gd/l2cap/classic/internal/link.h
+++ b/gd/l2cap/classic/internal/link.h
@@ -27,7 +27,7 @@
 #include "l2cap/classic/internal/fixed_channel_service_manager_impl.h"
 #include "l2cap/internal/fixed_channel_allocator.h"
 #include "l2cap/internal/parameter_provider.h"
-#include "l2cap/internal/reassembler.h"
+#include "l2cap/internal/receiver.h"
 #include "l2cap/internal/scheduler.h"
 #include "os/alarm.h"
 #include "os/handler.h"
@@ -87,6 +87,8 @@
   virtual std::shared_ptr<DynamicChannelImpl> AllocateReservedDynamicChannel(Cid reserved_cid, Psm psm, Cid remote_cid,
                                                                              SecurityPolicy security_policy);
 
+  virtual void SetChannelRetransmissionFlowControlMode(Cid cid, RetransmissionAndFlowControlModeOption mode);
+
   virtual void FreeDynamicChannel(Cid cid);
 
   // Check how many channels are acquired or in use, if zero, start tear down timer, if non-zero, cancel tear down timer
@@ -100,8 +102,8 @@
   l2cap::internal::FixedChannelAllocator<FixedChannelImpl, Link> fixed_channel_allocator_{this, l2cap_handler_};
   DynamicChannelAllocator dynamic_channel_allocator_{this, l2cap_handler_};
   std::unique_ptr<hci::AclConnection> acl_connection_;
-  l2cap::internal::Reassembler reassembler_;
   std::unique_ptr<l2cap::internal::Scheduler> scheduler_;
+  l2cap::internal::Receiver receiver_;
   l2cap::internal::ParameterProvider* parameter_provider_;
   DynamicChannelServiceManagerImpl* dynamic_service_manager_;
   FixedChannelServiceManagerImpl* fixed_service_manager_;
diff --git a/gd/l2cap/internal/basic_mode_channel_data_controller.cc b/gd/l2cap/internal/basic_mode_channel_data_controller.cc
new file mode 100644
index 0000000..1a049aa
--- /dev/null
+++ b/gd/l2cap/internal/basic_mode_channel_data_controller.cc
@@ -0,0 +1,46 @@
+/*
+ * Copyright 2019 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 "l2cap/internal/basic_mode_channel_data_controller.h"
+
+namespace bluetooth {
+namespace l2cap {
+namespace internal {
+
+BasicModeDataController::BasicModeDataController(Cid cid, Cid remote_cid, UpperQueueDownEnd* channel_queue_end,
+                                                 os::Handler* handler, Scheduler* scheduler)
+    : cid_(cid), remote_cid_(remote_cid), enqueue_buffer_(channel_queue_end), handler_(handler), scheduler_(scheduler) {
+}
+
+void BasicModeDataController::OnSdu(std::unique_ptr<packet::BasePacketBuilder> sdu) {
+  auto l2cap_information = BasicFrameBuilder::Create(remote_cid_, std::move(sdu));
+  pdu_queue_.emplace(std::move(l2cap_information));
+  scheduler_->OnPacketsReady(cid_, 1);
+}
+
+void BasicModeDataController::OnPdu(BasicFrameView pdu) {
+  enqueue_buffer_.Enqueue(std::make_unique<PacketView<kLittleEndian>>(pdu.GetPayload()), handler_);
+}
+
+std::unique_ptr<BasicFrameBuilder> BasicModeDataController::GetNextPacket() {
+  auto next = std::move(pdu_queue_.front());
+  pdu_queue_.pop();
+  return next;
+}
+
+}  // namespace internal
+}  // namespace l2cap
+}  // namespace bluetooth
diff --git a/gd/l2cap/internal/basic_mode_channel_data_controller.h b/gd/l2cap/internal/basic_mode_channel_data_controller.h
new file mode 100644
index 0000000..c19fb93
--- /dev/null
+++ b/gd/l2cap/internal/basic_mode_channel_data_controller.h
@@ -0,0 +1,64 @@
+/*
+ * Copyright 2019 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 <memory>
+#include <unordered_map>
+#include <utility>
+
+#include "common/bidi_queue.h"
+#include "l2cap/cid.h"
+#include "l2cap/internal/channel_impl.h"
+#include "l2cap/internal/data_controller.h"
+#include "l2cap/internal/scheduler.h"
+#include "l2cap/l2cap_packets.h"
+#include "l2cap/mtu.h"
+#include "os/handler.h"
+#include "os/queue.h"
+#include "packet/base_packet_builder.h"
+#include "packet/packet_view.h"
+
+namespace bluetooth {
+namespace l2cap {
+namespace internal {
+
+class BasicModeDataController : public DataController {
+ public:
+  using UpperEnqueue = packet::PacketView<packet::kLittleEndian>;
+  using UpperDequeue = packet::BasePacketBuilder;
+  using UpperQueueDownEnd = common::BidiQueueEnd<UpperEnqueue, UpperDequeue>;
+  BasicModeDataController(Cid cid, Cid remote_cid, UpperQueueDownEnd* channel_queue_end, os::Handler* handler,
+                          Scheduler* scheduler);
+
+  void OnSdu(std::unique_ptr<packet::BasePacketBuilder> sdu) override;
+
+  void OnPdu(BasicFrameView pdu) override;
+
+  std::unique_ptr<BasicFrameBuilder> GetNextPacket() override;
+
+ private:
+  Cid cid_;
+  Cid remote_cid_;
+  os::EnqueueBuffer<UpperEnqueue> enqueue_buffer_;
+  os::Handler* handler_;
+  std::queue<std::unique_ptr<BasicFrameBuilder>> pdu_queue_;
+  Scheduler* scheduler_;
+};
+
+}  // namespace internal
+}  // namespace l2cap
+}  // namespace bluetooth
diff --git a/gd/l2cap/internal/channel_impl.h b/gd/l2cap/internal/channel_impl.h
index c375b5f..92f2bbb 100644
--- a/gd/l2cap/internal/channel_impl.h
+++ b/gd/l2cap/internal/channel_impl.h
@@ -38,7 +38,7 @@
   GetQueueUpEnd() = 0;
 
   /**
-   * Return the queue end for lower layer (segmenter and reassembler)
+   * Return the queue end for lower layer (sender and receiver)
    */
   virtual common::BidiQueueEnd<packet::PacketView<packet::kLittleEndian>, packet::BasePacketBuilder>*
   GetQueueDownEnd() = 0;
diff --git a/gd/l2cap/internal/data_controller.h b/gd/l2cap/internal/data_controller.h
new file mode 100644
index 0000000..18cff2a
--- /dev/null
+++ b/gd/l2cap/internal/data_controller.h
@@ -0,0 +1,45 @@
+/*
+ * Copyright 2019 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 <memory>
+
+#include "l2cap/l2cap_packets.h"
+#include "packet/base_packet_builder.h"
+#include "packet/packet_view.h"
+
+namespace bluetooth {
+namespace l2cap {
+namespace internal {
+
+class DataController {
+ public:
+  virtual ~DataController() = default;
+
+  // SDU -> PDUs and notify Scheduler
+  virtual void OnSdu(std::unique_ptr<packet::BasePacketBuilder> sdu) = 0;
+
+  // PDUs -> SDU and enqueue to channel queue end
+  virtual void OnPdu(BasicFrameView pdu) = 0;
+
+  // Used by Scheduler to get next PDU
+  virtual std::unique_ptr<BasicFrameBuilder> GetNextPacket() = 0;
+};
+
+}  // namespace internal
+}  // namespace l2cap
+}  // namespace bluetooth
diff --git a/gd/l2cap/internal/enhanced_retransmission_mode_channel_data_controller.cc b/gd/l2cap/internal/enhanced_retransmission_mode_channel_data_controller.cc
new file mode 100644
index 0000000..55f98e2
--- /dev/null
+++ b/gd/l2cap/internal/enhanced_retransmission_mode_channel_data_controller.cc
@@ -0,0 +1,42 @@
+/*
+ * Copyright 2019 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 "l2cap/internal/enhanced_retransmission_mode_channel_data_controller.h"
+
+namespace bluetooth {
+namespace l2cap {
+namespace internal {
+ErtmController::ErtmController(Cid cid, Cid remote_cid, UpperQueueDownEnd* channel_queue_end, os::Handler* handler,
+                               Scheduler* scheduler)
+    : cid_(cid), enqueue_buffer_(channel_queue_end), handler_(handler), scheduler_(scheduler) {}
+
+void ErtmController::OnSdu(std::unique_ptr<packet::BasePacketBuilder> sdu) {
+  LOG_ERROR("Not implemented");
+}
+
+void ErtmController::OnPdu(BasicFrameView pdu) {
+  LOG_ERROR("Not implemented");
+}
+
+std::unique_ptr<BasicFrameBuilder> ErtmController::GetNextPacket() {
+  auto next = std::move(pdu_queue_.front());
+  pdu_queue_.pop();
+  return next;
+}
+
+}  // namespace internal
+}  // namespace l2cap
+}  // namespace bluetooth
diff --git a/gd/l2cap/internal/enhanced_retransmission_mode_channel_data_controller.h b/gd/l2cap/internal/enhanced_retransmission_mode_channel_data_controller.h
new file mode 100644
index 0000000..7db179c
--- /dev/null
+++ b/gd/l2cap/internal/enhanced_retransmission_mode_channel_data_controller.h
@@ -0,0 +1,60 @@
+/*
+ * Copyright 2019 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 <memory>
+#include <unordered_map>
+#include <utility>
+
+#include "common/bidi_queue.h"
+#include "l2cap/cid.h"
+#include "l2cap/internal/channel_impl.h"
+#include "l2cap/internal/data_controller.h"
+#include "l2cap/internal/scheduler.h"
+#include "l2cap/l2cap_packets.h"
+#include "l2cap/mtu.h"
+#include "os/handler.h"
+#include "os/queue.h"
+#include "packet/base_packet_builder.h"
+#include "packet/packet_view.h"
+
+namespace bluetooth {
+namespace l2cap {
+namespace internal {
+
+class ErtmController : public DataController {
+ public:
+  using UpperEnqueue = packet::PacketView<packet::kLittleEndian>;
+  using UpperDequeue = packet::BasePacketBuilder;
+  using UpperQueueDownEnd = common::BidiQueueEnd<UpperEnqueue, UpperDequeue>;
+  ErtmController(Cid cid, Cid remote_cid, UpperQueueDownEnd* channel_queue_end, os::Handler* handler,
+                 Scheduler* scheduler);
+  void OnSdu(std::unique_ptr<packet::BasePacketBuilder> sdu) override;
+  void OnPdu(BasicFrameView pdu) override;
+  std::unique_ptr<BasicFrameBuilder> GetNextPacket() override;
+
+ private:
+  [[maybe_unused]] Cid cid_;
+  [[maybe_unused]] os::EnqueueBuffer<UpperEnqueue> enqueue_buffer_;
+  [[maybe_unused]] os::Handler* handler_;
+  std::queue<std::unique_ptr<BasicFrameBuilder>> pdu_queue_;
+  [[maybe_unused]] Scheduler* scheduler_;
+};
+
+}  // namespace internal
+}  // namespace l2cap
+}  // namespace bluetooth
diff --git a/gd/l2cap/internal/reassembler.cc b/gd/l2cap/internal/reassembler.cc
deleted file mode 100644
index ede977c..0000000
--- a/gd/l2cap/internal/reassembler.cc
+++ /dev/null
@@ -1,90 +0,0 @@
-/*
- * Copyright 2019 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 "l2cap/internal/reassembler.h"
-
-#include "common/bidi_queue.h"
-#include "l2cap/cid.h"
-#include "l2cap/l2cap_packets.h"
-#include "packet/packet_view.h"
-
-namespace bluetooth {
-namespace l2cap {
-namespace internal {
-Reassembler::Reassembler(LowerQueueUpEnd* link_queue_up_end, os::Handler* handler)
-    : link_queue_up_end_(link_queue_up_end), handler_(handler) {
-  ASSERT(link_queue_up_end_ != nullptr && handler_ != nullptr);
-  link_queue_up_end_->RegisterDequeue(
-      handler_, common::Bind(&Reassembler::link_queue_dequeue_callback, common::Unretained(this)));
-}
-
-Reassembler::~Reassembler() {
-  link_queue_up_end_->UnregisterDequeue();
-}
-
-void Reassembler::AttachChannel(Cid cid, std::shared_ptr<ChannelImpl> channel) {
-  ASSERT_LOG(channel_map_.find(cid) == channel_map_.end(), "Channel is already attached");
-  channel_map_.emplace(std::piecewise_construct, std::forward_as_tuple(cid),
-                       std::forward_as_tuple(channel->GetQueueDownEnd(), channel));
-}
-
-void Reassembler::DetachChannel(Cid cid) {
-  ASSERT_LOG(channel_map_.find(cid) != channel_map_.end(), "Channel is not attached");
-  channel_map_.erase(cid);
-}
-
-void Reassembler::link_queue_dequeue_callback() {
-  auto packet = link_queue_up_end_->TryDequeue();
-  auto basic_frame_view = BasicFrameView::Create(*packet);
-  if (!basic_frame_view.IsValid()) {
-    LOG_WARN("Received an invalid basic frame");
-    return;
-  }
-  Cid cid = static_cast<Cid>(basic_frame_view.GetChannelId());
-  auto channel = channel_map_.find(cid);
-  if (channel == channel_map_.end() || (cid >= kFirstDynamicChannel && channel->second.channel_ == nullptr)) {
-    LOG_WARN("Received a packet with invalid cid: %d", cid);
-    return;  // Channel is not attached to scheduler
-  }
-
-  auto channel_mode = cid < kFirstDynamicChannel ? RetransmissionAndFlowControlModeOption::L2CAP_BASIC
-                                                 : channel->second.channel_->GetChannelMode();
-  switch (channel_mode) {
-    case RetransmissionAndFlowControlModeOption::L2CAP_BASIC:
-      handle_basic_mode_packet(cid, basic_frame_view);
-      break;
-    case RetransmissionAndFlowControlModeOption::ENHANCED_RETRANSMISSION:
-      handle_enhanced_retransmission_mode_packet(cid, std::move(basic_frame_view));
-      break;
-    default:
-      LOG_WARN("channel mode is not supported: %d", static_cast<int>(channel_mode));
-  }
-}
-
-void Reassembler::handle_basic_mode_packet(Cid cid, const BasicFrameView& view) {
-  auto channel = channel_map_.find(cid);
-  auto& enqueue_buffer = channel->second.enqueue_buffer_;
-
-  enqueue_buffer.Enqueue(std::make_unique<PacketView<kLittleEndian>>(view.GetPayload()), handler_);
-}
-
-void Reassembler::handle_enhanced_retransmission_mode_packet(Cid cid, BasicFrameView view) {
-  LOG_ERROR("Enhanced retransmission mode is not implemented");
-}
-
-}  // namespace internal
-}  // namespace l2cap
-}  // namespace bluetooth
diff --git a/gd/l2cap/internal/reassembler_test.cc b/gd/l2cap/internal/reassembler_test.cc
deleted file mode 100644
index 5db68e6..0000000
--- a/gd/l2cap/internal/reassembler_test.cc
+++ /dev/null
@@ -1,117 +0,0 @@
-/*
- * Copyright 2019 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 "l2cap/internal/reassembler.h"
-
-#include <gmock/gmock.h>
-#include <gtest/gtest.h>
-#include <future>
-
-#include "l2cap/internal/channel_impl_mock.h"
-#include "l2cap/l2cap_packets.h"
-#include "os/handler.h"
-#include "os/queue.h"
-#include "os/thread.h"
-#include "packet/raw_builder.h"
-
-namespace bluetooth {
-namespace l2cap {
-namespace internal {
-namespace {
-
-using ::testing::Return;
-
-std::unique_ptr<BasicFrameBuilder> CreateSampleL2capPacket(Cid cid, std::vector<uint8_t> payload) {
-  auto raw_builder = std::make_unique<packet::RawBuilder>();
-  raw_builder->AddOctets(payload);
-  return BasicFrameBuilder::Create(cid, std::move(raw_builder));
-}
-
-PacketView<kLittleEndian> GetPacketView(std::unique_ptr<packet::BasePacketBuilder> packet) {
-  auto bytes = std::make_shared<std::vector<uint8_t>>();
-  BitInserter i(*bytes);
-  bytes->reserve(packet->size());
-  packet->Serialize(i);
-  return packet::PacketView<packet::kLittleEndian>(bytes);
-}
-
-void sync_handler(os::Handler* handler) {
-  std::promise<void> promise;
-  auto future = promise.get_future();
-  handler->Post(common::BindOnce(&std::promise<void>::set_value, common::Unretained(&promise)));
-  auto status = future.wait_for(std::chrono::milliseconds(3));
-  EXPECT_EQ(status, std::future_status::ready);
-}
-
-class L2capClassicReassemblerTest : public ::testing::Test {
- protected:
-  void SetUp() override {
-    thread_ = new os::Thread("test_thread", os::Thread::Priority::NORMAL);
-    user_handler_ = new os::Handler(thread_);
-    queue_handler_ = new os::Handler(thread_);
-    reassembler_ = new Reassembler(link_queue_.GetUpEnd(), queue_handler_);
-  }
-
-  void TearDown() override {
-    delete reassembler_;
-    queue_handler_->Clear();
-    user_handler_->Clear();
-    delete queue_handler_;
-    delete user_handler_;
-    delete thread_;
-  }
-
-  os::Thread* thread_ = nullptr;
-  os::Handler* user_handler_ = nullptr;
-  os::Handler* queue_handler_ = nullptr;
-  common::BidiQueue<Reassembler::LowerDequeue, Reassembler::LowerEnqueue> link_queue_{10};
-  Reassembler* reassembler_ = nullptr;
-};
-
-TEST_F(L2capClassicReassemblerTest, receive_basic_mode_packet_for_fixed_channel) {
-  common::BidiQueue<Reassembler::UpperEnqueue, Reassembler::UpperDequeue> channel_one_queue_{10};
-  common::BidiQueue<Reassembler::UpperEnqueue, Reassembler::UpperDequeue> channel_two_queue_{10};
-
-  auto mock_channel_1 = std::make_shared<testing::MockChannelImpl>();
-  ON_CALL(*mock_channel_1, GetQueueDownEnd()).WillByDefault(Return(channel_one_queue_.GetDownEnd()));
-  auto mock_channel_2 = std::make_shared<testing::MockChannelImpl>();
-  ON_CALL(*mock_channel_2, GetQueueDownEnd()).WillByDefault(Return(channel_two_queue_.GetDownEnd()));
-  reassembler_->AttachChannel(1, mock_channel_1);
-  reassembler_->AttachChannel(2, mock_channel_2);
-  os::EnqueueBuffer<Reassembler::UpperEnqueue> link_queue_enqueue_buffer{link_queue_.GetDownEnd()};
-  auto packet_one = CreateSampleL2capPacket(1, {1, 2, 3});
-  auto packet_two = CreateSampleL2capPacket(2, {4, 5, 6, 7});
-  auto packet_one_view = GetPacketView(std::move(packet_one));
-  auto packet_two_view = GetPacketView(std::move(packet_two));
-  link_queue_enqueue_buffer.Enqueue(std::make_unique<Reassembler::UpperEnqueue>(packet_one_view), queue_handler_);
-  link_queue_enqueue_buffer.Enqueue(std::make_unique<Reassembler::UpperEnqueue>(packet_two_view), queue_handler_);
-  sync_handler(queue_handler_);
-  sync_handler(user_handler_);
-  sync_handler(queue_handler_);
-  auto packet = channel_one_queue_.GetUpEnd()->TryDequeue();
-  EXPECT_NE(packet, nullptr);
-  EXPECT_EQ(packet->size(), 3);
-  packet = channel_two_queue_.GetUpEnd()->TryDequeue();
-  EXPECT_NE(packet, nullptr);
-  EXPECT_EQ(packet->size(), 4);
-  reassembler_->DetachChannel(1);
-  reassembler_->DetachChannel(2);
-}
-
-}  // namespace
-}  // namespace internal
-}  // namespace l2cap
-}  // namespace bluetooth
diff --git a/gd/l2cap/internal/receiver.cc b/gd/l2cap/internal/receiver.cc
new file mode 100644
index 0000000..437064e
--- /dev/null
+++ b/gd/l2cap/internal/receiver.cc
@@ -0,0 +1,56 @@
+/*
+ * Copyright 2019 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 "l2cap/internal/receiver.h"
+
+#include "common/bidi_queue.h"
+#include "l2cap/cid.h"
+#include "l2cap/l2cap_packets.h"
+#include "packet/packet_view.h"
+
+namespace bluetooth {
+namespace l2cap {
+namespace internal {
+Receiver::Receiver(LowerQueueUpEnd* link_queue_up_end, os::Handler* handler, Scheduler* scheduler)
+    : link_queue_up_end_(link_queue_up_end), handler_(handler), scheduler_(scheduler) {
+  ASSERT(link_queue_up_end_ != nullptr && handler_ != nullptr);
+  link_queue_up_end_->RegisterDequeue(handler_,
+                                      common::Bind(&Receiver::link_queue_dequeue_callback, common::Unretained(this)));
+}
+
+Receiver::~Receiver() {
+  link_queue_up_end_->UnregisterDequeue();
+}
+
+void Receiver::link_queue_dequeue_callback() {
+  auto packet = link_queue_up_end_->TryDequeue();
+  auto basic_frame_view = BasicFrameView::Create(*packet);
+  if (!basic_frame_view.IsValid()) {
+    LOG_WARN("Received an invalid basic frame");
+    return;
+  }
+  Cid cid = static_cast<Cid>(basic_frame_view.GetChannelId());
+  auto* data_controller = scheduler_->GetDataController(cid);
+  if (data_controller == nullptr) {
+    LOG_WARN("Received a packet with invalid cid: %d", cid);
+    return;
+  }
+  data_controller->OnPdu(basic_frame_view);
+}
+
+}  // namespace internal
+}  // namespace l2cap
+}  // namespace bluetooth
diff --git a/gd/l2cap/internal/reassembler.h b/gd/l2cap/internal/receiver.h
similarity index 62%
rename from gd/l2cap/internal/reassembler.h
rename to gd/l2cap/internal/receiver.h
index 30ae555..e3c035e 100644
--- a/gd/l2cap/internal/reassembler.h
+++ b/gd/l2cap/internal/receiver.h
@@ -23,6 +23,7 @@
 #include "common/bidi_queue.h"
 #include "l2cap/cid.h"
 #include "l2cap/internal/channel_impl.h"
+#include "l2cap/internal/scheduler.h"
 #include "l2cap/l2cap_packets.h"
 #include "l2cap/mtu.h"
 #include "os/queue.h"
@@ -31,17 +32,17 @@
 
 namespace bluetooth {
 namespace l2cap {
-
 namespace internal {
 
 /**
- * Handle the reassembly of L2CAP SDU from PDU.
+ * Handle receiving L2CAP PDUs from link queue and distribute them into into channel data controllers.
  * Dequeue incoming packets from LinkQueueUpEnd, and enqueue it to ChannelQueueDownEnd. Note: If a channel
  * cannot dequeue from ChannelQueueDownEnd so that the buffer for incoming packet is full, further incoming packets will
  * be dropped.
  * The Reassembler keeps the reference to ChannelImpl objects, because it needs to check channel mode and parameters.
+ * The Reassembler also keeps the reference to Scheduler, to get Segmenter and send signals (Tx, Rx seq) to it.
  */
-class Reassembler {
+class Receiver {
  public:
   using UpperEnqueue = packet::PacketView<packet::kLittleEndian>;
   using UpperDequeue = packet::BasePacketBuilder;
@@ -50,37 +51,15 @@
   using LowerDequeue = UpperEnqueue;
   using LowerQueueUpEnd = common::BidiQueueEnd<LowerEnqueue, LowerDequeue>;
 
-  Reassembler(LowerQueueUpEnd* link_queue_up_end, os::Handler* handler);
-  ~Reassembler();
-
-  /**
-   * Attach a channel for packet reassembly.
-   * If the channel is a dynamic channel, a shared_ptr reference to DynamicChannelImpl is needed to read necessary
-   * config. If the channel is a fixed channel, use nullptr.
-   * TODO (b/144503952): Rethink about channel abstraction
-   */
-  void AttachChannel(Cid cid, std::shared_ptr<ChannelImpl> channel);
-
-  /**
-   * Detach a channel for packet reassembly. Incoming packets won't be delivered to the specified cid.
-   */
-  void DetachChannel(Cid cid);
+  Receiver(LowerQueueUpEnd* link_queue_up_end, os::Handler* handler, Scheduler* scheduler);
+  ~Receiver();
 
  private:
-  struct ChannelBuffer {
-    ChannelBuffer(UpperQueueDownEnd* queue_end, std::shared_ptr<ChannelImpl> channel)
-        : enqueue_buffer_(queue_end), channel_(std::move(channel)) {}
-    os::EnqueueBuffer<UpperEnqueue> enqueue_buffer_;
-    std::shared_ptr<ChannelImpl> channel_;
-  };
-
   LowerQueueUpEnd* link_queue_up_end_;
   os::Handler* handler_;
-  std::unordered_map<Cid, ChannelBuffer> channel_map_;
+  Scheduler* scheduler_;
 
   void link_queue_dequeue_callback();
-  void handle_basic_mode_packet(Cid cid, const BasicFrameView& view);
-  void handle_enhanced_retransmission_mode_packet(Cid cid, BasicFrameView view);
 };
 
 }  // namespace internal
diff --git a/gd/l2cap/internal/receiver_test.cc b/gd/l2cap/internal/receiver_test.cc
new file mode 100644
index 0000000..a401d22
--- /dev/null
+++ b/gd/l2cap/internal/receiver_test.cc
@@ -0,0 +1,36 @@
+/*
+ * Copyright 2019 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 "l2cap/internal/receiver.h"
+
+#include <gmock/gmock.h>
+#include <gtest/gtest.h>
+#include <future>
+
+#include "l2cap/internal/channel_impl_mock.h"
+#include "l2cap/l2cap_packets.h"
+#include "os/handler.h"
+#include "os/queue.h"
+#include "os/thread.h"
+#include "packet/raw_builder.h"
+
+namespace bluetooth {
+namespace l2cap {
+namespace internal {
+namespace {}  // namespace
+}  // namespace internal
+}  // namespace l2cap
+}  // namespace bluetooth
diff --git a/gd/l2cap/internal/scheduler.h b/gd/l2cap/internal/scheduler.h
index 65094c7..d400aca 100644
--- a/gd/l2cap/internal/scheduler.h
+++ b/gd/l2cap/internal/scheduler.h
@@ -16,9 +16,13 @@
 
 #pragma once
 
+#include <cstdint>
+
 #include "common/bidi_queue.h"
 #include "l2cap/cid.h"
 #include "l2cap/internal/channel_impl.h"
+#include "l2cap/internal/data_controller.h"
+#include "l2cap/internal/sender.h"
 #include "l2cap/l2cap_packets.h"
 #include "packet/base_packet_builder.h"
 #include "packet/packet_view.h"
@@ -63,7 +67,19 @@
   /**
    * Callback from the segmenter to indicate that the scheduler could dequeue number_packets from it
    */
-  virtual void NotifyPacketsReady(Cid cid, int number_packets) {}
+  virtual void OnPacketsReady(Cid cid, int number_packets) {}
+
+  /**
+   * Set the channel mode for a cid
+   */
+  virtual void SetChannelRetransmissionFlowControlMode(Cid cid, RetransmissionAndFlowControlModeOption mode) {}
+
+  /**
+   * Get the data controller for Reassembler
+   */
+  virtual DataController* GetDataController(Cid cid) {
+    return nullptr;
+  }
 
   virtual ~Scheduler() = default;
 };
diff --git a/gd/l2cap/internal/scheduler_fifo.cc b/gd/l2cap/internal/scheduler_fifo.cc
index c77e937..f60f38f 100644
--- a/gd/l2cap/internal/scheduler_fifo.cc
+++ b/gd/l2cap/internal/scheduler_fifo.cc
@@ -47,7 +47,7 @@
   segmenter_map_.erase(cid);
 }
 
-void Fifo::NotifyPacketsReady(Cid cid, int number_packets) {
+void Fifo::OnPacketsReady(Cid cid, int number_packets) {
   next_to_dequeue_and_num_packets.push(std::make_pair(cid, number_packets));
   try_register_link_queue_enqueue();
 }
@@ -62,7 +62,7 @@
   }
   auto packet = segmenter_map_.find(channel_id)->second.GetNextPacket();
 
-  segmenter_map_.find(channel_id)->second.NotifyPacketSent();
+  segmenter_map_.find(channel_id)->second.OnPacketSent();
   if (next_to_dequeue_and_num_packets.empty()) {
     link_queue_up_end_->UnregisterEnqueue();
     link_queue_enqueue_registered_ = false;
@@ -79,6 +79,18 @@
   link_queue_enqueue_registered_ = true;
 }
 
+void Fifo::SetChannelRetransmissionFlowControlMode(Cid cid, RetransmissionAndFlowControlModeOption mode) {
+  ASSERT(segmenter_map_.find(cid) != segmenter_map_.end());
+  segmenter_map_.find(cid)->second.SetChannelRetransmissionFlowControlMode(mode);
+}
+
+DataController* Fifo::GetDataController(Cid cid) {
+  if (segmenter_map_.find(cid) == segmenter_map_.end()) {
+    return nullptr;
+  }
+  return segmenter_map_.find(cid)->second.GetDataController();
+}
+
 }  // namespace internal
 }  // namespace l2cap
 }  // namespace bluetooth
diff --git a/gd/l2cap/internal/scheduler_fifo.h b/gd/l2cap/internal/scheduler_fifo.h
index a2921fa..6be3a09 100644
--- a/gd/l2cap/internal/scheduler_fifo.h
+++ b/gd/l2cap/internal/scheduler_fifo.h
@@ -24,13 +24,12 @@
 #include "l2cap/cid.h"
 #include "l2cap/internal/channel_impl.h"
 #include "l2cap/internal/scheduler.h"
-#include "l2cap/internal/segmenter.h"
+#include "l2cap/internal/sender.h"
 #include "os/handler.h"
 #include "os/queue.h"
 
 namespace bluetooth {
 namespace l2cap {
-
 namespace internal {
 
 class Fifo : public Scheduler {
@@ -39,12 +38,14 @@
   ~Fifo() override;
   void AttachChannel(Cid cid, std::shared_ptr<ChannelImpl> channel) override;
   void DetachChannel(Cid cid) override;
-  void NotifyPacketsReady(Cid cid, int number_packets) override;
+  void OnPacketsReady(Cid cid, int number_packets) override;
+  void SetChannelRetransmissionFlowControlMode(Cid cid, RetransmissionAndFlowControlModeOption mode) override;
+  DataController* GetDataController(Cid cid) override;
 
  private:
   LowerQueueUpEnd* link_queue_up_end_;
   os::Handler* handler_;
-  std::unordered_map<Cid, Segmenter> segmenter_map_;
+  std::unordered_map<Cid, Sender> segmenter_map_;
   std::queue<std::pair<Cid, int>> next_to_dequeue_and_num_packets;
 
   bool link_queue_enqueue_registered_ = false;
diff --git a/gd/l2cap/internal/scheduler_mock.h b/gd/l2cap/internal/scheduler_mock.h
index 82e35a3..5fdffbe 100644
--- a/gd/l2cap/internal/scheduler_mock.h
+++ b/gd/l2cap/internal/scheduler_mock.h
@@ -30,7 +30,7 @@
  public:
   MOCK_METHOD(void, AttachChannel, (Cid cid, std::shared_ptr<l2cap::internal::ChannelImpl> channel), (override));
   MOCK_METHOD(void, DetachChannel, (Cid cid), (override));
-  MOCK_METHOD(void, NotifyPacketsReady, (Cid cid, int number_packet), (override));
+  MOCK_METHOD(void, OnPacketsReady, (Cid cid, int number_packet), (override));
 };
 
 }  // namespace testing
diff --git a/gd/l2cap/internal/segmenter.cc b/gd/l2cap/internal/segmenter.cc
deleted file mode 100644
index e1f4c10..0000000
--- a/gd/l2cap/internal/segmenter.cc
+++ /dev/null
@@ -1,92 +0,0 @@
-/*
- * Copyright 2019 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 <string>
-#include <unordered_map>
-
-#include "common/bind.h"
-#include "l2cap/cid.h"
-#include "l2cap/classic/internal/dynamic_channel_impl.h"
-#include "l2cap/internal/scheduler.h"
-#include "l2cap/internal/segmenter.h"
-#include "os/handler.h"
-#include "os/log.h"
-#include "os/queue.h"
-#include "packet/base_packet_builder.h"
-
-namespace bluetooth {
-namespace l2cap {
-namespace internal {
-
-Segmenter::Segmenter(os::Handler* handler, Scheduler* scheduler, std::shared_ptr<ChannelImpl> channel)
-    : handler_(handler), queue_end_(channel->GetQueueDownEnd()), scheduler_(scheduler), channel_id_(channel->GetCid()),
-      remote_channel_id_(channel->GetRemoteCid()), channel_(channel) {
-  try_register_dequeue();
-}
-
-Segmenter::~Segmenter() {
-  if (is_dequeue_registered_) {
-    queue_end_->UnregisterDequeue();
-  }
-}
-
-void Segmenter::NotifyPacketSent() {
-  try_register_dequeue();
-}
-
-std::unique_ptr<Segmenter::UpperDequeue> Segmenter::GetNextPacket() {
-  ASSERT_LOG(!pdu_buffer_.empty(), "No packet is available");
-  auto packet = std::move(pdu_buffer_.front());
-  pdu_buffer_.pop();
-  return packet;
-}
-
-void Segmenter::try_register_dequeue() {
-  if (is_dequeue_registered_) {
-    return;
-  }
-  queue_end_->RegisterDequeue(handler_, common::Bind(&Segmenter::dequeue_callback, common::Unretained(this)));
-  is_dequeue_registered_ = true;
-}
-
-void Segmenter::dequeue_callback() {
-  auto packet = queue_end_->TryDequeue();
-  ASSERT(packet != nullptr);
-  // TODO(hsz): Construct PDU(s) according to channel mode.
-  if (channel_ == nullptr || channel_->GetChannelMode() == RetransmissionAndFlowControlModeOption::L2CAP_BASIC) {
-    handle_basic_mode_sdu(std::move(packet));
-  }
-  if (channel_ != nullptr &&
-      channel_->GetChannelMode() == RetransmissionAndFlowControlModeOption::ENHANCED_RETRANSMISSION) {
-    handle_enhanced_retransmission_mode_sdu(std::move(packet));
-  }
-}
-
-void Segmenter::handle_basic_mode_sdu(std::unique_ptr<UpperDequeue> packet) {
-  auto pdu = BasicFrameBuilder::Create(remote_channel_id_, std::move(packet));
-  pdu_buffer_.emplace(std::move(pdu));
-  queue_end_->UnregisterDequeue();
-  is_dequeue_registered_ = false;
-  scheduler_->NotifyPacketsReady(channel_id_, 1);
-}
-
-void Segmenter::handle_enhanced_retransmission_mode_sdu(std::unique_ptr<UpperDequeue> packet) {
-  LOG_ERROR("Not implemented");
-}
-
-}  // namespace internal
-}  // namespace l2cap
-}  // namespace bluetooth
diff --git a/gd/l2cap/internal/sender.cc b/gd/l2cap/internal/sender.cc
new file mode 100644
index 0000000..5296eff
--- /dev/null
+++ b/gd/l2cap/internal/sender.cc
@@ -0,0 +1,96 @@
+/*
+ * Copyright 2019 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 <string>
+#include <unordered_map>
+
+#include "common/bind.h"
+#include "l2cap/cid.h"
+#include "l2cap/classic/internal/dynamic_channel_impl.h"
+#include "l2cap/internal/basic_mode_channel_data_controller.h"
+#include "l2cap/internal/enhanced_retransmission_mode_channel_data_controller.h"
+#include "l2cap/internal/scheduler.h"
+#include "l2cap/internal/sender.h"
+#include "os/handler.h"
+#include "os/log.h"
+#include "os/queue.h"
+#include "packet/base_packet_builder.h"
+
+namespace bluetooth {
+namespace l2cap {
+namespace internal {
+
+Sender::Sender(os::Handler* handler, Scheduler* scheduler, std::shared_ptr<ChannelImpl> channel)
+    : handler_(handler), queue_end_(channel->GetQueueDownEnd()), scheduler_(scheduler), channel_id_(channel->GetCid()),
+      remote_channel_id_(channel->GetRemoteCid()),
+      data_controller_(std::make_unique<BasicModeDataController>(channel_id_, remote_channel_id_, queue_end_, handler_,
+                                                                 scheduler_)) {
+  try_register_dequeue();
+}
+
+Sender::~Sender() {
+  if (is_dequeue_registered_) {
+    queue_end_->UnregisterDequeue();
+  }
+}
+
+void Sender::OnPacketSent() {
+  try_register_dequeue();
+}
+
+std::unique_ptr<Sender::UpperDequeue> Sender::GetNextPacket() {
+  return data_controller_->GetNextPacket();
+}
+
+void Sender::SetChannelRetransmissionFlowControlMode(RetransmissionAndFlowControlModeOption mode) {
+  if (mode_ == mode) {
+    return;
+  }
+  if (mode_ == RetransmissionAndFlowControlModeOption::L2CAP_BASIC) {
+    data_controller_ =
+        std::make_unique<BasicModeDataController>(channel_id_, remote_channel_id_, queue_end_, handler_, scheduler_);
+    return;
+  }
+  if (mode == RetransmissionAndFlowControlModeOption::ENHANCED_RETRANSMISSION) {
+    data_controller_ =
+        std::make_unique<ErtmController>(channel_id_, remote_channel_id_, queue_end_, handler_, scheduler_);
+    return;
+  }
+}
+
+DataController* Sender::GetDataController() {
+  return data_controller_.get();
+}
+
+void Sender::try_register_dequeue() {
+  if (is_dequeue_registered_) {
+    return;
+  }
+  queue_end_->RegisterDequeue(handler_, common::Bind(&Sender::dequeue_callback, common::Unretained(this)));
+  is_dequeue_registered_ = true;
+}
+
+void Sender::dequeue_callback() {
+  auto packet = queue_end_->TryDequeue();
+  ASSERT(packet != nullptr);
+  data_controller_->OnSdu(std::move(packet));
+  queue_end_->UnregisterDequeue();
+  is_dequeue_registered_ = false;
+}
+
+}  // namespace internal
+}  // namespace l2cap
+}  // namespace bluetooth
diff --git a/gd/l2cap/internal/segmenter.h b/gd/l2cap/internal/sender.h
similarity index 79%
rename from gd/l2cap/internal/segmenter.h
rename to gd/l2cap/internal/sender.h
index dd4a420..97d509c 100644
--- a/gd/l2cap/internal/segmenter.h
+++ b/gd/l2cap/internal/sender.h
@@ -21,8 +21,10 @@
 
 #include "common/bidi_queue.h"
 #include "common/bind.h"
+#include "data_controller.h"
 #include "l2cap/cid.h"
 #include "l2cap/internal/channel_impl.h"
+#include "l2cap/internal/data_controller.h"
 #include "os/handler.h"
 #include "os/queue.h"
 #include "packet/base_packet_builder.h"
@@ -37,40 +39,42 @@
  * A middle layer between L2CAP channel and outgoing packet scheduler.
  * Fetches data (SDU) from an L2CAP channel queue end, handles L2CAP segmentation, and gives data to L2CAP scheduler.
  */
-class Segmenter {
+class Sender {
  public:
   using UpperEnqueue = packet::PacketView<packet::kLittleEndian>;
   using UpperDequeue = packet::BasePacketBuilder;
   using UpperQueueDownEnd = common::BidiQueueEnd<UpperEnqueue, UpperDequeue>;
 
-  Segmenter(os::Handler* handler, Scheduler* scheduler, std::shared_ptr<ChannelImpl> channel);
-  ~Segmenter();
+  Sender(os::Handler* handler, Scheduler* scheduler, std::shared_ptr<ChannelImpl> channel);
+  ~Sender();
 
   /**
    * Callback from scheduler to indicate that scheduler already dequeued a packet from segmenter's queue.
    * Segmenter can continue dequeuing from channel queue end.
    */
-  void NotifyPacketSent();
+  void OnPacketSent();
 
   /**
    * Called by the scheduler to return the next PDU to be sent
    */
   std::unique_ptr<UpperDequeue> GetNextPacket();
 
+  void SetChannelRetransmissionFlowControlMode(RetransmissionAndFlowControlModeOption mode);
+
+  DataController* GetDataController();
+
  private:
   os::Handler* handler_;
   UpperQueueDownEnd* queue_end_;
-  std::queue<std::unique_ptr<UpperDequeue>> pdu_buffer_;
   Scheduler* scheduler_;
   const Cid channel_id_;
   const Cid remote_channel_id_;
-  std::shared_ptr<ChannelImpl> channel_;
   bool is_dequeue_registered_ = false;
+  RetransmissionAndFlowControlModeOption mode_ = RetransmissionAndFlowControlModeOption::L2CAP_BASIC;
+  std::unique_ptr<DataController> data_controller_;
 
   void try_register_dequeue();
   void dequeue_callback();
-  void handle_basic_mode_sdu(std::unique_ptr<UpperDequeue> packet);
-  void handle_enhanced_retransmission_mode_sdu(std::unique_ptr<UpperDequeue> packet);
 };
 }  // namespace internal
 }  // namespace l2cap
diff --git a/gd/l2cap/internal/segmenter_test.cc b/gd/l2cap/internal/sender_test.cc
similarity index 90%
rename from gd/l2cap/internal/segmenter_test.cc
rename to gd/l2cap/internal/sender_test.cc
index a1c1b24..8f67813 100644
--- a/gd/l2cap/internal/segmenter_test.cc
+++ b/gd/l2cap/internal/sender_test.cc
@@ -14,7 +14,7 @@
  * limitations under the License.
  */
 
-#include "l2cap/internal/segmenter.h"
+#include "l2cap/internal/sender.h"
 
 #include <gmock/gmock.h>
 #include <gtest/gtest.h>
@@ -42,7 +42,7 @@
 
 class FakeScheduler : public Scheduler {
  public:
-  void NotifyPacketsReady(Cid cid, int number_packets) override {
+  void OnPacketsReady(Cid cid, int number_packets) override {
     on_packets_ready_(cid, number_packets);
   }
 
@@ -54,7 +54,7 @@
 
 class L2capSegmenterTest : public ::testing::Test {
  public:
-  std::unique_ptr<Segmenter::UpperDequeue> enqueue_callback() {
+  std::unique_ptr<Sender::UpperDequeue> enqueue_callback() {
     auto packet_one = CreateSdu({1, 2, 3});
     channel_queue_.GetUpEnd()->UnregisterEnqueue();
     return packet_one;
@@ -71,7 +71,7 @@
         .WillRepeatedly(Return(RetransmissionAndFlowControlModeOption::L2CAP_BASIC));
     EXPECT_CALL(*mock_channel_, GetCid()).WillRepeatedly(Return(0x41));
     EXPECT_CALL(*mock_channel_, GetRemoteCid()).WillRepeatedly(Return(0x41));
-    segmenter_ = new Segmenter(queue_handler_, &scheduler_, mock_channel_);
+    segmenter_ = new Sender(queue_handler_, &scheduler_, mock_channel_);
   }
 
   void TearDown() override {
@@ -86,9 +86,9 @@
   os::Thread* thread_ = nullptr;
   os::Handler* user_handler_ = nullptr;
   os::Handler* queue_handler_ = nullptr;
-  common::BidiQueue<Segmenter::UpperEnqueue, Segmenter::UpperDequeue> channel_queue_{10};
+  common::BidiQueue<Sender::UpperEnqueue, Sender::UpperDequeue> channel_queue_{10};
   std::shared_ptr<testing::MockChannelImpl> mock_channel_;
-  Segmenter* segmenter_ = nullptr;
+  Sender* segmenter_ = nullptr;
   FakeScheduler scheduler_;
 };