# This test demonstrates the "lazy replication" feature. The leader sends MsgApp
# messages to StateReplication peers only when requested explicitly by the
# application.

# Skip logging the boilerplate. Set up a raft group of 3 nodes, and elect node 1
# as the leader. Nodes 2 and 3 are the followers.
log-level none
----
ok

add-nodes 3 voters=(1,2,3) index=10 lazy-replication=true
----
ok

campaign 1
----
ok

stabilize
----
ok

log-level debug
----
ok

# Propose a couple of entries.
propose 1 data-1
----
ok

propose 1 data-2
----
ok

# NB: no entries are sent to the followers yet.
stabilize
----
> 1 handling Ready
  Ready MustSync=true:
  Entries:
  1/12 EntryNormal "data-1"
  1/13 EntryNormal "data-2"

# Attempt to send a misaligned MsgApp. No-op.
send-msg-app 1 to=2 lo=10 hi=13
----
could not send MsgApp (10,13] to 2

# Send a MsgApp to node 2, containing both entries.
send-msg-app 1 to=2 lo=11 hi=13
----
1->2 MsgApp Term:1 Log:1/11 Commit:11 Entries:[
  1/12 EntryNormal "data-1"
  1/13 EntryNormal "data-2"
]

# Send a MsgApp to node 3, containing only one entry.
send-msg-app 1 to=3 lo=11 hi=12
----
1->3 MsgApp Term:1 Log:1/11 Commit:11 Entries:[1/12 EntryNormal "data-1"]

# The followers receive the entries and reply to the leader. The leader commits
# both entries, but the replication flow to node 3 still has one pending entry.
stabilize
----
> 2 receiving messages
  1->2 MsgApp Term:1 Log:1/11 Commit:11 Entries:[
    1/12 EntryNormal "data-1"
    1/13 EntryNormal "data-2"
  ]
> 3 receiving messages
  1->3 MsgApp Term:1 Log:1/11 Commit:11 Entries:[1/12 EntryNormal "data-1"]
> 2 handling Ready
  Ready MustSync=true:
  Entries:
  1/12 EntryNormal "data-1"
  1/13 EntryNormal "data-2"
  Messages:
  2->1 MsgAppResp Term:1 Log:0/13 Commit:11
> 3 handling Ready
  Ready MustSync=true:
  Entries:
  1/12 EntryNormal "data-1"
  Messages:
  3->1 MsgAppResp Term:1 Log:0/12 Commit:11
> 1 receiving messages
  2->1 MsgAppResp Term:1 Log:0/13 Commit:11
  3->1 MsgAppResp Term:1 Log:0/12 Commit:11
> 1 handling Ready
  Ready MustSync=true:
  HardState Term:1 Vote:1 Commit:13 Lead:1 LeadEpoch:1
  CommittedEntries:
  1/12 EntryNormal "data-1"
  1/13 EntryNormal "data-2"
  Messages:
  1->2 MsgApp Term:1 Log:1/13 Commit:13
  1->3 MsgApp Term:1 Log:1/12 Commit:13
> 2 receiving messages
  1->2 MsgApp Term:1 Log:1/13 Commit:13
> 3 receiving messages
  1->3 MsgApp Term:1 Log:1/12 Commit:13
> 2 handling Ready
  Ready MustSync=true:
  HardState Term:1 Vote:1 Commit:13 Lead:1 LeadEpoch:1
  CommittedEntries:
  1/12 EntryNormal "data-1"
  1/13 EntryNormal "data-2"
  Messages:
  2->1 MsgAppResp Term:1 Log:0/13 Commit:13
> 3 handling Ready
  Ready MustSync=true:
  HardState Term:1 Vote:1 Commit:12 Lead:1 LeadEpoch:1
  CommittedEntries:
  1/12 EntryNormal "data-1"
  Messages:
  3->1 MsgAppResp Term:1 Log:0/12 Commit:12
> 1 receiving messages
  2->1 MsgAppResp Term:1 Log:0/13 Commit:13
  3->1 MsgAppResp Term:1 Log:0/12 Commit:12

# One entry still to be replicated to node 3.
status 1
----
1: StateReplicate match=13 next=14 sentCommit=11 matchCommit=11
2: StateReplicate match=13 next=14 sentCommit=13 matchCommit=13
3: StateReplicate match=12 next=13 sentCommit=13 matchCommit=12

# Replicate it.
send-msg-app 1 to=3 lo=12 hi=13
----
1->3 MsgApp Term:1 Log:1/12 Commit:13 Entries:[1/13 EntryNormal "data-2"]

stabilize
----
> 3 receiving messages
  1->3 MsgApp Term:1 Log:1/12 Commit:13 Entries:[1/13 EntryNormal "data-2"]
> 3 handling Ready
  Ready MustSync=true:
  HardState Term:1 Vote:1 Commit:13 Lead:1 LeadEpoch:1
  Entries:
  1/13 EntryNormal "data-2"
  CommittedEntries:
  1/13 EntryNormal "data-2"
  Messages:
  3->1 MsgAppResp Term:1 Log:0/13 Commit:13
> 1 receiving messages
  3->1 MsgAppResp Term:1 Log:0/13 Commit:13

# The leader has converged to a fully replicated state.
status 1
----
1: StateReplicate match=13 next=14 sentCommit=11 matchCommit=11
2: StateReplicate match=13 next=14 sentCommit=13 matchCommit=13
3: StateReplicate match=13 next=14 sentCommit=13 matchCommit=13

################################################################################
# Test switching back to the eager replication mode.

propose 1 data-3
----
ok

# MsgApps are still not sent eagerly.
stabilize
----
> 1 handling Ready
  Ready MustSync=true:
  Entries:
  1/14 EntryNormal "data-3"

set-lazy-replication 1 false
----
ok

# Immediately after the switch, the appends are broadcast to the followers.
stabilize
----
> 1 handling Ready
  Ready MustSync=false:
  Messages:
  1->2 MsgApp Term:1 Log:1/13 Commit:13 Entries:[1/14 EntryNormal "data-3"]
  1->3 MsgApp Term:1 Log:1/13 Commit:13 Entries:[1/14 EntryNormal "data-3"]
> 2 receiving messages
  1->2 MsgApp Term:1 Log:1/13 Commit:13 Entries:[1/14 EntryNormal "data-3"]
> 3 receiving messages
  1->3 MsgApp Term:1 Log:1/13 Commit:13 Entries:[1/14 EntryNormal "data-3"]
> 2 handling Ready
  Ready MustSync=true:
  Entries:
  1/14 EntryNormal "data-3"
  Messages:
  2->1 MsgAppResp Term:1 Log:0/14 Commit:13
> 3 handling Ready
  Ready MustSync=true:
  Entries:
  1/14 EntryNormal "data-3"
  Messages:
  3->1 MsgAppResp Term:1 Log:0/14 Commit:13
> 1 receiving messages
  2->1 MsgAppResp Term:1 Log:0/14 Commit:13
  3->1 MsgAppResp Term:1 Log:0/14 Commit:13
> 1 handling Ready
  Ready MustSync=true:
  HardState Term:1 Vote:1 Commit:14 Lead:1 LeadEpoch:1
  CommittedEntries:
  1/14 EntryNormal "data-3"
  Messages:
  1->2 MsgApp Term:1 Log:1/14 Commit:14
  1->3 MsgApp Term:1 Log:1/14 Commit:14
> 2 receiving messages
  1->2 MsgApp Term:1 Log:1/14 Commit:14
> 3 receiving messages
  1->3 MsgApp Term:1 Log:1/14 Commit:14
> 2 handling Ready
  Ready MustSync=true:
  HardState Term:1 Vote:1 Commit:14 Lead:1 LeadEpoch:1
  CommittedEntries:
  1/14 EntryNormal "data-3"
  Messages:
  2->1 MsgAppResp Term:1 Log:0/14 Commit:14
> 3 handling Ready
  Ready MustSync=true:
  HardState Term:1 Vote:1 Commit:14 Lead:1 LeadEpoch:1
  CommittedEntries:
  1/14 EntryNormal "data-3"
  Messages:
  3->1 MsgAppResp Term:1 Log:0/14 Commit:14
> 1 receiving messages
  2->1 MsgAppResp Term:1 Log:0/14 Commit:14
  3->1 MsgAppResp Term:1 Log:0/14 Commit:14

# A new proposal triggers replication immediately.
propose 1 data-4
----
ok

# If we switch back to lazy replication while there are pending MsgApp messages
# in the queue (generated after the proposal above), the messages will pop up in
# the next Ready.
set-lazy-replication 1 true
----
ok

stabilize
----
> 1 handling Ready
  Ready MustSync=true:
  Entries:
  1/15 EntryNormal "data-4"
  Messages:
  1->2 MsgApp Term:1 Log:1/14 Commit:14 Entries:[1/15 EntryNormal "data-4"]
  1->3 MsgApp Term:1 Log:1/14 Commit:14 Entries:[1/15 EntryNormal "data-4"]
> 2 receiving messages
  1->2 MsgApp Term:1 Log:1/14 Commit:14 Entries:[1/15 EntryNormal "data-4"]
> 3 receiving messages
  1->3 MsgApp Term:1 Log:1/14 Commit:14 Entries:[1/15 EntryNormal "data-4"]
> 2 handling Ready
  Ready MustSync=true:
  Entries:
  1/15 EntryNormal "data-4"
  Messages:
  2->1 MsgAppResp Term:1 Log:0/15 Commit:14
> 3 handling Ready
  Ready MustSync=true:
  Entries:
  1/15 EntryNormal "data-4"
  Messages:
  3->1 MsgAppResp Term:1 Log:0/15 Commit:14
> 1 receiving messages
  2->1 MsgAppResp Term:1 Log:0/15 Commit:14
  3->1 MsgAppResp Term:1 Log:0/15 Commit:14
> 1 handling Ready
  Ready MustSync=true:
  HardState Term:1 Vote:1 Commit:15 Lead:1 LeadEpoch:1
  CommittedEntries:
  1/15 EntryNormal "data-4"
  Messages:
  1->2 MsgApp Term:1 Log:1/15 Commit:15
  1->3 MsgApp Term:1 Log:1/15 Commit:15
> 2 receiving messages
  1->2 MsgApp Term:1 Log:1/15 Commit:15
> 3 receiving messages
  1->3 MsgApp Term:1 Log:1/15 Commit:15
> 2 handling Ready
  Ready MustSync=true:
  HardState Term:1 Vote:1 Commit:15 Lead:1 LeadEpoch:1
  CommittedEntries:
  1/15 EntryNormal "data-4"
  Messages:
  2->1 MsgAppResp Term:1 Log:0/15 Commit:15
> 3 handling Ready
  Ready MustSync=true:
  HardState Term:1 Vote:1 Commit:15 Lead:1 LeadEpoch:1
  CommittedEntries:
  1/15 EntryNormal "data-4"
  Messages:
  3->1 MsgAppResp Term:1 Log:0/15 Commit:15
> 1 receiving messages
  2->1 MsgAppResp Term:1 Log:0/15 Commit:15
  3->1 MsgAppResp Term:1 Log:0/15 Commit:15

# After the Ready handling, messages are no longer sent eagerly.
propose 1 data-5
----
ok

stabilize
----
> 1 handling Ready
  Ready MustSync=true:
  Entries:
  1/16 EntryNormal "data-5"
