summaryrefslogtreecommitdiff
path: root/zookeeper-server/src/main/java/org/apache/zookeeper/server/quorum/Observer.java
blob: 55e273b07133f4e2d38c27c121b8bea9c3f11279 (plain)
1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
18
19
20
21
22
23
24
25
26
27
28
29
30
31
32
33
34
35
36
37
38
39
40
41
42
43
44
45
46
47
48
49
50
51
52
53
54
55
56
57
58
59
60
61
62
63
64
65
66
67
68
69
70
71
72
73
74
75
76
77
78
79
80
81
82
83
84
85
86
87
88
89
90
91
92
93
94
95
96
97
98
99
100
101
102
103
104
105
106
107
108
109
110
111
112
113
114
115
116
117
118
119
120
121
122
123
124
125
126
127
128
129
130
131
132
133
134
135
136
137
138
139
140
141
142
143
144
145
146
147
148
149
150
151
152
153
154
155
156
157
158
159
160
161
162
163
164
165
166
167
168
169
170
171
172
173
174
175
176
177
178
179
180
181
182
183
184
185
186
187
188
189
190
191
192
193
194
195
196
197
198
199
200
201
202
203
204
205
206
207
208
209
210
211
212
213
214
215
216
217
218
219
220
221
222
223
224
225
226
227
228
229
230
231
232
233
234
235
236
237
238
239
240
241
242
243
244
245
246
247
248
249
250
251
252
253
254
255
256
257
258
259
260
261
262
263
264
265
266
267
268
269
270
271
272
273
274
275
276
277
278
279
280
281
282
283
284
285
286
287
288
289
290
291
292
293
294
295
296
297
298
299
300
301
302
303
304
305
306
307
/*
 * Licensed to the Apache Software Foundation (ASF) under one
 * or more contributor license agreements.  See the NOTICE file
 * distributed with this work for additional information
 * regarding copyright ownership.  The ASF licenses this file
 * to you 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.
 */

package org.apache.zookeeper.server.quorum;

import static java.nio.charset.StandardCharsets.UTF_8;
import java.nio.ByteBuffer;
import java.util.concurrent.ThreadLocalRandom;
import java.util.concurrent.atomic.AtomicReference;
import org.apache.jute.Record;
import org.apache.zookeeper.server.ObserverBean;
import org.apache.zookeeper.server.Request;
import org.apache.zookeeper.server.ServerMetrics;
import org.apache.zookeeper.server.TxnLogEntry;
import org.apache.zookeeper.server.quorum.QuorumPeer.QuorumServer;
import org.apache.zookeeper.server.quorum.flexible.QuorumVerifier;
import org.apache.zookeeper.server.util.SerializeUtils;
import org.apache.zookeeper.txn.SetDataTxn;
import org.apache.zookeeper.txn.TxnDigest;
import org.apache.zookeeper.txn.TxnHeader;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;

/**
 * Observers are peers that do not take part in the atomic broadcast protocol.
 * Instead, they are informed of successful proposals by the Leader. Observers
 * therefore naturally act as a relay point for publishing the proposal stream
 * and can relieve Followers of some of the connection load. Observers may
 * submit proposals, but do not vote in their acceptance.
 *
 * See ZOOKEEPER-368 for a discussion of this feature.
 */
public class Observer extends Learner {

    private static final Logger LOG = LoggerFactory.getLogger(Observer.class);

    /**
     * When observer lost its connection with the leader, it waits for 0 to the
     * specified value before trying to reconnect with the leader. So that
     * the entire observer fleet won't try to run leader election and reconnect
     * to the leader at once. Default value is zero.
     */
    public static final String OBSERVER_RECONNECT_DELAY_MS = "zookeeper.observer.reconnectDelayMs";

    /**
     * Delay the Observer's participation in a leader election upon disconnect
     * so as to prevent unexpected additional load on the voting peers during
     * the process. Default value is 200.
     */
    public static final String OBSERVER_ELECTION_DELAY_MS = "zookeeper.observer.election.DelayMs";

    private static final long reconnectDelayMs;

    private static volatile long observerElectionDelayMs;

    static {
        reconnectDelayMs = Long.getLong(OBSERVER_RECONNECT_DELAY_MS, 0);
        LOG.info("{} = {}", OBSERVER_RECONNECT_DELAY_MS, reconnectDelayMs);
        observerElectionDelayMs = Long.getLong(OBSERVER_ELECTION_DELAY_MS, 200);
        LOG.info("{} = {}", OBSERVER_ELECTION_DELAY_MS , observerElectionDelayMs);
    }

    /**
     * next learner master to try, when specified
     */
    private static final AtomicReference<QuorumPeer.QuorumServer> nextLearnerMaster = new AtomicReference<>();

    private QuorumPeer.QuorumServer currentLearnerMaster = null;

    Observer(QuorumPeer self, ObserverZooKeeperServer observerZooKeeperServer) {
        this.self = self;
        this.zk = observerZooKeeperServer;
    }

    @Override
    public String toString() {
        StringBuilder sb = new StringBuilder();
        sb.append("Observer ").append(sock);
        sb.append(" pendingRevalidationCount:").append(pendingRevalidations.size());
        return sb.toString();
    }

    /**
     * the main method called by the observer to observe the leader
     * @throws Exception
     */
    void observeLeader() throws Exception {
        zk.registerJMX(new ObserverBean(this, zk), self.jmxLocalPeerBean);
        long connectTime = 0;
        boolean completedSync = false;
        try {
            self.setZabState(QuorumPeer.ZabState.DISCOVERY);
            QuorumServer master = findLearnerMaster();
            try {
                connectToLeader(master.addr, master.hostname);
                connectTime = System.currentTimeMillis();
                long newLeaderZxid = registerWithLeader(Leader.OBSERVERINFO);
                if (self.isReconfigStateChange()) {
                    throw new Exception("learned about role change");
                }

                self.setLeaderAddressAndId(master.addr, master.getId());
                self.setZabState(QuorumPeer.ZabState.SYNCHRONIZATION);
                syncWithLeader(newLeaderZxid);
                self.setZabState(QuorumPeer.ZabState.BROADCAST);
                completedSync = true;
                QuorumPacket qp = new QuorumPacket();
                while (this.isRunning() && nextLearnerMaster.get() == null) {
                    readPacket(qp);
                    processPacket(qp);
                }
            } catch (Exception e) {
                LOG.warn("Exception when observing the leader", e);
                closeSocket();

                // clear pending revalidations
                pendingRevalidations.clear();
            }
        } finally {
            currentLearnerMaster = null;
            zk.unregisterJMX(this);
            if (connectTime != 0) {
                long connectionDuration = System.currentTimeMillis() - connectTime;

                LOG.info(
                    "Disconnected from leader (with address: {}). Was connected for {}ms. Sync state: {}",
                    leaderAddr,
                    connectionDuration,
                    completedSync);
                messageTracker.dumpToLog(leaderAddr.toString());
            }
        }
    }

    private QuorumServer findLearnerMaster() {
        QuorumPeer.QuorumServer prescribedLearnerMaster = nextLearnerMaster.getAndSet(null);
        if (prescribedLearnerMaster != null
            && self.validateLearnerMaster(Long.toString(prescribedLearnerMaster.id)) == null) {
            LOG.warn("requested next learner master {} is no longer valid", prescribedLearnerMaster);
            prescribedLearnerMaster = null;
        }
        final QuorumPeer.QuorumServer master = (prescribedLearnerMaster == null)
            ? self.findLearnerMaster(findLeader())
            : prescribedLearnerMaster;
        currentLearnerMaster = master;
        if (master == null) {
            LOG.warn("No learner master found");
        } else {
            LOG.info("Observing new leader sid={} addr={}", master.id, master.addr);
        }
        return master;
    }

    /**
     * Controls the response of an observer to the receipt of a quorumpacket
     * @param qp
     * @throws Exception
     */
    protected void processPacket(QuorumPacket qp) throws Exception {
        TxnLogEntry logEntry;
        TxnHeader hdr;
        TxnDigest digest;
        Record txn;
        switch (qp.getType()) {
        case Leader.PING:
            ping(qp);
            break;
        case Leader.PROPOSAL:
            LOG.warn("Ignoring proposal");
            break;
        case Leader.COMMIT:
            LOG.warn("Ignoring commit");
            break;
        case Leader.UPTODATE:
            LOG.error("Received an UPTODATE message after Observer started");
            break;
        case Leader.REVALIDATE:
            revalidate(qp);
            break;
        case Leader.SYNC:
            ((ObserverZooKeeperServer) zk).sync();
            break;
        case Leader.INFORM:
            ServerMetrics.getMetrics().LEARNER_COMMIT_RECEIVED_COUNT.add(1);
            logEntry = SerializeUtils.deserializeTxn(qp.getData());
            hdr = logEntry.getHeader();
            txn = logEntry.getTxn();
            digest = logEntry.getDigest();
            Request request = new Request(hdr.getClientId(), hdr.getCxid(), hdr.getType(), hdr, txn, 0);
            request.logLatency(ServerMetrics.getMetrics().COMMIT_PROPAGATION_LATENCY);
            request.setTxnDigest(digest);
            ObserverZooKeeperServer obs = (ObserverZooKeeperServer) zk;
            obs.commitRequest(request);
            break;
        case Leader.INFORMANDACTIVATE:
            // get new designated leader from (current) leader's message
            ByteBuffer buffer = ByteBuffer.wrap(qp.getData());
            long suggestedLeaderId = buffer.getLong();

            byte[] remainingdata = new byte[buffer.remaining()];
            buffer.get(remainingdata);
            logEntry = SerializeUtils.deserializeTxn(remainingdata);
            hdr = logEntry.getHeader();
            txn = logEntry.getTxn();
            digest = logEntry.getDigest();
            QuorumVerifier qv = self.configFromString(new String(((SetDataTxn) txn).getData(), UTF_8));

            request = new Request(hdr.getClientId(), hdr.getCxid(), hdr.getType(), hdr, txn, 0);
            request.setTxnDigest(digest);
            obs = (ObserverZooKeeperServer) zk;

            boolean majorChange = self.processReconfig(qv, suggestedLeaderId, qp.getZxid(), true);

            obs.commitRequest(request);

            if (majorChange) {
                throw new Exception("changes proposed in reconfig");
            }
            break;
        default:
            LOG.warn("Unknown packet type: {}", LearnerHandler.packetToString(qp));
            break;
        }
    }

    /**
     * Shutdown the Observer.
     */
    public void shutdown() {
        LOG.info("shutdown Observer");
        super.shutdown();
    }

    static void waitForReconnectDelay() {
        waitForReconnectDelayHelper(reconnectDelayMs);
    }

    static void waitForObserverElectionDelay() {
        waitForReconnectDelayHelper(observerElectionDelayMs);
    }

    private static void waitForReconnectDelayHelper(long delayValueMs) {
        if (delayValueMs > 0) {
            long randomDelay = ThreadLocalRandom.current().nextLong(delayValueMs);
            LOG.info("Waiting for {} ms before reconnecting with the leader", randomDelay);
            try {
                Thread.sleep(randomDelay);
            } catch (InterruptedException e) {
                LOG.warn("Interrupted while waiting", e);
            }
        }
    }

    public long getLearnerMasterId() {
        QuorumPeer.QuorumServer current = currentLearnerMaster;
        return current == null ? -1 : current.id;
    }

    /**
     * Prompts the Observer to disconnect from its current learner master and reconnect
     * to the specified server. If that connection attempt fails, the Observer will
     * fail over to the next available learner master.
     */
    public boolean setLearnerMaster(String learnerMaster) {
        final QuorumPeer.QuorumServer server = self.validateLearnerMaster(learnerMaster);
        if (server == null) {
            return false;
        } else if (server.equals(currentLearnerMaster)) {
            LOG.info("Already connected to requested learner master sid={} addr={}", server.id, server.addr);
            return true;
        } else {
            LOG.info("Requesting disconnect and reconnect to new learner master sid={} addr={}", server.id, server.addr);
            nextLearnerMaster.set(server);
            return true;
        }
    }

    public QuorumPeer.QuorumServer getCurrentLearnerMaster() {
        return currentLearnerMaster;
    }

    public static long getObserverElectionDelayMs() {
        return observerElectionDelayMs;
    }

    public static void setObserverElectionDelayMs(long electionDelayMs) {
        observerElectionDelayMs = electionDelayMs;
        LOG.info("{} = {}", OBSERVER_ELECTION_DELAY_MS, observerElectionDelayMs);
    }

}