-
Notifications
You must be signed in to change notification settings - Fork 505
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
HDDS-11627. Support getBlock operation on short-circuit channel. #7456
Draft
ChenSammi
wants to merge
5
commits into
apache:HDDS-10685
Choose a base branch
from
ChenSammi:HDDS-11627
base: HDDS-10685
Could not load branches
Branch not found: {{ refName }}
Loading
Could not load tags
Nothing to show
Loading
Are you sure you want to change the base?
Some commits from the old base branch may be removed from the timeline,
and old review comments may become outdated.
Draft
Changes from all commits
Commits
Show all changes
5 commits
Select commit
Hold shift + click to select a range
e04c83b
HDDS-11627. Support getBlock operation on short-circuit channel.
ChenSammi 0e49636
address comments
ChenSammi 42a9f2c
remove SHORT_CIRCUIT replication type
ChenSammi bd94222
use hadoop 3.3.7 snapshot
ChenSammi 43f9d5c
checkstyle and findbug issues
ChenSammi File filter
Filter by extension
Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
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
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 | Diff line number | Diff line change |
---|---|---|
|
@@ -19,6 +19,8 @@ | |
package org.apache.hadoop.hdds.scm; | ||
|
||
import java.io.IOException; | ||
import java.net.InetSocketAddress; | ||
import java.util.concurrent.ConcurrentHashMap; | ||
import java.util.concurrent.TimeUnit; | ||
|
||
import org.apache.hadoop.hdds.conf.Config; | ||
|
@@ -29,6 +31,9 @@ | |
import org.apache.hadoop.hdds.protocol.proto.HddsProtos; | ||
import org.apache.hadoop.hdds.scm.client.ClientTrustManager; | ||
import org.apache.hadoop.hdds.scm.pipeline.Pipeline; | ||
import org.apache.hadoop.hdds.scm.storage.DomainSocketFactory; | ||
import org.apache.hadoop.net.NetUtils; | ||
import org.apache.hadoop.ozone.util.OzoneNetUtils; | ||
import org.apache.hadoop.security.UserGroupInformation; | ||
|
||
import com.google.common.annotations.VisibleForTesting; | ||
|
@@ -38,6 +43,7 @@ | |
import com.google.common.cache.RemovalListener; | ||
import com.google.common.cache.RemovalNotification; | ||
import static java.util.concurrent.TimeUnit.MILLISECONDS; | ||
import static org.apache.hadoop.hdds.DatanodeVersion.SHORT_CIRCUIT_READS; | ||
import static org.apache.hadoop.hdds.conf.ConfigTag.OZONE; | ||
import static org.apache.hadoop.hdds.conf.ConfigTag.PERFORMANCE; | ||
import static org.apache.hadoop.hdds.scm.exceptions.SCMException.ResultCodes.NO_REPLICA_FOUND; | ||
|
@@ -64,8 +70,8 @@ public class XceiverClientManager extends XceiverClientCreator { | |
|
||
private final Cache<String, XceiverClientSpi> clientCache; | ||
private final CacheMetrics cacheMetrics; | ||
|
||
private static XceiverClientMetrics metrics; | ||
private final ConcurrentHashMap<String, DatanodeDetails> localDNCache; | ||
|
||
/** | ||
* Creates a new XceiverClientManager for non secured ozone cluster. | ||
|
@@ -105,6 +111,7 @@ public void onRemoval( | |
}).build(); | ||
|
||
cacheMetrics = CacheMetrics.create(clientCache, this); | ||
this.localDNCache = new ConcurrentHashMap<>(); | ||
} | ||
|
||
@VisibleForTesting | ||
|
@@ -117,17 +124,54 @@ public Cache<String, XceiverClientSpi> getClientCache() { | |
* | ||
* If there is already a cached XceiverClientSpi, simply return | ||
* the cached otherwise create a new one. | ||
* | ||
* @param pipeline the container pipeline for the client connection | ||
* @return XceiverClientSpi connected to a container | ||
* @throws IOException if a XceiverClientSpi cannot be acquired | ||
*/ | ||
@Override | ||
public XceiverClientSpi acquireClient(Pipeline pipeline) | ||
throws IOException { | ||
return acquireClient(pipeline, false, false); | ||
} | ||
|
||
/** | ||
* Acquires a XceiverClientSpi connected to a container for read. | ||
* | ||
* If there is already a cached XceiverClientSpi, simply return | ||
* the cached otherwise create a new one. | ||
* | ||
* @param pipeline the container pipeline for the client connection | ||
* @return XceiverClientSpi connected to a container | ||
* @throws IOException if a XceiverClientSpi cannot be acquired | ||
*/ | ||
@Override | ||
public XceiverClientSpi acquireClientForReadData(Pipeline pipeline, boolean allowShortCircuit) | ||
throws IOException { | ||
return acquireClient(pipeline, false, allowShortCircuit); | ||
} | ||
|
||
/** | ||
* Acquires a XceiverClientSpi connected to a container capable of | ||
* storing the specified key. | ||
* | ||
* If there is already a cached XceiverClientSpi, simply return | ||
* the cached otherwise create a new one. | ||
* | ||
* @param pipeline the container pipeline for the client connection | ||
* @return XceiverClientSpi connected to a container | ||
* @throws IOException if a XceiverClientSpi cannot be acquired | ||
*/ | ||
@Override | ||
public XceiverClientSpi acquireClient(Pipeline pipeline, | ||
boolean topologyAware) throws IOException { | ||
boolean topologyAware, boolean allowShortCircuit) throws IOException { | ||
Preconditions.checkNotNull(pipeline); | ||
Preconditions.checkArgument(pipeline.getNodes() != null); | ||
Preconditions.checkArgument(!pipeline.getNodes().isEmpty(), | ||
NO_REPLICA_FOUND); | ||
|
||
synchronized (clientCache) { | ||
XceiverClientSpi info = getClient(pipeline, topologyAware); | ||
XceiverClientSpi info = getClient(pipeline, topologyAware, allowShortCircuit); | ||
info.incrementReference(); | ||
return info; | ||
} | ||
|
@@ -141,7 +185,7 @@ public void releaseClient(XceiverClientSpi client, boolean invalidateClient, | |
client.decrementReference(); | ||
if (invalidateClient) { | ||
Pipeline pipeline = client.getPipeline(); | ||
String key = getPipelineCacheKey(pipeline, topologyAware); | ||
String key = getPipelineCacheKey(pipeline, topologyAware, client instanceof XceiverClientShortCircuit); | ||
XceiverClientSpi cachedClient = clientCache.getIfPresent(key); | ||
if (cachedClient == client) { | ||
clientCache.invalidate(key); | ||
|
@@ -150,24 +194,44 @@ public void releaseClient(XceiverClientSpi client, boolean invalidateClient, | |
} | ||
} | ||
|
||
protected XceiverClientSpi getClient(Pipeline pipeline, boolean topologyAware) | ||
protected XceiverClientSpi getClient(Pipeline pipeline, boolean topologyAware, boolean allowShortCircuit) | ||
throws IOException { | ||
try { | ||
// create different client different pipeline node based on | ||
// network topology | ||
String key = getPipelineCacheKey(pipeline, topologyAware); | ||
return clientCache.get(key, () -> newClient(pipeline)); | ||
// create different client different pipeline node based on network topology | ||
String key = getPipelineCacheKey(pipeline, topologyAware, allowShortCircuit); | ||
return clientCache.get(key, () -> newClient(pipeline, localDNCache.get(key))); | ||
} catch (Exception e) { | ||
throw new IOException( | ||
"Exception getting XceiverClient: " + e, e); | ||
} | ||
} | ||
|
||
private String getPipelineCacheKey(Pipeline pipeline, | ||
boolean topologyAware) { | ||
String key = pipeline.getId().getId().toString() + pipeline.getType(); | ||
private String getPipelineCacheKey(Pipeline pipeline, boolean topologyAware, boolean allowShortCircuit) { | ||
String key = pipeline.getId().getId().toString() + "-" + pipeline.getType(); | ||
boolean isEC = pipeline.getType() == HddsProtos.ReplicationType.EC; | ||
if (topologyAware || isEC) { | ||
DatanodeDetails localDN = null; | ||
|
||
if ((!isEC) && allowShortCircuit && isShortCircuitEnabled()) { | ||
int port = 0; | ||
InetSocketAddress localAddr = null; | ||
for (DatanodeDetails dn : pipeline.getNodes()) { | ||
// read port from the data node, on failure use default configured port. | ||
port = dn.getPort(DatanodeDetails.Port.Name.STANDALONE).getValue(); | ||
InetSocketAddress addr = NetUtils.createSocketAddr(dn.getIpAddress(), port); | ||
if (OzoneNetUtils.isAddressLocal(addr) && | ||
dn.getCurrentVersion() >= SHORT_CIRCUIT_READS.toProtoValue()) { | ||
localAddr = addr; | ||
localDN = dn; | ||
break; | ||
} | ||
} | ||
if (localAddr != null) { | ||
// Find a local DN and short circuit read is enabled | ||
key += "@" + localAddr.getHostName() + ":" + port + "/" + DomainSocketFactory.FEATURE_FLAG; | ||
} | ||
} | ||
|
||
if (localDN == null && (topologyAware || isEC)) { | ||
try { | ||
DatanodeDetails closestNode = pipeline.getClosestNode(); | ||
// Pipeline cache key uses host:port suffix to handle | ||
|
@@ -185,7 +249,7 @@ private String getPipelineCacheKey(Pipeline pipeline, | |
// Standalone port is chosen since all datanodes should have a | ||
// standalone port regardless of version and this port should not | ||
// have any collisions. | ||
key += closestNode.getHostName() + closestNode.getPort( | ||
key += closestNode.getHostName() + ":" + closestNode.getPort( | ||
DatanodeDetails.Port.Name.STANDALONE); | ||
} catch (IOException e) { | ||
LOG.error("Failed to get closest node to create pipeline cache key:" + | ||
|
@@ -197,12 +261,16 @@ private String getPipelineCacheKey(Pipeline pipeline, | |
// Append user short name to key to prevent a different user | ||
// from using same instance of xceiverClient. | ||
try { | ||
key += UserGroupInformation.getCurrentUser().getShortUserName(); | ||
key = UserGroupInformation.getCurrentUser().getShortUserName() + "@" + key; | ||
} catch (IOException e) { | ||
LOG.error("Failed to get current user to create pipeline cache key:" + | ||
e.getMessage()); | ||
} | ||
} | ||
|
||
if (localDN != null) { | ||
localDNCache.put(key, localDN); | ||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Move above log inside if condition. |
||
} | ||
return key; | ||
} | ||
|
||
|
@@ -211,12 +279,14 @@ private String getPipelineCacheKey(Pipeline pipeline, | |
*/ | ||
@Override | ||
public void close() { | ||
super.close(); | ||
//closing is done through RemovalListener | ||
clientCache.invalidateAll(); | ||
clientCache.cleanUp(); | ||
if (LOG.isDebugEnabled()) { | ||
LOG.debug("XceiverClient cache stats: {}", clientCache.stats()); | ||
} | ||
localDNCache.clear(); | ||
cacheMetrics.unregister(); | ||
|
||
if (metrics != null) { | ||
|
Oops, something went wrong.
Add this suggestion to a batch that can be applied as a single commit.
This suggestion is invalid because no changes were made to the code.
Suggestions cannot be applied while the pull request is closed.
Suggestions cannot be applied while viewing a subset of changes.
Only one suggestion per line can be applied in a batch.
Add this suggestion to a batch that can be applied as a single commit.
Applying suggestions on deleted lines is not supported.
You must change the existing code in this line in order to create a valid suggestion.
Outdated suggestions cannot be applied.
This suggestion has been applied or marked resolved.
Suggestions cannot be applied from pending reviews.
Suggestions cannot be applied on multi-line comments.
Suggestions cannot be applied while the pull request is queued to merge.
Suggestion cannot be applied right now. Please check back later.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
XceiverClientGrpc.LOG is used in unit test TestShortCircuitChunkInputStream.