Avoid using unfortunate terms in more places
We have switched all doc guides to use "mirror" or "secondary" years ago but these were never updated. Renaming functions and record/HTTP API fields (including CLI tools) would be major a breaking change, so they will be aliased or renamed with a lot more extensive review in the future.
This commit is contained in:
		
							parent
							
								
									b76bd6d653
								
							
						
					
					
						commit
						bd3c18963f
					
				|  | @ -688,7 +688,7 @@ with(#resource{} = Name, F, E, RetriesLeft) -> | |||
|               fun () -> F(Q) end); | ||||
|         %% The queue is supposed to be active. | ||||
|         %% The master node can go away or queue can be killed | ||||
|         %% so we retry, waiting for a slave to take over. | ||||
|         %% so we retry, waiting for a mirror to take over. | ||||
|         {ok, Q} when ?amqqueue_state_is(Q, live) -> | ||||
|             %% We check is_process_alive(QPid) in case we receive a | ||||
|             %% nodedown (for example) in F() that has nothing to do | ||||
|  | @ -1526,7 +1526,7 @@ wait_for_promoted_or_stopped(Q0) -> | |||
|                         true -> | ||||
|                             timer:sleep(100), | ||||
|                             wait_for_promoted_or_stopped(Q); | ||||
|                         %% All slave pids are stopped. | ||||
|                         %% All mirror pids are stopped. | ||||
|                         %% No process left for the queue | ||||
|                         false -> {stopped, Q} | ||||
|                     end | ||||
|  | @ -1857,8 +1857,8 @@ forget_all_durable(Node) -> | |||
|           end), | ||||
|     ok. | ||||
| 
 | ||||
| %% Try to promote a slave while down - it should recover as a | ||||
| %% master. We try to take the oldest slave here for best chance of | ||||
| %% Try to promote a mirror while down - it should recover as a | ||||
| %% master. We try to take the oldest mirror here for best chance of | ||||
| %% recovery. | ||||
| forget_node_for_queue(DeadNode, Q) | ||||
|   when ?amqqueue_is_quorum(Q) -> | ||||
|  | @ -1983,12 +1983,12 @@ maybe_clear_recoverable_node(Node, Q) -> | |||
|         true  -> | ||||
|             %% There is a race with | ||||
|             %% rabbit_mirror_queue_slave:record_synchronised/1 called | ||||
|             %% by the incoming slave node and this function, called | ||||
|             %% by the incoming mirror node and this function, called | ||||
|             %% by the master node. If this function is executed after | ||||
|             %% record_synchronised/1, the node is erroneously removed | ||||
|             %% from the recoverable mirrors list. | ||||
|             %% | ||||
|             %% We check if the slave node's queue PID is alive. If it is | ||||
|             %% We check if the mirror node's queue PID is alive. If it is | ||||
|             %% the case, then this function is executed after. In this | ||||
|             %% situation, we don't touch the queue record, it is already | ||||
|             %% correct. | ||||
|  | @ -2120,9 +2120,9 @@ deliver(Qs, Delivery = #delivery{flow = Flow, | |||
|                                  confirm = Confirm}, QueueState0) -> | ||||
|     {Quorum, MPids, SPids} = qpids(Qs), | ||||
|     QPids = MPids ++ SPids, | ||||
|     %% We use up two credits to send to a slave since the message | ||||
|     %% arrives at the slave from two directions. We will ack one when | ||||
|     %% the slave receives the message direct from the channel, and the | ||||
|     %% We use up two credits to send to a mirror since the message | ||||
|     %% arrives at the mirror from two directions. We will ack one when | ||||
|     %% the mirror receives the message direct from the channel, and the | ||||
|     %% other when it receives it via GM. | ||||
| 
 | ||||
|     case Flow of | ||||
|  |  | |||
|  | @ -1554,7 +1554,7 @@ handle_cast({deliver, | |||
|                end, | ||||
|     State1 = State#q{senders = Senders1}, | ||||
|     noreply(maybe_deliver_or_enqueue(Delivery, SlaveWhenPublished, State1)); | ||||
| %% [0] The second ack is since the channel thought we were a slave at | ||||
| %% [0] The second ack is since the channel thought we were a mirror at | ||||
| %% the time it published this message, so it used two credits (see | ||||
| %% rabbit_amqqueue:deliver/2). | ||||
| 
 | ||||
|  | @ -1656,7 +1656,7 @@ handle_cast(notify_decorators, State) -> | |||
| handle_cast(policy_changed, State = #q{q = Q0}) -> | ||||
|     Name = amqqueue:get_name(Q0), | ||||
|     %% We depend on the #q.q field being up to date at least WRT | ||||
|     %% policy (but not slave pids) in various places, so when it | ||||
|     %% policy (but not mirror pids) in various places, so when it | ||||
|     %% changes we go and read it from Mnesia again. | ||||
|     %% | ||||
|     %% This also has the side effect of waking us up so we emit a | ||||
|  | @ -1666,7 +1666,7 @@ handle_cast(policy_changed, State = #q{q = Q0}) -> | |||
| 
 | ||||
| handle_cast({sync_start, _, _}, State = #q{q = Q}) -> | ||||
|     Name = amqqueue:get_name(Q), | ||||
|     %% Only a slave should receive this, it means we are a duplicated master | ||||
|     %% Only a mirror should receive this, it means we are a duplicated master | ||||
|     rabbit_mirror_queue_misc:log_warning( | ||||
|       Name, "Stopping after receiving sync_start from another master", []), | ||||
|     stop(State). | ||||
|  |  | |||
|  | @ -95,8 +95,8 @@ | |||
| %% | ||||
| %% The key purpose of also sending messages directly from the channels | ||||
| %% to the mirrors is that without this, in the event of the death of | ||||
| %% the master, messages could be lost until a suitable slave is | ||||
| %% promoted. However, that is not the only reason. A slave cannot send | ||||
| %% the master, messages could be lost until a suitable mirror is | ||||
| %% promoted. However, that is not the only reason. A mirror cannot send | ||||
| %% confirms for a message until it has seen it from the | ||||
| %% channel. Otherwise, it might send a confirm to a channel for a | ||||
| %% message that it might *never* receive from that channel. This can | ||||
|  | @ -108,48 +108,48 @@ | |||
| %% | ||||
| %% Hence the mirrors have to wait until they've seen both the publish | ||||
| %% via gm, and the publish via the channel before they issue the | ||||
| %% confirm. Either form of publish can arrive first, and a slave can | ||||
| %% confirm. Either form of publish can arrive first, and a mirror can | ||||
| %% be upgraded to the master at any point during this | ||||
| %% process. Confirms continue to be issued correctly, however. | ||||
| %% | ||||
| %% Because the slave is a full process, it impersonates parts of the | ||||
| %% Because the mirror is a full process, it impersonates parts of the | ||||
| %% amqqueue API. However, it does not need to implement all parts: for | ||||
| %% example, no ack or consumer-related message can arrive directly at | ||||
| %% a slave from a channel: it is only publishes that pass both | ||||
| %% a mirror from a channel: it is only publishes that pass both | ||||
| %% directly to the mirrors and go via gm. | ||||
| %% | ||||
| %% Slaves can be added dynamically. When this occurs, there is no | ||||
| %% attempt made to sync the current contents of the master with the | ||||
| %% new slave, thus the slave will start empty, regardless of the state | ||||
| %% of the master. Thus the slave needs to be able to detect and ignore | ||||
| %% new slave, thus the mirror will start empty, regardless of the state | ||||
| %% of the master. Thus the mirror needs to be able to detect and ignore | ||||
| %% operations which are for messages it has not received: because of | ||||
| %% the strict FIFO nature of queues in general, this is | ||||
| %% straightforward - all new publishes that the new slave receives via | ||||
| %% straightforward - all new publishes that the new mirror receives via | ||||
| %% gm should be processed as normal, but fetches which are for | ||||
| %% messages the slave has never seen should be ignored. Similarly, | ||||
| %% acks for messages the slave never fetched should be | ||||
| %% messages the mirror has never seen should be ignored. Similarly, | ||||
| %% acks for messages the mirror never fetched should be | ||||
| %% ignored. Similarly, we don't republish rejected messages that we | ||||
| %% haven't seen. Eventually, as the master is consumed from, the | ||||
| %% messages at the head of the queue which were there before the slave | ||||
| %% joined will disappear, and the slave will become fully synced with | ||||
| %% joined will disappear, and the mirror will become fully synced with | ||||
| %% the state of the master. | ||||
| %% | ||||
| %% The detection of the sync-status is based on the depth of the BQs, | ||||
| %% where the depth is defined as the sum of the length of the BQ (as | ||||
| %% per BQ:len) and the messages pending an acknowledgement. When the | ||||
| %% depth of the slave is equal to the master's, then the slave is | ||||
| %% depth of the mirror is equal to the master's, then the mirror is | ||||
| %% synchronised. We only store the difference between the two for | ||||
| %% simplicity. Comparing the length is not enough since we need to | ||||
| %% take into account rejected messages which will make it back into | ||||
| %% the master queue but can't go back in the slave, since we don't | ||||
| %% want "holes" in the slave queue. Note that the depth, and the | ||||
| %% length likewise, must always be shorter on the slave - we assert | ||||
| %% want "holes" in the mirror queue. Note that the depth, and the | ||||
| %% length likewise, must always be shorter on the mirror - we assert | ||||
| %% that in various places. In case mirrors are joined to an empty queue | ||||
| %% which only goes on to receive publishes, they start by asking the | ||||
| %% master to broadcast its depth. This is enough for mirrors to always | ||||
| %% be able to work out when their head does not differ from the master | ||||
| %% (and is much simpler and cheaper than getting the master to hang on | ||||
| %% to the guid of the msg at the head of its queue). When a slave is | ||||
| %% to the guid of the msg at the head of its queue). When a mirror is | ||||
| %% promoted to a master, it unilaterally broadcasts its depth, in | ||||
| %% order to solve the problem of depth requests from new mirrors being | ||||
| %% unanswered by a dead master. | ||||
|  | @ -171,23 +171,23 @@ | |||
| %% over gm, the mirrors must convert the msg_ids to acktags (a mapping | ||||
| %% the mirrors themselves must maintain). | ||||
| %% | ||||
| %% When the master dies, a slave gets promoted. This will be the | ||||
| %% eldest slave, and thus the hope is that that slave is most likely | ||||
| %% When the master dies, a mirror gets promoted. This will be the | ||||
| %% eldest slave, and thus the hope is that that mirror is most likely | ||||
| %% to be sync'd with the master. The design of gm is that the | ||||
| %% notification of the death of the master will only appear once all | ||||
| %% messages in-flight from the master have been fully delivered to all | ||||
| %% members of the gm group. Thus at this point, the slave that gets | ||||
| %% members of the gm group. Thus at this point, the mirror that gets | ||||
| %% promoted cannot broadcast different events in a different order | ||||
| %% than the master for the same msgs: there is no possibility for the | ||||
| %% same msg to be processed by the old master and the new master - if | ||||
| %% it was processed by the old master then it will have been processed | ||||
| %% by the slave before the slave was promoted, and vice versa. | ||||
| %% by the mirror before the mirror was promoted, and vice versa. | ||||
| %% | ||||
| %% Upon promotion, all msgs pending acks are requeued as normal, the | ||||
| %% slave constructs state suitable for use in the master module, and | ||||
| %% mirror constructs state suitable for use in the master module, and | ||||
| %% then dynamically changes into an amqqueue_process with the master | ||||
| %% as the bq, and the slave's bq as the master's bq. Thus the very | ||||
| %% same process that was the slave is now a full amqqueue_process. | ||||
| %% same process that was the mirror is now a full amqqueue_process. | ||||
| %% | ||||
| %% It is important that we avoid memory leaks due to the death of | ||||
| %% senders (i.e. channels) and partial publications. A sender | ||||
|  | @ -200,7 +200,7 @@ | |||
| %% then hold on to the message, assuming they'll receive some | ||||
| %% instruction eventually from the master. Thus we have both mirrors | ||||
| %% and the master monitor all senders they become aware of. But there | ||||
| %% is a race: if the slave receives a DOWN of a sender, how does it | ||||
| %% is a race: if the mirror receives a DOWN of a sender, how does it | ||||
| %% know whether or not the master is going to send it instructions | ||||
| %% regarding those messages? | ||||
| %% | ||||
|  | @ -221,12 +221,12 @@ | |||
| %% master will ask the coordinator to set up a new monitor, and | ||||
| %% will continue to process the messages normally. Slaves may thus | ||||
| %% receive publishes via gm from previously declared "dead" senders, | ||||
| %% but again, this is fine: should the slave have just thrown out the | ||||
| %% but again, this is fine: should the mirror have just thrown out the | ||||
| %% message it had received directly from the sender (due to receiving | ||||
| %% a sender_death message via gm), it will be able to cope with the | ||||
| %% publication purely from the master via gm. | ||||
| %% | ||||
| %% When a slave receives a DOWN message for a sender, if it has not | ||||
| %% When a mirror receives a DOWN message for a sender, if it has not | ||||
| %% received the sender_death message from the master via gm already, | ||||
| %% then it will wait 20 seconds before broadcasting a request for | ||||
| %% confirmation from the master that the sender really has died. | ||||
|  | @ -235,9 +235,9 @@ | |||
| %% sender. The master will thus monitor the sender, receive the DOWN, | ||||
| %% and subsequently broadcast the sender_death message, allowing the | ||||
| %% mirrors to tidy up. This process can repeat for the same sender: | ||||
| %% consider one slave receives the publication, then the DOWN, then | ||||
| %% consider one mirror receives the publication, then the DOWN, then | ||||
| %% asks for confirmation of death, then the master broadcasts the | ||||
| %% sender_death message. Only then does another slave receive the | ||||
| %% sender_death message. Only then does another mirror receive the | ||||
| %% publication and thus set up its monitoring. Eventually that slave | ||||
| %% too will receive the DOWN, ask for confirmation and the master will | ||||
| %% monitor the sender again, receive another DOWN, and send out | ||||
|  | @ -245,30 +245,30 @@ | |||
| %% requesting death confirmation, this is highly unlikely, but it is a | ||||
| %% possibility. | ||||
| %% | ||||
| %% When the 20 second timer expires, the slave first checks to see | ||||
| %% When the 20 second timer expires, the mirror first checks to see | ||||
| %% whether it still needs confirmation of the death before requesting | ||||
| %% it. This prevents unnecessary traffic on gm as it allows one | ||||
| %% broadcast of the sender_death message to satisfy many mirrors. | ||||
| %% | ||||
| %% If we consider the promotion of a slave at this point, we have two | ||||
| %% possibilities: that of the slave that has received the DOWN and is | ||||
| %% If we consider the promotion of a mirror at this point, we have two | ||||
| %% possibilities: that of the mirror that has received the DOWN and is | ||||
| %% thus waiting for confirmation from the master that the sender | ||||
| %% really is down; and that of the slave that has not received the | ||||
| %% really is down; and that of the mirror that has not received the | ||||
| %% DOWN. In the first case, in the act of promotion to master, the new | ||||
| %% master will monitor again the dead sender, and after it has | ||||
| %% finished promoting itself, it should find another DOWN waiting, | ||||
| %% which it will then broadcast. This will allow mirrors to tidy up as | ||||
| %% normal. In the second case, we have the possibility that | ||||
| %% confirmation-of-sender-death request has been broadcast, but that | ||||
| %% it was broadcast before the master failed, and that the slave being | ||||
| %% it was broadcast before the master failed, and that the mirror being | ||||
| %% promoted does not know anything about that sender, and so will not | ||||
| %% monitor it on promotion. Thus a slave that broadcasts such a | ||||
| %% monitor it on promotion. Thus a mirror that broadcasts such a | ||||
| %% request, at the point of broadcasting it, recurses, setting another | ||||
| %% 20 second timer. As before, on expiry of the timer, the mirrors | ||||
| %% checks to see whether it still has not received a sender_death | ||||
| %% message for the dead sender, and if not, broadcasts a death | ||||
| %% confirmation request. Thus this ensures that even when a master | ||||
| %% dies and the new slave has no knowledge of the dead sender, it will | ||||
| %% dies and the new mirror has no knowledge of the dead sender, it will | ||||
| %% eventually receive a death confirmation request, shall monitor the | ||||
| %% dead sender, receive the DOWN and broadcast the sender_death | ||||
| %% message. | ||||
|  | @ -281,17 +281,17 @@ | |||
| %% mirrors will receive it via gm, will publish it to their BQ and will | ||||
| %% set up monitoring on the sender. They will then receive the DOWN | ||||
| %% message and the master will eventually publish the corresponding | ||||
| %% sender_death message. The slave will then be able to tidy up its | ||||
| %% sender_death message. The mirror will then be able to tidy up its | ||||
| %% state as normal. | ||||
| %% | ||||
| %% Recovery of mirrored queues is straightforward: as nodes die, the | ||||
| %% remaining nodes record this, and eventually a situation is reached | ||||
| %% in which only one node is alive, which is the master. This is the | ||||
| %% only node which, upon recovery, will resurrect a mirrored queue: | ||||
| %% nodes which die and then rejoin as a slave will start off empty as | ||||
| %% nodes which die and then rejoin as a mirror will start off empty as | ||||
| %% if they have no mirrored content at all. This is not surprising: to | ||||
| %% achieve anything more sophisticated would require the master and | ||||
| %% recovering slave to be able to check to see whether they agree on | ||||
| %% recovering mirror to be able to check to see whether they agree on | ||||
| %% the last seen state of the queue: checking depth alone is not | ||||
| %% sufficient in this case. | ||||
| %% | ||||
|  | @ -361,8 +361,8 @@ handle_cast({gm_deaths, DeadGMPids}, State = #state{q = Q}) when ?amqqueue_pid_r | |||
|             noreply(State); | ||||
|         {ok, _MPid0, DeadPids, _ExtraNodes} -> | ||||
|             %% see rabbitmq-server#914; | ||||
|             %% Different slave is now master, stop current coordinator normally. | ||||
|             %% Initiating queue is now slave and the least we could do is report | ||||
|             %% Different mirror is now master, stop current coordinator normally. | ||||
|             %% Initiating queue is now mirror and the least we could do is report | ||||
|             %% deaths which we 'think' we saw. | ||||
|             %% NOTE: Reported deaths here, could be inconsistent. | ||||
|             rabbit_mirror_queue_misc:report_deaths(MPid, false, QueueName, | ||||
|  | @ -416,7 +416,7 @@ code_change(_OldVsn, State, _Extra) -> | |||
| 
 | ||||
| handle_pre_hibernate(State = #state { gm = GM }) -> | ||||
|     %% Since GM notifications of deaths are lazy we might not get a | ||||
|     %% timely notification of slave death if policy changes when | ||||
|     %% timely notification of mirror death if policy changes when | ||||
|     %% everything is idle. So cause some activity just before we | ||||
|     %% sleep. This won't cause us to go into perpetual motion as the | ||||
|     %% heartbeat does not wake up coordinator or mirrors. | ||||
|  |  | |||
|  | @ -117,7 +117,7 @@ init_with_existing_bq(Q0, BQ, BQS) when ?is_amqqueue(Q0) -> | |||
|         ok = rabbit_misc:execute_mnesia_transaction(Fun), | ||||
|         {_MNode, SNodes} = rabbit_mirror_queue_misc:suggested_queue_nodes(Q0), | ||||
|         %% We need synchronous add here (i.e. do not return until the | ||||
|         %% slave is running) so that when queue declaration is finished | ||||
|         %% mirror is running) so that when queue declaration is finished | ||||
|         %% all mirrors are up; we don't want to end up with unsynced mirrors | ||||
|         %% just by declaring a new queue. But add can't be synchronous all | ||||
|         %% the time as it can be called by mirrors and that's | ||||
|  | @ -209,7 +209,7 @@ terminate(Reason, | |||
|                    QName, "Stopping all nodes on master shutdown since no " | ||||
|                    "synchronised mirror (replica) is available~n", []), | ||||
|                  stop_all_slaves(Reason, State); | ||||
|         false -> %% Just let some other slave take over. | ||||
|         false -> %% Just let some other mirror take over. | ||||
|                  ok | ||||
|     end, | ||||
|     State #state { backing_queue_state = BQ:terminate(Reason, BQS) }. | ||||
|  | @ -262,7 +262,7 @@ batch_publish(Publishes, ChPid, Flow, | |||
|                       MsgSizes), | ||||
|     BQS1 = BQ:batch_publish(Publishes2, ChPid, Flow, BQS), | ||||
|     ensure_monitoring(ChPid, State #state { backing_queue_state = BQS1 }). | ||||
| %% [0] When the slave process handles the publish command, it sets the | ||||
| %% [0] When the mirror process handles the publish command, it sets the | ||||
| %% IsDelivered flag to true, so to avoid iterating over the messages | ||||
| %% again at the slave, we do it here. | ||||
| 
 | ||||
|  | @ -464,7 +464,7 @@ is_duplicate(Message = #basic_message { id = MsgId }, | |||
|             {Result, BQS1} = BQ:is_duplicate(Message, BQS), | ||||
|             {Result, State #state { backing_queue_state = BQS1 }}; | ||||
|         {ok, published} -> | ||||
|             %% It already got published when we were a slave and no | ||||
|             %% It already got published when we were a mirror and no | ||||
|             %% confirmation is waiting. amqqueue_process will have, in | ||||
|             %% its msg_id_to_channel mapping, the entry for dealing | ||||
|             %% with the confirm when that comes back in (it's added | ||||
|  | @ -474,7 +474,7 @@ is_duplicate(Message = #basic_message { id = MsgId }, | |||
|             {{true, drop}, State #state { seen_status = maps:remove(MsgId, SS) }}; | ||||
|         {ok, Disposition} | ||||
|           when Disposition =:= confirmed | ||||
|             %% It got published when we were a slave via gm, and | ||||
|             %% It got published when we were a mirror via gm, and | ||||
|             %% confirmed some time after that (maybe even after | ||||
|             %% promotion), but before we received the publish from the | ||||
|             %% channel, so couldn't previously know what the | ||||
|  |  | |||
|  | @ -91,7 +91,7 @@ remove_from_queue(QueueName, Self, DeadGMPids) -> | |||
|                                                 %% GM altered, & if all pids are | ||||
|                                                 %% perceived as dead, rather do | ||||
|                                                 %% do nothing here, & trust the | ||||
|                                                 %% promoted slave to have updated | ||||
|                                                 %% promoted mirror to have updated | ||||
|                                                 %% mnesia during the alteration. | ||||
|                                                 {QPid, SPids}; | ||||
|                                             _  -> promote_slave(Alive) | ||||
|  | @ -133,16 +133,16 @@ remove_from_queue(QueueName, Self, DeadGMPids) -> | |||
|                       end | ||||
|               end | ||||
|       end). | ||||
| %% [1] We still update mnesia here in case the slave that is supposed | ||||
| %% [1] We still update mnesia here in case the mirror that is supposed | ||||
| %% to become master dies before it does do so, in which case the dead | ||||
| %% old master might otherwise never get removed, which in turn might | ||||
| %% prevent promotion of another slave (e.g. us). | ||||
| %% prevent promotion of another mirror (e.g. us). | ||||
| %% | ||||
| %% Note however that we do not update the master pid. Otherwise we can | ||||
| %% have the situation where a slave updates the mnesia record for a | ||||
| %% queue, promoting another slave before that slave realises it has | ||||
| %% have the situation where a mirror updates the mnesia record for a | ||||
| %% queue, promoting another mirror before that mirror realises it has | ||||
| %% become the new master, which is bad because it could then mean the | ||||
| %% slave (now master) receives messages it's not ready for (for | ||||
| %% mirror (now master) receives messages it's not ready for (for | ||||
| %% example, new consumers). | ||||
| %% | ||||
| %% We set slave_pids to Alive rather than SPids1 since otherwise we'd | ||||
|  | @ -156,7 +156,7 @@ remove_from_queue(QueueName, Self, DeadGMPids) -> | |||
| %% aforementioned restriction on updating the master pid, that pid may | ||||
| %% not be present in gm_pids, but only if said master has died. | ||||
| 
 | ||||
| %% Sometimes a slave dying means we need to start more on other | ||||
| %% Sometimes a mirror dying means we need to start more on other | ||||
| %% nodes - "exactly" mode can cause this to happen. | ||||
| slaves_to_start_on_failure(Q, DeadGMPids) -> | ||||
|     %% In case Mnesia has not caught up yet, filter out nodes we know | ||||
|  | @ -358,7 +358,7 @@ stop_all_slaves(Reason, SPids, QName, GM, WaitTimeout) -> | |||
|                 Acc | ||||
|         end | ||||
|     end, [], PidsMRefs), | ||||
|     %% Normally when we remove a slave another slave or master will | ||||
|     %% Normally when we remove a mirror another mirror or master will | ||||
|     %% notice and update Mnesia. But we just removed them all, and | ||||
|     %% have stopped listening ourselves. So manually clean up. | ||||
|     rabbit_misc:execute_mnesia_transaction(fun () -> | ||||
|  | @ -367,7 +367,7 @@ stop_all_slaves(Reason, SPids, QName, GM, WaitTimeout) -> | |||
|         Q2 = amqqueue:set_slave_pids(Q1, []), | ||||
|         %% Restarted mirrors on running nodes can | ||||
|         %% ensure old incarnations are stopped using | ||||
|         %% the pending slave pids. | ||||
|         %% the pending mirror pids. | ||||
|         Q3 = amqqueue:set_slave_pids_pending_shutdown(Q2, PendingSlavePids), | ||||
|         rabbit_mirror_queue_misc:store_updated_slaves(Q3) | ||||
|     end), | ||||
|  | @ -376,7 +376,7 @@ stop_all_slaves(Reason, SPids, QName, GM, WaitTimeout) -> | |||
| %%---------------------------------------------------------------------------- | ||||
| 
 | ||||
| promote_slave([SPid | SPids]) -> | ||||
|     %% The slave pids are maintained in descending order of age, so | ||||
|     %% The mirror pids are maintained in descending order of age, so | ||||
|     %% the one to promote is the oldest. | ||||
|     {SPid, SPids}. | ||||
| 
 | ||||
|  | @ -587,12 +587,12 @@ wait_for_new_master(QName, Destination, N) -> | |||
|             end | ||||
|     end. | ||||
| 
 | ||||
| %% The arrival of a newly synced slave may cause the master to die if | ||||
| %% The arrival of a newly synced mirror may cause the master to die if | ||||
| %% the policy does not want the master but it has been kept alive | ||||
| %% because there were no synced mirrors. | ||||
| %% | ||||
| %% We don't just call update_mirrors/2 here since that could decide to | ||||
| %% start a slave for some other reason, and since we are the slave ATM | ||||
| %% start a mirror for some other reason, and since we are the mirror ATM | ||||
| %% that allows complicated deadlocks. | ||||
| 
 | ||||
| -spec maybe_drop_master_after_sync(amqqueue:amqqueue()) -> 'ok'. | ||||
|  |  | |||
|  | @ -307,8 +307,8 @@ handle_cast({gm, Instruction}, State = #state{q = Q0}) when ?is_amqqueue(Q0) -> | |||
|                true -> | ||||
|                    handle_process_result(process_instruction(Instruction, State)); | ||||
|                false -> | ||||
|                    %% Potentially a duplicated slave caused by a partial partition, | ||||
|                    %% will stop as a new slave could start unaware of our presence | ||||
|                    %% Potentially a duplicated mirror caused by a partial partition, | ||||
|                    %% will stop as a new mirror could start unaware of our presence | ||||
|                    {stop, shutdown, State} | ||||
|            end; | ||||
|        {error, not_found} -> | ||||
|  | @ -637,7 +637,7 @@ promote_me(From, #state { q                   = Q0, | |||
|                           msg_id_status       = MS, | ||||
|                           known_senders       = KS}) when ?is_amqqueue(Q0) -> | ||||
|     QName = amqqueue:get_name(Q0), | ||||
|     rabbit_mirror_queue_misc:log_info(QName, "Promoting slave ~s to master~n", | ||||
|     rabbit_mirror_queue_misc:log_info(QName, "Promoting mirror ~s to master~n", | ||||
|                                       [rabbit_misc:pid_to_string(self())]), | ||||
|     Q1 = amqqueue:set_pid(Q0, self()), | ||||
|     DeathFun = rabbit_mirror_queue_master:sender_death_fun(), | ||||
|  | @ -827,7 +827,7 @@ forget_sender(down_from_gm, down_from_gm)        -> false; %% [1] | |||
| forget_sender(down_from_ch, down_from_ch)        -> false; | ||||
| forget_sender(Down1, Down2) when Down1 =/= Down2 -> true. | ||||
| 
 | ||||
| %% [1] If another slave goes through confirm_sender_death/1 before we | ||||
| %% [1] If another mirror goes through confirm_sender_death/1 before we | ||||
| %% do we can get two GM sender_death messages in a row for the same | ||||
| %% channel - don't treat that as anything special. | ||||
| 
 | ||||
|  |  | |||
|  | @ -213,7 +213,7 @@ await_slaves(Ref, SPids) -> | |||
|                      {'DOWN', _, process, SPid, _} -> false | ||||
|                  end]. | ||||
| %% [0] This check is in case there's been a partition which has then | ||||
| %% healed in between the master retrieving the slave pids from Mnesia | ||||
| %% healed in between the master retrieving the mirror pids from Mnesia | ||||
| %% and sending 'sync_start' over GM. If so there might be mirrors on the | ||||
| %% other side of the partition which we can monitor (since they have | ||||
| %% rejoined the distributed system with us) but which did not get the | ||||
|  |  | |||
|  | @ -439,7 +439,7 @@ nodes_policy_should_pick_master_from_its_params(Config) -> | |||
|     Info = find_queue(?QNAME, A), | ||||
|     SSPids = proplists:get_value(synchronised_slave_pids, Info), | ||||
| 
 | ||||
|     %% Choose slave that isn't the first sync slave. Cover a bug that always | ||||
|     %% Choose mirror that isn't the first sync slave. Cover a bug that always | ||||
|     %% chose the first, even if it was not part of the policy | ||||
|     LastSlave = node(lists:last(SSPids)), | ||||
|     ?assertEqual(true, apply_policy_to_declared_queue(Config, Ch, [A], | ||||
|  |  | |||
|  | @ -447,7 +447,7 @@ mirror_queue_sync(Config) -> | |||
|     ok = rabbit_ct_broker_helpers:set_ha_policy(Config, 0, | ||||
|       <<"^mirror_queue_sync-queue$">>, <<"all">>), | ||||
|     publish(Ch, Q, [1, 2, 3, 1, 2, 3]), | ||||
|     %% master now has 9, slave 6. | ||||
|     %% master now has 9, mirror 6. | ||||
|     get_partial(Ch, Q, manual_ack, [3, 3, 3, 2, 2, 2]), | ||||
|     %% So some but not all are unacked at the slave | ||||
|     Nodename0 = rabbit_ct_broker_helpers:get_node_config(Config, 0, nodename), | ||||
|  | @ -519,7 +519,7 @@ mirror_queue_auto_ack(Config) -> | |||
|     %% Restart one of the mirrors so `request_depth` is triggered | ||||
|     rabbit_ct_broker_helpers:restart_node(Config, SNode1), | ||||
| 
 | ||||
|     %% The alive slave must have the same pid after its neighbour is restarted | ||||
|     %% The alive mirror must have the same pid after its neighbour is restarted | ||||
|     timer:sleep(3000), %% ugly but we can't know when the `depth` instruction arrives | ||||
|     Slaves = nodes_and_pids(slave_pids(Config, A, rabbit_misc:r(<<"/">>, queue, Q))), | ||||
|     SPid2 = proplists:get_value(SNode2, Slaves), | ||||
|  |  | |||
|  | @ -98,9 +98,9 @@ slave_synchronization(Config) -> | |||
|     %% the master. | ||||
|     rabbit_ct_broker_helpers:stop_broker(Config, Slave), | ||||
| 
 | ||||
|     %% We get and ack one message when the slave is down, and check that when we | ||||
|     %% start the slave it's not marked as synced until ack the message.  We also | ||||
|     %% publish another message when the slave is up. | ||||
|     %% We get and ack one message when the mirror is down, and check that when we | ||||
|     %% start the mirror it's not marked as synced until ack the message.  We also | ||||
|     %% publish another message when the mirror is up. | ||||
|     send_dummy_message(Channel, Queue),                                 % 1 - 0 | ||||
|     {#'basic.get_ok'{delivery_tag = Tag1}, _} = | ||||
|         amqp_channel:call(Channel, #'basic.get'{queue = Queue}),        % 0 - 1 | ||||
|  | @ -115,7 +115,7 @@ slave_synchronization(Config) -> | |||
| 
 | ||||
|     slave_synced(Master, Queue), | ||||
| 
 | ||||
|     %% We restart the slave and we send a message, so that the slave will only | ||||
|     %% We restart the mirror and we send a message, so that the mirror will only | ||||
|     %% have one of the messages. | ||||
|     rabbit_ct_broker_helpers:stop_broker(Config, Slave), | ||||
|     rabbit_ct_broker_helpers:start_broker(Config, Slave), | ||||
|  | @ -124,7 +124,7 @@ slave_synchronization(Config) -> | |||
| 
 | ||||
|     slave_unsynced(Master, Queue), | ||||
| 
 | ||||
|     %% We reject the message that the slave doesn't have, and verify that it's | ||||
|     %% We reject the message that the mirror doesn't have, and verify that it's | ||||
|     %% still unsynced | ||||
|     {#'basic.get_ok'{delivery_tag = Tag2}, _} = | ||||
|         amqp_channel:call(Channel, #'basic.get'{queue = Queue}),        % 1 - 1 | ||||
|  |  | |||
|  | @ -673,7 +673,7 @@ class Lister: | |||
|                         if depth < max_depth: | ||||
|                             add(column, depth + 1, subitem, fun) | ||||
|                 elif type(subitem) == list: | ||||
|                     # The first branch has slave nodes in queues in | ||||
|                     # The first branch has mirror nodes in queues in | ||||
|                     # mind (which come out looking decent); the second | ||||
|                     # one has applications in nodes (which look less | ||||
|                     # so, but what would look good?). | ||||
|  |  | |||
		Loading…
	
		Reference in New Issue