Commit
This commit does not belong to any branch on this repository, and may belong to a fork outside of the repository.
[ALLUXIO-3181] Close client sockets during stepdown (#7230)
* Close client sockets during stepdown * Add license * Fix findbugs * Switch to scheduled executor
- Loading branch information
Showing
4 changed files
with
145 additions
and
13 deletions.
There are no files selected for viewing
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
115 changes: 115 additions & 0 deletions
115
core/server/master/src/main/java/alluxio/master/thrift/SocketTrackingTServerSocket.java
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Original file line | Diff line number | Diff line change |
---|---|---|---|
@@ -0,0 +1,115 @@ | |||
/* | |||
* The Alluxio Open Foundation licenses this work under the Apache License, version 2.0 | |||
* (the "License"). You may not use this work except in compliance with the License, which is | |||
* available at www.apache.org/licenses/LICENSE-2.0 | |||
* | |||
* This software is distributed on an "AS IS" basis, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, | |||
* either express or implied, as more fully set forth in the License. | |||
* | |||
* See the NOTICE file distributed with this work for information regarding copyright ownership. | |||
*/ | |||
|
|||
package alluxio.master.thrift; | |||
|
|||
import alluxio.Configuration; | |||
import alluxio.PropertyKey; | |||
import alluxio.util.ThreadFactoryUtils; | |||
|
|||
import com.google.common.io.Closer; | |||
import org.apache.thrift.transport.TServerSocket; | |||
import org.apache.thrift.transport.TSocket; | |||
import org.apache.thrift.transport.TTransportException; | |||
import org.slf4j.Logger; | |||
import org.slf4j.LoggerFactory; | |||
|
|||
import java.io.IOException; | |||
import java.net.InetSocketAddress; | |||
import java.net.Socket; | |||
import java.util.Iterator; | |||
import java.util.Set; | |||
import java.util.concurrent.ConcurrentHashMap; | |||
import java.util.concurrent.Executors; | |||
import java.util.concurrent.ScheduledExecutorService; | |||
import java.util.concurrent.TimeUnit; | |||
|
|||
/** | |||
* Extension of TServerSocket which tracks all accepted sockets and closes them when the server | |||
* socket is closed. | |||
*/ | |||
public class SocketTrackingTServerSocket extends TServerSocket { | |||
private static final Logger LOG = LoggerFactory.getLogger(SocketTrackingTServerSocket.class); | |||
private static final long CLEANUP_INTERVAL_MS = | |||
Configuration.getMs(PropertyKey.MASTER_CLIENT_SOCKET_CLEANUP_INTERVAL); | |||
|
|||
private final Set<Socket> mSockets = ConcurrentHashMap.newKeySet(); | |||
private final ScheduledExecutorService mExecutor; | |||
|
|||
/** | |||
* @param bindAddr bind address for the socket | |||
* @param clientTimeout timeout for client sockets from accept | |||
*/ | |||
public SocketTrackingTServerSocket(InetSocketAddress bindAddr, int clientTimeout) | |||
throws TTransportException { | |||
super(bindAddr, clientTimeout); | |||
mExecutor = Executors | |||
.newSingleThreadScheduledExecutor(ThreadFactoryUtils.build("socket-closer-thread", true)); | |||
mExecutor.scheduleAtFixedRate(this::removeClosedSockets, CLEANUP_INTERVAL_MS, | |||
CLEANUP_INTERVAL_MS, TimeUnit.MILLISECONDS); | |||
} | |||
|
|||
@Override | |||
public TSocket acceptImpl() throws TTransportException { | |||
TSocket socket = super.acceptImpl(); | |||
mSockets.add(socket.getSocket()); | |||
return socket; | |||
} | |||
|
|||
@Override | |||
public void close() { | |||
super.close(); | |||
try { | |||
closeClientSockets(); | |||
} catch (IOException e) { | |||
LOG.error("Could not close client sockets", e); | |||
} | |||
mExecutor.shutdownNow(); | |||
try { | |||
if (!mExecutor.awaitTermination(1, TimeUnit.SECONDS)) { | |||
LOG.warn("Failed to stop socket cleanup thread."); | |||
} | |||
} catch (InterruptedException e) { | |||
Thread.currentThread().interrupt(); | |||
return; | |||
} | |||
} | |||
|
|||
/** | |||
* Closes all socket connections that have been accepted by this server socket. | |||
*/ | |||
private void closeClientSockets() throws IOException { | |||
Closer closer = Closer.create(); | |||
int count = 0; | |||
for (Socket s : mSockets) { | |||
if (!s.isClosed()) { | |||
closer.register(s); | |||
count++; | |||
} | |||
} | |||
closer.close(); | |||
LOG.info("Closed {} client sockets", count); | |||
} | |||
|
|||
/** | |||
* Periodically clean up any closed sockets. | |||
*/ | |||
private void removeClosedSockets() { | |||
// This is best-effort, and may not remove sockets added to the mSockets set after the | |||
// iterator was created. Those sockets will be checked on the next sweep. | |||
for (Iterator<Socket> it = mSockets.iterator(); it.hasNext();) { | |||
Socket s = it.next(); | |||
if (s.isClosed()) { | |||
it.remove(); | |||
} | |||
} | |||
} | |||
} |