summaryrefslogtreecommitdiff
path: root/zookeeper-server/src/main/java/org/apache/zookeeper/server/watch/WatcherCleaner.java
blob: c3b7e602f1ccf908192682d713f36763123d9c3b (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
/*
 * 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.watch;

import java.util.HashSet;
import java.util.Set;
import java.util.concurrent.ThreadLocalRandom;
import java.util.concurrent.atomic.AtomicInteger;
import org.apache.zookeeper.common.Time;
import org.apache.zookeeper.server.RateLogger;
import org.apache.zookeeper.server.ServerMetrics;
import org.apache.zookeeper.server.WorkerService;
import org.apache.zookeeper.server.WorkerService.WorkRequest;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;

/**
 * Thread used to lazily clean up the closed watcher, it will trigger the
 * clean up when the dead watchers get certain number or some number of
 * seconds has elapsed since last clean up.
 *
 * Cost of running it:
 *
 * - need to go through all the paths even if the watcher may only
 *   watching a single path
 * - block in the path BitHashSet when we try to check the dead watcher
 *   which won't block other stuff
 */
public class WatcherCleaner extends Thread {

    private static final Logger LOG = LoggerFactory.getLogger(WatcherCleaner.class);
    private final RateLogger RATE_LOGGER = new RateLogger(LOG);

    private volatile boolean stopped = false;
    private final Object cleanEvent = new Object();
    private final Object processingCompletedEvent = new Object();
    private final WorkerService cleaners;

    private final Set<Integer> deadWatchers;
    private final IDeadWatcherListener listener;
    private final int watcherCleanThreshold;
    private final int watcherCleanIntervalInSeconds;
    private final int maxInProcessingDeadWatchers;
    private final AtomicInteger totalDeadWatchers = new AtomicInteger();

    public WatcherCleaner(IDeadWatcherListener listener) {
        this(
            listener,
            Integer.getInteger("zookeeper.watcherCleanThreshold", 1000),
            Integer.getInteger("zookeeper.watcherCleanIntervalInSeconds", 600),
            Integer.getInteger("zookeeper.watcherCleanThreadsNum", 2),
            Integer.getInteger("zookeeper.maxInProcessingDeadWatchers", -1));
    }

    public WatcherCleaner(IDeadWatcherListener listener, int watcherCleanThreshold, int watcherCleanIntervalInSeconds, int watcherCleanThreadsNum, int maxInProcessingDeadWatchers) {
        this.listener = listener;
        this.watcherCleanThreshold = watcherCleanThreshold;
        this.watcherCleanIntervalInSeconds = watcherCleanIntervalInSeconds;
        int suggestedMaxInProcessingThreshold = watcherCleanThreshold * watcherCleanThreadsNum;
        if (maxInProcessingDeadWatchers > 0 && maxInProcessingDeadWatchers < suggestedMaxInProcessingThreshold) {
            maxInProcessingDeadWatchers = suggestedMaxInProcessingThreshold;
            LOG.info(
                "The maxInProcessingDeadWatchers config is smaller than the suggested one, change it to use {}",
                maxInProcessingDeadWatchers);
        }
        this.maxInProcessingDeadWatchers = maxInProcessingDeadWatchers;
        this.deadWatchers = new HashSet<>();
        this.cleaners = new WorkerService("DeadWatcherCleanner", watcherCleanThreadsNum, false);

        LOG.info(
            "watcherCleanThreshold={}, watcherCleanIntervalInSeconds={}"
                + ", watcherCleanThreadsNum={}, maxInProcessingDeadWatchers={}",
            watcherCleanThreshold,
            watcherCleanIntervalInSeconds,
            watcherCleanThreadsNum,
            maxInProcessingDeadWatchers);
    }

    public void addDeadWatcher(int watcherBit) {
        // Wait if there are too many watchers waiting to be closed,
        // this is will slow down the socket packet processing and
        // the adding watches in the ZK pipeline.
        while (maxInProcessingDeadWatchers > 0 && !stopped && totalDeadWatchers.get() >= maxInProcessingDeadWatchers) {
            try {
                RATE_LOGGER.rateLimitLog("Waiting for dead watchers cleaning");
                long startTime = Time.currentElapsedTime();
                synchronized (processingCompletedEvent) {
                    processingCompletedEvent.wait(100);
                }
                long latency = Time.currentElapsedTime() - startTime;
                ServerMetrics.getMetrics().ADD_DEAD_WATCHER_STALL_TIME.add(latency);
            } catch (InterruptedException e) {
                LOG.info("Got interrupted while waiting for dead watches queue size");
                break;
            }
        }
        synchronized (this) {
            if (deadWatchers.add(watcherBit)) {
                totalDeadWatchers.incrementAndGet();
                ServerMetrics.getMetrics().DEAD_WATCHERS_QUEUED.add(1);
                if (deadWatchers.size() >= watcherCleanThreshold) {
                    synchronized (cleanEvent) {
                        cleanEvent.notifyAll();
                    }
                }
            }
        }
    }

    @Override
    public void run() {
        while (!stopped) {
            synchronized (cleanEvent) {
                try {
                    // add some jitter to avoid cleaning dead watchers at the
                    // same time in the quorum
                    if (!stopped && deadWatchers.size() < watcherCleanThreshold) {
                        int maxWaitMs = (watcherCleanIntervalInSeconds
                                         + ThreadLocalRandom.current().nextInt(watcherCleanIntervalInSeconds / 2 + 1)) * 1000;
                        cleanEvent.wait(maxWaitMs);
                    }
                } catch (InterruptedException e) {
                    LOG.info("Received InterruptedException while waiting for cleanEvent");
                    break;
                }
            }

            if (deadWatchers.isEmpty()) {
                continue;
            }

            synchronized (this) {
                // Clean the dead watchers need to go through all the current
                // watches, which is pretty heavy and may take a second if
                // there are millions of watches, that's why we're doing lazily
                // batch clean up in a separate thread with a snapshot of the
                // current dead watchers.
                final Set<Integer> snapshot = new HashSet<>(deadWatchers);
                deadWatchers.clear();
                int total = snapshot.size();
                LOG.info("Processing {} dead watchers", total);
                cleaners.schedule(new WorkRequest() {
                    @Override
                    public void doWork() throws Exception {
                        long startTime = Time.currentElapsedTime();
                        listener.processDeadWatchers(snapshot);
                        long latency = Time.currentElapsedTime() - startTime;
                        LOG.info("Takes {} to process {} watches", latency, total);
                        ServerMetrics.getMetrics().DEAD_WATCHERS_CLEANER_LATENCY.add(latency);
                        ServerMetrics.getMetrics().DEAD_WATCHERS_CLEARED.add(total);
                        totalDeadWatchers.addAndGet(-total);
                        synchronized (processingCompletedEvent) {
                            processingCompletedEvent.notifyAll();
                        }
                    }
                });
            }
        }
        LOG.info("WatcherCleaner thread exited");
    }

    public void shutdown() {
        stopped = true;
        deadWatchers.clear();
        cleaners.stop();
        this.interrupt();
        if (LOG.isInfoEnabled()) {
            LOG.info("WatcherCleaner thread shutdown is initiated");
        }
    }

}