|
@@ -25,6 +25,10 @@ import com.google.common.cache.CacheBuilder;
|
|
|
import com.google.common.cache.RemovalListener;
|
|
|
import com.google.common.cache.RemovalNotification;
|
|
|
import org.apache.hadoop.conf.Configuration;
|
|
|
+import org.apache.hadoop.hdds.conf.Config;
|
|
|
+import org.apache.hadoop.hdds.conf.ConfigGroup;
|
|
|
+import org.apache.hadoop.hdds.conf.ConfigType;
|
|
|
+import org.apache.hadoop.hdds.conf.OzoneConfiguration;
|
|
|
import org.apache.hadoop.hdds.scm.pipeline.Pipeline;
|
|
|
import org.apache.hadoop.hdds.protocol.proto.HddsProtos;
|
|
|
import org.apache.hadoop.ozone.OzoneConfigKeys;
|
|
@@ -38,14 +42,9 @@ import java.io.IOException;
|
|
|
import java.util.concurrent.Callable;
|
|
|
import java.util.concurrent.TimeUnit;
|
|
|
|
|
|
-import static org.apache.hadoop.hdds.scm.ScmConfigKeys
|
|
|
- .SCM_CONTAINER_CLIENT_MAX_SIZE_DEFAULT;
|
|
|
-import static org.apache.hadoop.hdds.scm.ScmConfigKeys
|
|
|
- .SCM_CONTAINER_CLIENT_MAX_SIZE_KEY;
|
|
|
-import static org.apache.hadoop.hdds.scm.ScmConfigKeys
|
|
|
- .SCM_CONTAINER_CLIENT_STALE_THRESHOLD_DEFAULT;
|
|
|
-import static org.apache.hadoop.hdds.scm.ScmConfigKeys
|
|
|
- .SCM_CONTAINER_CLIENT_STALE_THRESHOLD_KEY;
|
|
|
+import static java.util.concurrent.TimeUnit.MILLISECONDS;
|
|
|
+import static org.apache.hadoop.hdds.conf.ConfigTag.OZONE;
|
|
|
+import static org.apache.hadoop.hdds.conf.ConfigTag.PERFORMANCE;
|
|
|
|
|
|
/**
|
|
|
* XceiverClientManager is responsible for the lifecycle of XceiverClient
|
|
@@ -76,20 +75,21 @@ public class XceiverClientManager implements Closeable {
|
|
|
* @param conf configuration
|
|
|
*/
|
|
|
public XceiverClientManager(Configuration conf) {
|
|
|
+ this(conf, OzoneConfiguration.of(conf).getObject(ScmClientConfig.class));
|
|
|
+ }
|
|
|
+
|
|
|
+ public XceiverClientManager(Configuration conf, ScmClientConfig clientConf) {
|
|
|
+ Preconditions.checkNotNull(clientConf);
|
|
|
Preconditions.checkNotNull(conf);
|
|
|
- int maxSize = conf.getInt(SCM_CONTAINER_CLIENT_MAX_SIZE_KEY,
|
|
|
- SCM_CONTAINER_CLIENT_MAX_SIZE_DEFAULT);
|
|
|
- long staleThresholdMs = conf.getTimeDuration(
|
|
|
- SCM_CONTAINER_CLIENT_STALE_THRESHOLD_KEY,
|
|
|
- SCM_CONTAINER_CLIENT_STALE_THRESHOLD_DEFAULT, TimeUnit.MILLISECONDS);
|
|
|
+ long staleThresholdMs = clientConf.getStaleThreshold(MILLISECONDS);
|
|
|
this.useRatis = conf.getBoolean(
|
|
|
ScmConfigKeys.DFS_CONTAINER_RATIS_ENABLED_KEY,
|
|
|
ScmConfigKeys.DFS_CONTAINER_RATIS_ENABLED_DEFAULT);
|
|
|
this.conf = conf;
|
|
|
this.isSecurityEnabled = OzoneSecurityUtil.isSecurityEnabled(conf);
|
|
|
this.clientCache = CacheBuilder.newBuilder()
|
|
|
- .expireAfterAccess(staleThresholdMs, TimeUnit.MILLISECONDS)
|
|
|
- .maximumSize(maxSize)
|
|
|
+ .expireAfterAccess(staleThresholdMs, MILLISECONDS)
|
|
|
+ .maximumSize(clientConf.getMaxSize())
|
|
|
.removalListener(
|
|
|
new RemovalListener<String, XceiverClientSpi>() {
|
|
|
@Override
|
|
@@ -299,4 +299,65 @@ public class XceiverClientManager implements Closeable {
|
|
|
|
|
|
return metrics;
|
|
|
}
|
|
|
+
|
|
|
+ /**
|
|
|
+ * Configuration for HDDS client.
|
|
|
+ */
|
|
|
+ @ConfigGroup(prefix = "scm.container.client")
|
|
|
+ public static class ScmClientConfig {
|
|
|
+
|
|
|
+ private int maxSize;
|
|
|
+ private long staleThreshold;
|
|
|
+ private int maxOutstandingRequests;
|
|
|
+
|
|
|
+ public long getStaleThreshold(TimeUnit unit) {
|
|
|
+ return unit.convert(staleThreshold, MILLISECONDS);
|
|
|
+ }
|
|
|
+
|
|
|
+ @Config(key = "idle.threshold",
|
|
|
+ type = ConfigType.TIME, timeUnit = MILLISECONDS,
|
|
|
+ defaultValue = "10s",
|
|
|
+ tags = { OZONE, PERFORMANCE },
|
|
|
+ description =
|
|
|
+ "In the standalone pipelines, the SCM clients use netty to "
|
|
|
+ + " communicate with the container. It also uses connection pooling"
|
|
|
+ + " to reduce client side overheads. This allows a connection to"
|
|
|
+ + " stay idle for a while before the connection is closed."
|
|
|
+ )
|
|
|
+ public void setStaleThreshold(long staleThreshold) {
|
|
|
+ this.staleThreshold = staleThreshold;
|
|
|
+ }
|
|
|
+
|
|
|
+ public int getMaxSize() {
|
|
|
+ return maxSize;
|
|
|
+ }
|
|
|
+
|
|
|
+ @Config(key = "max.size",
|
|
|
+ defaultValue = "256",
|
|
|
+ tags = { OZONE, PERFORMANCE },
|
|
|
+ description =
|
|
|
+ "Controls the maximum number of connections that are cached via"
|
|
|
+ + " client connection pooling. If the number of connections"
|
|
|
+ + " exceed this count, then the oldest idle connection is evicted."
|
|
|
+ )
|
|
|
+ public void setMaxSize(int maxSize) {
|
|
|
+ this.maxSize = maxSize;
|
|
|
+ }
|
|
|
+
|
|
|
+ public int getMaxOutstandingRequests() {
|
|
|
+ return maxOutstandingRequests;
|
|
|
+ }
|
|
|
+
|
|
|
+ @Config(key = "max.outstanding.requests",
|
|
|
+ defaultValue = "100",
|
|
|
+ tags = { OZONE, PERFORMANCE },
|
|
|
+ description =
|
|
|
+ "Controls the maximum number of outstanding async requests that can"
|
|
|
+ + " be handled by the Standalone as well as Ratis client."
|
|
|
+ )
|
|
|
+ public void setMaxOutstandingRequests(int maxOutstandingRequests) {
|
|
|
+ this.maxOutstandingRequests = maxOutstandingRequests;
|
|
|
+ }
|
|
|
+ }
|
|
|
+
|
|
|
}
|