Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

ZOOKEEPER-4712: Fix partially shutdown of ZooKeeperServer and its processors #2154

Merged
merged 9 commits into from
Sep 20, 2024
Original file line number Diff line number Diff line change
Expand Up @@ -924,7 +924,7 @@ public boolean isRunning() {
return state == State.RUNNING;
}

public void shutdown() {
public final void shutdown() {
shutdown(false);
}

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -155,11 +155,11 @@ protected void unregisterMetrics() {
}

@Override
public synchronized void shutdown() {
public synchronized void shutdown(boolean fullyShutDown) {
if (containerManager != null) {
containerManager.stop();
}
super.shutdown();
super.shutdown(fullyShutDown);
}

@Override
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -900,26 +900,26 @@ boolean isRunning() {
}

void closeSocket() {
if (sock != null) {
if (sockBeingClosed.compareAndSet(false, true)) {
if (closeSocketAsync) {
final Thread closingThread = new Thread(() -> closeSockSync(), "CloseSocketThread(sid:" + zk.getServerId());
closingThread.setDaemon(true);
closingThread.start();
} else {
closeSockSync();
}
if (sockBeingClosed.compareAndSet(false, true)) {
if (sock == null) { // Closing before establishing the connection is a noop
return;
}
Comment on lines +904 to +906
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

nit: Why have you moved the null check inside the lock?

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Close could be called from different threads, and sockBeingClosed ensures memory visibility for sock, as it's set after sock is assigned in connectToLeader.
The only other thread I can see that closes the learner, right now, is the sync processor, which is initialised after sock is assigned, so it works as-was, but I still prefer to be explicit about this.

Socket socket = sock;
sock = null;
if (closeSocketAsync) {
final Thread closingThread = new Thread(() -> closeSockSync(socket), "CloseSocketThread(sid:" + zk.getServerId());
closingThread.setDaemon(true);
closingThread.start();
} else {
closeSockSync(socket);
}
}
}

void closeSockSync() {
private static void closeSockSync(Socket socket) {
try {
long startTime = Time.currentElapsedTime();
if (sock != null) {
sock.close();
sock = null;
}
socket.close();
ServerMetrics.getMetrics().SOCKET_CLOSING_TIME.add(Time.currentElapsedTime() - startTime);
} catch (IOException e) {
LOG.warn("Ignoring error closing connection to leader", e);
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -152,24 +152,25 @@ protected void unregisterJMX(Learner peer) {
}

@Override
public synchronized void shutdown() {
public synchronized void shutdown(boolean fullyShutDown) {
if (!canShutdown()) {
super.shutdown(fullyShutDown);
LOG.debug("ZooKeeper server is not running, so not proceeding to shutdown!");
return;
}
kezhuw marked this conversation as resolved.
Show resolved Hide resolved
LOG.info("Shutting down");
try {
super.shutdown();
} catch (Exception e) {
LOG.warn("Ignoring unexpected exception during shutdown", e);
}
try {
if (syncProcessor != null) {
syncProcessor.shutdown();
}
} catch (Exception e) {
LOG.warn("Ignoring unexpected exception in syncprocessor shutdown", e);
}
try {
super.shutdown(fullyShutDown);
} catch (Exception e) {
LOG.warn("Ignoring unexpected exception during shutdown", e);
}
}

}
Original file line number Diff line number Diff line change
Expand Up @@ -44,7 +44,7 @@ public class ObserverZooKeeperServer extends LearnerZooKeeperServer {
* take periodic snapshot. Default is ON.
*/

private boolean syncRequestProcessorEnabled = this.self.getSyncEnabled();
private final boolean syncRequestProcessorEnabled = this.self.getSyncEnabled();

/*
* Pending sync requests
Expand Down Expand Up @@ -127,18 +127,6 @@ public String getState() {
return "observer";
}

@Override
public synchronized void shutdown() {
if (!canShutdown()) {
LOG.debug("ZooKeeper server is not running, so not proceeding to shutdown!");
return;
}
super.shutdown();
if (syncRequestProcessorEnabled && syncProcessor != null) {
syncProcessor.shutdown();
}
}

@Override
public void dumpMonitorValues(BiConsumer<String, Object> response) {
super.dumpMonitorValues(response);
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -190,8 +190,9 @@ public long getServerId() {
}

@Override
public synchronized void shutdown() {
public synchronized void shutdown(boolean fullyShutDown) {
if (!canShutdown()) {
super.shutdown(fullyShutDown);
LOG.debug("ZooKeeper server is not running, so not proceeding to shutdown!");
return;
}
Expand All @@ -206,7 +207,7 @@ public synchronized void shutdown() {
self.adminServer.setZooKeeperServer(null);

// shutdown the server itself
super.shutdown();
super.shutdown(fullyShutDown);
}

@Override
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -20,6 +20,8 @@

import java.io.Flushable;
import java.io.IOException;
import java.net.Socket;

jonmv marked this conversation as resolved.
Show resolved Hide resolved
import org.apache.zookeeper.ZooDefs.OpCode;
import org.apache.zookeeper.server.Request;
import org.apache.zookeeper.server.RequestProcessor;
Expand All @@ -46,7 +48,7 @@ public void processRequest(Request si) {
learner.writePacket(qp, false);
} catch (IOException e) {
LOG.warn("Closing connection to leader, exception during packet send", e);
learner.closeSockSync();
learner.closeSocket();
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

+1 on this.

This is the leftover of ZOOKEEPER-4409. We should respect zookeeper.learner.closeSocketAsync here.

}
}
}
Expand All @@ -56,7 +58,7 @@ public void flush() throws IOException {
learner.writePacket(null, true);
} catch (IOException e) {
LOG.warn("Closing connection to leader, exception during packet send", e);
learner.closeSockSync();
learner.closeSocket();
}
}

Expand Down