summaryrefslogtreecommitdiff
path: root/zookeeper-server/src/main/java/org/apache/zookeeper/server/embedded/ZooKeeperServerEmbeddedImpl.java
blob: ad41a3228e7b2b891fa0705deeb4fa237f60b6e9 (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
package org.apache.zookeeper.server.embedded;

import java.io.OutputStream;
import java.net.InetSocketAddress;
import java.nio.file.Files;
import java.nio.file.Path;
import java.util.Map;
import java.util.Properties;
import java.util.concurrent.CompletableFuture;
import java.util.concurrent.TimeUnit;
import java.util.concurrent.TimeoutException;
import javax.security.sasl.SaslException;
import org.apache.zookeeper.server.DatadirCleanupManager;
import org.apache.zookeeper.server.ExitCode;
import org.apache.zookeeper.server.ServerConfig;
import org.apache.zookeeper.server.ZooKeeperServerMain;
import org.apache.zookeeper.server.quorum.QuorumPeer;
import org.apache.zookeeper.server.quorum.QuorumPeerConfig;
import org.apache.zookeeper.server.quorum.QuorumPeerMain;
import org.apache.zookeeper.util.ServiceUtils;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;

/**
 * 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.
 */
/**
 * Implementation of ZooKeeperServerEmbedded.
 */
class ZooKeeperServerEmbeddedImpl implements ZooKeeperServerEmbedded {

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

    private final QuorumPeerConfig config;
    private QuorumPeerMain maincluster;
    private ZooKeeperServerMain mainsingle;
    private Thread thread;
    private DatadirCleanupManager purgeMgr;
    private final ExitHandler exitHandler;
    private volatile boolean stopping;

    private int boundClientPort;
    private int boundSecureClientPort;

    ZooKeeperServerEmbeddedImpl(Properties p, Path baseDir, ExitHandler exitHandler) throws Exception {
        if (!p.containsKey("dataDir")) {
            p.put("dataDir", baseDir.resolve("data").toAbsolutePath().toString());
        }
        Path configFile = Files.createTempFile(baseDir, "zookeeper.configuration", ".properties");
        try (OutputStream oo = Files.newOutputStream(configFile)) {
            p.store(oo, "Automatically generated at every-boot");
        }
        this.exitHandler = exitHandler;
        LOG.info("Current configuration is at {}", configFile.toAbsolutePath());
        config = new QuorumPeerConfig();
        config.parse(configFile.toAbsolutePath().toString());
        LOG.info("ServerID:" + config.getServerId());
        LOG.info("DataDir:" + config.getDataDir());
        LOG.info("Servers:" + config.getServers());
        LOG.info("ElectionPort:" + config.getElectionPort());
        LOG.info("SyncLimit:" + config.getSyncLimit());
        LOG.info("PeerType:" + config.getPeerType());
        LOG.info("Distributed:" + config.isDistributed());
        LOG.info("SyncEnabled:" + config.getSyncEnabled());
        LOG.info("MetricsProviderClassName:" + config.getMetricsProviderClassName());

        for (Map.Entry<Long, QuorumPeer.QuorumServer> server : config.getServers().entrySet()) {
            LOG.info("Server: " + server.getKey() + " -> addr " + server.getValue().addr + " elect "
                    + server.getValue().electionAddr + " id=" + server.getValue().id + " type "
                    + server.getValue().type);
        }
    }

    @Override
    public void start() throws Exception {
        start(Integer.MAX_VALUE);
    }

    @Override
    public void start(long startupTimeout) throws Exception {
        switch (exitHandler) {
            case EXIT:
                ServiceUtils.setSystemExitProcedure(ServiceUtils.SYSTEM_EXIT);
                break;
            case LOG_ONLY:
                ServiceUtils.setSystemExitProcedure(ServiceUtils.LOG_ONLY);
                break;
            default:
                ServiceUtils.setSystemExitProcedure(ServiceUtils.SYSTEM_EXIT);
                break;
        }
        final CompletableFuture<String> started = new CompletableFuture<>();

        if (config.getServers().size() > 1 || config.isDistributed()) {
            LOG.info("Running ZK Server in single Quorum MODE");

            maincluster = new QuorumPeerMain() {
                protected QuorumPeer getQuorumPeer() throws SaslException {
                    return new QuorumPeer() {
                        @Override
                        public void start() {
                            super.start();
                            boundClientPort = getClientPort();
                            boundSecureClientPort = getSecureClientPort();
                            LOG.info("ZK Server {} started", this);
                            started.complete(null);
                        }
                    };
                }
            };

            // Start and schedule the the purge task
            purgeMgr = new DatadirCleanupManager(config
                    .getDataDir(), config.getDataLogDir(), config
                    .getSnapRetainCount(), config.getPurgeInterval());
            purgeMgr.start();

            thread = new Thread("zkservermainrunner") {
                @Override
                public void run() {
                    try {
                        maincluster.runFromConfig(config);
                        maincluster.close();
                        LOG.info("ZK server died. Requsting stop on JVM");
                        if (!stopping) {
                            ServiceUtils.requestSystemExit(ExitCode.EXECUTION_FINISHED.getValue());
                        }
                    } catch (Throwable t) {
                        LOG.error("error during server lifecycle", t);
                        maincluster.close();
                        if (!stopping) {
                            ServiceUtils.requestSystemExit(ExitCode.INVALID_INVOCATION.getValue());
                        }
                    }
                }
            };
            thread.start();
        } else {
            LOG.info("Running ZK Server in single STANDALONE MODE");
            mainsingle = new ZooKeeperServerMain() {
                @Override
                public void serverStarted() {
                    LOG.info("ZK Server started");
                    boundClientPort = getClientPort();
                    boundSecureClientPort = getSecureClientPort();
                    started.complete(null);
                }
            };
            purgeMgr = new DatadirCleanupManager(config
                    .getDataDir(), config.getDataLogDir(), config
                    .getSnapRetainCount(), config.getPurgeInterval());
            purgeMgr.start();
            thread = new Thread("zkservermainrunner") {
                @Override
                public void run() {
                    try {
                        ServerConfig cc = new ServerConfig();
                        cc.readFrom(config);
                        LOG.info("ZK server starting");
                        mainsingle.runFromConfig(cc);
                        LOG.info("ZK server died. Requesting stop on JVM");
                        if (!stopping) {
                            ServiceUtils.requestSystemExit(ExitCode.EXECUTION_FINISHED.getValue());
                        }
                    } catch (Throwable t) {
                        LOG.error("error during server lifecycle", t);
                        mainsingle.close();
                        if (!stopping) {
                            ServiceUtils.requestSystemExit(ExitCode.INVALID_INVOCATION.getValue());
                        }
                    }
                }
            };
            thread.start();
        }

        try {
            started.get(startupTimeout, TimeUnit.MILLISECONDS);
        } catch (TimeoutException err) {
            LOG.info("Startup timed out, trying to close");
            close();
            throw err;
        }
    }

    @Override
    public String getConnectionString() {
        return prettifyConnectionString(config.getClientPortAddress(), boundClientPort);
    }

    @Override
    public String getSecureConnectionString() {
        return prettifyConnectionString(config.getSecureClientPortAddress(), boundSecureClientPort);
    }

    private String prettifyConnectionString(InetSocketAddress confAddress, int boundPort) {
        if (confAddress != null) {
            return confAddress.getHostString()
                .replace("0.0.0.0", "localhost")
                .replace("0:0:0:0:0:0:0:0", "localhost")
                + ":" + boundPort;
        }
        throw new IllegalStateException("No client address is configured");
    }

    @Override
    public void close() {
        LOG.info("Stopping ZK Server");
        stopping = true;
        if (mainsingle != null) {
            mainsingle.close();
        }
        if (maincluster != null) {
            maincluster.close();
        }
    }
}