Skip to content
Merged
Show file tree
Hide file tree
Changes from 1 commit
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
Expand Up @@ -201,6 +201,13 @@ public enum ChecksumCombineMode {
// 3 concurrent stripe read should be enough.
private int ecReconstructStripeReadPoolLimit = 10 * 3;

@Config(key = "ec.reconstruct.stripe.write.pool.limit",
defaultValue = "30",
description = "Thread pool max size for parallelly write" +
" available ec chunks to reconstruct the whole stripe.",
tags = ConfigTag.CLIENT)
private int ecReconstructStripeWritePoolLimit = 10 * 3;

@Config(key = "checksum.combine.mode",
defaultValue = "COMPOSITE_CRC",
description = "The combined checksum type [MD5MD5CRC / COMPOSITE_CRC] "
Expand Down Expand Up @@ -387,6 +394,14 @@ public int getEcReconstructStripeReadPoolLimit() {
return ecReconstructStripeReadPoolLimit;
}

public void setEcReconstructStripeWritePoolLimit(int poolLimit) {
this.ecReconstructStripeWritePoolLimit = poolLimit;
}

public int getEcReconstructStripeWritePoolLimit() {
return ecReconstructStripeWritePoolLimit;
}

public void setFsDefaultBucketLayout(String bucketLayout) {
if (!bucketLayout.isEmpty()) {
this.fsDefaultBucketLayout = bucketLayout;
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -44,6 +44,7 @@
import org.apache.hadoop.hdds.scm.XceiverClientSpi;
import org.apache.hadoop.hdds.scm.container.common.helpers.StorageContainerException;
import org.apache.hadoop.hdds.scm.pipeline.Pipeline;
import org.apache.hadoop.ozone.client.io.BlockOutputStreamResourceProvider;
import org.apache.hadoop.ozone.common.Checksum;
import org.apache.hadoop.ozone.common.ChecksumData;
import org.apache.hadoop.ozone.common.ChunkBuffer;
Expand Down Expand Up @@ -145,7 +146,8 @@ public BlockOutputStream(
BufferPool bufferPool,
OzoneClientConfig config,
Token<? extends TokenIdentifier> token,
ContainerClientMetrics clientMetrics, StreamBufferArgs streamBufferArgs
StreamBufferArgs streamBufferArgs,
BlockOutputStreamResourceProvider blockOutputStreamResourceProvider
) throws IOException {
this.xceiverClientFactory = xceiverClientManager;
this.config = config;
Expand Down Expand Up @@ -189,7 +191,7 @@ public BlockOutputStream(
ioException = new AtomicReference<>(null);
checksum = new Checksum(config.getChecksumType(),
config.getBytesPerChecksum());
this.clientMetrics = clientMetrics;
this.clientMetrics = blockOutputStreamResourceProvider.getClientMetrics();
this.pipeline = pipeline;
this.streamBufferArgs = streamBufferArgs;
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -23,13 +23,13 @@
import org.apache.hadoop.hdds.protocol.DatanodeDetails;
import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos;
import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.ChunkInfo;
import org.apache.hadoop.hdds.scm.ContainerClientMetrics;
import org.apache.hadoop.hdds.scm.OzoneClientConfig;
import org.apache.hadoop.hdds.scm.StreamBufferArgs;
import org.apache.hadoop.hdds.scm.XceiverClientFactory;
import org.apache.hadoop.hdds.scm.XceiverClientReply;
import org.apache.hadoop.hdds.scm.pipeline.Pipeline;
import org.apache.hadoop.ozone.OzoneConsts;
import org.apache.hadoop.ozone.client.io.BlockOutputStreamResourceProvider;
import org.apache.hadoop.ozone.common.ChunkBuffer;
import org.apache.hadoop.ozone.container.common.helpers.BlockData;
import org.apache.hadoop.security.token.Token;
Expand Down Expand Up @@ -75,10 +75,13 @@ public ECBlockOutputStream(
BufferPool bufferPool,
OzoneClientConfig config,
Token<? extends TokenIdentifier> token,
ContainerClientMetrics clientMetrics, StreamBufferArgs streamBufferArgs
StreamBufferArgs streamBufferArgs,
BlockOutputStreamResourceProvider blockOutputStreamResourceProvider
) throws IOException {
super(blockID, xceiverClientManager,
pipeline, bufferPool, config, token, clientMetrics, streamBufferArgs);
pipeline, bufferPool, config, token, streamBufferArgs,
blockOutputStreamResourceProvider
);
// In EC stream, there will be only one node in pipeline.
this.datanodeDetails = pipeline.getClosestNode();
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -21,12 +21,12 @@
import org.apache.hadoop.fs.Syncable;
import org.apache.hadoop.hdds.client.BlockID;
import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.ContainerCommandResponseProto;
import org.apache.hadoop.hdds.scm.ContainerClientMetrics;
import org.apache.hadoop.hdds.scm.OzoneClientConfig;
import org.apache.hadoop.hdds.scm.StreamBufferArgs;
import org.apache.hadoop.hdds.scm.XceiverClientFactory;
import org.apache.hadoop.hdds.scm.XceiverClientReply;
import org.apache.hadoop.hdds.scm.pipeline.Pipeline;
import org.apache.hadoop.ozone.client.io.BlockOutputStreamResourceProvider;
import org.apache.hadoop.ozone.common.ChunkBuffer;
import org.apache.hadoop.security.token.Token;
import org.apache.hadoop.security.token.TokenIdentifier;
Expand Down Expand Up @@ -65,8 +65,8 @@ public class RatisBlockOutputStream extends BlockOutputStream
/**
* Creates a new BlockOutputStream.
*
* @param blockID block ID
* @param bufferPool pool of buffers
* @param blockID block ID
* @param bufferPool pool of buffers
*/
@SuppressWarnings("checkstyle:ParameterNumber")
public RatisBlockOutputStream(
Expand All @@ -76,10 +76,13 @@ public RatisBlockOutputStream(
BufferPool bufferPool,
OzoneClientConfig config,
Token<? extends TokenIdentifier> token,
ContainerClientMetrics clientMetrics, StreamBufferArgs streamBufferArgs
StreamBufferArgs streamBufferArgs,
BlockOutputStreamResourceProvider blockOutputStreamResourceProvider
) throws IOException {
super(blockID, xceiverClientManager, pipeline,
bufferPool, config, token, clientMetrics, streamBufferArgs);
bufferPool, config, token, streamBufferArgs,
blockOutputStreamResourceProvider
);
this.commitWatcher = new CommitWatcher(bufferPool, getXceiverClient());
}

Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,60 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/

package org.apache.hadoop.ozone.client.io;

import java.util.concurrent.ExecutorService;
import java.util.function.Supplier;
import org.apache.hadoop.hdds.scm.ContainerClientMetrics;

/**
* Provides resources for BlockOutputStream, including executor service,
* and client metrics.
*/
public final class BlockOutputStreamResourceProvider {
Copy link
Contributor

Choose a reason for hiding this comment

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

It seems adding this new BlockOutputStreamResourceProvider is for parameter passing instead of add a new executorServiceSupplier parameter. However, executorServiceSupplier and clientMetrics are not related.

Let's pass the builders; filed HDDS-10387.

Copy link
Contributor Author

Choose a reason for hiding this comment

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

Done.

Copy link
Contributor

Choose a reason for hiding this comment

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

This class only has executorServiceSupplier. Let's pass it directly.

private final Supplier<ExecutorService> executorServiceSupplier;
private final ContainerClientMetrics clientMetrics;

/**
* Creates an instance of blockOutputStreamResourceProvider.
*/
public static BlockOutputStreamResourceProvider create(
Supplier<ExecutorService> executorServiceSupplier, ContainerClientMetrics clientMetrics) {
return new BlockOutputStreamResourceProvider(executorServiceSupplier, clientMetrics);
}

private BlockOutputStreamResourceProvider(Supplier<ExecutorService> executorServiceSupplier,
ContainerClientMetrics clientMetrics) {
this.executorServiceSupplier = executorServiceSupplier;
this.clientMetrics = clientMetrics;
}

/**
* Provides an ExecutorService, lazily initialized upon first request.
*/
public ExecutorService getExecutorService() {
return executorServiceSupplier.get();
}

/**
* Returns the ContainerClientMetrics instance.
*/
public ContainerClientMetrics getClientMetrics() {
return clientMetrics;
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -43,10 +43,12 @@
import org.apache.hadoop.hdds.scm.pipeline.MockPipeline;
import org.apache.hadoop.hdds.scm.pipeline.Pipeline;
import org.apache.hadoop.ozone.OzoneConsts;
import org.apache.hadoop.ozone.client.io.BlockOutputStreamResourceProvider;
import org.apache.ratis.thirdparty.com.google.protobuf.ByteString;
import org.junit.jupiter.params.ParameterizedTest;
import org.junit.jupiter.params.provider.ValueSource;

import static java.util.concurrent.Executors.newFixedThreadPool;
import static org.junit.jupiter.api.Assertions.assertEquals;
import static org.mockito.Mockito.any;
import static org.mockito.Mockito.mock;
Expand Down Expand Up @@ -108,7 +110,9 @@ private BlockOutputStream createBlockOutputStream(BufferPool bufferPool)
bufferPool,
config,
null,
ContainerClientMetrics.acquire(), streamBufferArgs);
streamBufferArgs,
BlockOutputStreamResourceProvider.create(() -> newFixedThreadPool(10),
ContainerClientMetrics.acquire()));
}

/**
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -45,6 +45,7 @@
import org.apache.hadoop.ozone.OzoneConsts;
import org.apache.hadoop.ozone.client.io.BlockInputStreamFactory;
import org.apache.hadoop.ozone.client.io.BlockInputStreamFactoryImpl;
import org.apache.hadoop.ozone.client.io.BlockOutputStreamResourceProvider;
import org.apache.hadoop.ozone.client.io.ECBlockInputStreamProxy;
import org.apache.hadoop.ozone.client.io.ECBlockReconstructedStripeInputStream;
import org.apache.hadoop.ozone.container.common.helpers.BlockData;
Expand Down Expand Up @@ -73,6 +74,7 @@
import java.util.concurrent.ThreadFactory;
import java.util.concurrent.ThreadPoolExecutor;
import java.util.concurrent.TimeUnit;
import java.util.function.Function;
import java.util.stream.Collectors;

import static org.apache.hadoop.ozone.container.ec.reconstruction.TokenHelper.encode;
Expand Down Expand Up @@ -101,18 +103,22 @@ public class ECReconstructionCoordinator implements Closeable {

private static final int EC_RECONSTRUCT_STRIPE_READ_POOL_MIN_SIZE = 3;

// TODO: Adjusts to the appropriate value when the ec-reconstruct-writer thread pool is used.
private static final int EC_RECONSTRUCT_STRIPE_WRITE_POOL_MIN_SIZE = 0;

private final ECContainerOperationClient containerOperationClient;

private final ByteBufferPool byteBufferPool;

private final ExecutorService ecReconstructExecutor;

private final ExecutorService ecReconstructReadExecutor;
private volatile ExecutorService ecReconstructWriteExecutor;
private final BlockInputStreamFactory blockInputStreamFactory;
private final TokenHelper tokenHelper;
private final ContainerClientMetrics clientMetrics;
private final ECReconstructionMetrics metrics;
private final StateContext context;
private final OzoneClientConfig ozoneClientConfig;
private final BlockOutputStreamResourceProvider
blockOutputStreamResourceProvider;

public ECReconstructionCoordinator(
ConfigurationSource conf, CertificateClient certificateClient,
Expand All @@ -123,22 +129,33 @@ public ECReconstructionCoordinator(
this.containerOperationClient = new ECContainerOperationClient(conf,
certificateClient);
this.byteBufferPool = new ElasticByteBufferPool();
ThreadFactory threadFactory = new ThreadFactoryBuilder()
.setNameFormat(threadNamePrefix + "ec-reconstruct-reader-TID-%d")
Function<String, ThreadFactory> threadFactoryCreator = name ->
new ThreadFactoryBuilder()
.setNameFormat(threadNamePrefix + name)
.build();
ozoneClientConfig = conf.getObject(OzoneClientConfig.class);
this.ecReconstructExecutor =
this.ecReconstructReadExecutor =
new ThreadPoolExecutor(EC_RECONSTRUCT_STRIPE_READ_POOL_MIN_SIZE,
ozoneClientConfig.getEcReconstructStripeReadPoolLimit(),
60,
TimeUnit.SECONDS,
new SynchronousQueue<>(),
threadFactory,
threadFactoryCreator.apply("ec-reconstruct-reader-TID-%d"),
new ThreadPoolExecutor.CallerRunsPolicy());
this.ecReconstructWriteExecutor =
new ThreadPoolExecutor(EC_RECONSTRUCT_STRIPE_WRITE_POOL_MIN_SIZE,
conf.getObject(OzoneClientConfig.class)
.getEcReconstructStripeWritePoolLimit(),
60,
TimeUnit.SECONDS,
new SynchronousQueue<>(),
threadFactoryCreator.apply("ec-reconstruct-writer-TID-%d"),
new ThreadPoolExecutor.CallerRunsPolicy());
this.blockInputStreamFactory = BlockInputStreamFactoryImpl
.getInstance(byteBufferPool, () -> ecReconstructExecutor);
.getInstance(byteBufferPool, () -> ecReconstructReadExecutor);
blockOutputStreamResourceProvider = BlockOutputStreamResourceProvider.create(
() -> ecReconstructWriteExecutor, ContainerClientMetrics.acquire());
tokenHelper = new TokenHelper(new SecurityConfig(conf), secretKeyClient);
this.clientMetrics = ContainerClientMetrics.acquire();
this.metrics = metrics;
}

Expand Down Expand Up @@ -232,7 +249,9 @@ private ECBlockOutputStream getECBlockOutputStream(
containerOperationClient.singleNodePipeline(datanodeDetails,
repConfig, replicaIndex),
BufferPool.empty(), ozoneClientConfig,
blockLocationInfo.getToken(), clientMetrics, streamBufferArgs);
blockLocationInfo.getToken(), streamBufferArgs,
blockOutputStreamResourceProvider
);
}

@VisibleForTesting
Expand Down Expand Up @@ -272,7 +291,7 @@ public void reconstructECBlockGroup(BlockLocationInfo blockLocationInfo,
repConfig, blockLocationInfo, true,
this.containerOperationClient.getXceiverClientManager(), null,
this.blockInputStreamFactory, byteBufferPool,
this.ecReconstructExecutor)) {
this.ecReconstructReadExecutor)) {

ECBlockOutputStream[] targetBlockStreams =
new ECBlockOutputStream[toReconstructIndexes.size()];
Expand Down Expand Up @@ -457,6 +476,10 @@ public void close() throws IOException {
if (containerOperationClient != null) {
containerOperationClient.close();
}
if (ecReconstructWriteExecutor != null) {
ecReconstructWriteExecutor.shutdownNow();
ecReconstructWriteExecutor = null;
}
}

private Pipeline rebuildInputPipeline(ECReplicationConfig repConfig,
Expand Down
Loading