This is an automated email from the ASF dual-hosted git repository.
adoroszlai pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/ozone.git
The following commit(s) were added to refs/heads/master by this push:
new 5dd14acf4f HDDS-10294. Actual client configuration ignored in
ECReconstructionCoordinator (#6163)
5dd14acf4f is described below
commit 5dd14acf4f0826fdb795866c274db301a7682981
Author: Doroszlai, Attila <[email protected]>
AuthorDate: Mon Feb 5 15:12:41 2024 +0100
HDDS-10294. Actual client configuration ignored in
ECReconstructionCoordinator (#6163)
---
.../ec/reconstruction/ECReconstructionCoordinator.java | 17 ++++++++---------
1 file changed, 8 insertions(+), 9 deletions(-)
diff --git
a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/ec/reconstruction/ECReconstructionCoordinator.java
b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/ec/reconstruction/ECReconstructionCoordinator.java
index 24e76821f9..234439a00c 100644
---
a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/ec/reconstruction/ECReconstructionCoordinator.java
+++
b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/ec/reconstruction/ECReconstructionCoordinator.java
@@ -112,6 +112,7 @@ public class ECReconstructionCoordinator implements
Closeable {
private final ContainerClientMetrics clientMetrics;
private final ECReconstructionMetrics metrics;
private final StateContext context;
+ private final OzoneClientConfig ozoneClientConfig;
public ECReconstructionCoordinator(
ConfigurationSource conf, CertificateClient certificateClient,
@@ -125,10 +126,10 @@ public class ECReconstructionCoordinator implements
Closeable {
ThreadFactory threadFactory = new ThreadFactoryBuilder()
.setNameFormat(threadNamePrefix + "ec-reconstruct-reader-TID-%d")
.build();
+ ozoneClientConfig = conf.getObject(OzoneClientConfig.class);
this.ecReconstructExecutor =
new ThreadPoolExecutor(EC_RECONSTRUCT_STRIPE_READ_POOL_MIN_SIZE,
- conf.getObject(OzoneClientConfig.class)
- .getEcReconstructStripeReadPoolLimit(),
+ ozoneClientConfig.getEcReconstructStripeReadPoolLimit(),
60,
TimeUnit.SECONDS,
new SynchronousQueue<>(),
@@ -222,16 +223,15 @@ public class ECReconstructionCoordinator implements
Closeable {
private ECBlockOutputStream getECBlockOutputStream(
BlockLocationInfo blockLocationInfo, DatanodeDetails datanodeDetails,
- ECReplicationConfig repConfig, int replicaIndex,
- OzoneClientConfig configuration) throws IOException {
+ ECReplicationConfig repConfig, int replicaIndex) throws IOException {
StreamBufferArgs streamBufferArgs =
- StreamBufferArgs.getDefaultStreamBufferArgs(repConfig, configuration);
+ StreamBufferArgs.getDefaultStreamBufferArgs(repConfig,
ozoneClientConfig);
return new ECBlockOutputStream(
blockLocationInfo.getBlockID(),
containerOperationClient.getXceiverClientManager(),
containerOperationClient.singleNodePipeline(datanodeDetails,
repConfig, replicaIndex),
- BufferPool.empty(), configuration,
+ BufferPool.empty(), ozoneClientConfig,
blockLocationInfo.getToken(), clientMetrics, streamBufferArgs);
}
@@ -277,15 +277,14 @@ public class ECReconstructionCoordinator implements
Closeable {
ECBlockOutputStream[] targetBlockStreams =
new ECBlockOutputStream[toReconstructIndexes.size()];
ByteBuffer[] bufs = new ByteBuffer[toReconstructIndexes.size()];
- OzoneClientConfig configuration = new OzoneClientConfig();
try {
for (int i = 0; i < toReconstructIndexes.size(); i++) {
int replicaIndex = toReconstructIndexes.get(i);
DatanodeDetails datanodeDetails =
targetMap.get(replicaIndex);
targetBlockStreams[i] = getECBlockOutputStream(blockLocationInfo,
- datanodeDetails, repConfig, replicaIndex,
- configuration);
+ datanodeDetails, repConfig, replicaIndex
+ );
bufs[i] = byteBufferPool.getBuffer(false,
repConfig.getEcChunkSize());
// Make sure it's clean. Don't want to reuse the erroneously returned
// buffers from the pool.
---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]