szetszwo commented on code in PR #6222: URL: https://github.com/apache/ozone/pull/6222#discussion_r1491475800
########## hadoop-hdds/client/src/main/java/org/apache/hadoop/ozone/client/io/BlockOutputStreamResourceProvider.java: ########## @@ -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 { Review Comment: 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. ########## hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/rpc/RpcClient.java: ########## @@ -2496,26 +2507,45 @@ public void setTimes(OzoneObj obj, String keyName, long mtime, long atime) ozoneManagerClient.setTimes(builder.build(), mtime, atime); } + private ExecutorService createThreadPoolExecutor( + int corePoolSize, int maximumPoolSize, String threadNameFormat) { + return new ThreadPoolExecutor(corePoolSize, maximumPoolSize, + 60, TimeUnit.SECONDS, new SynchronousQueue<>(), + new ThreadFactoryBuilder() + .setNameFormat(threadNameFormat) + .build(), + new ThreadPoolExecutor.CallerRunsPolicy()); + } + public ExecutorService getECReconstructExecutor() { - // local ref to a volatile to ensure access - // to a completed initialized object - ExecutorService executor = ecReconstructExecutor; - if (executor == null) { + ExecutorService localRef = ecReconstructExecutor; + if (localRef == null) { synchronized (this) { - executor = ecReconstructExecutor; - if (executor == null) { - ecReconstructExecutor = new ThreadPoolExecutor( - EC_RECONSTRUCT_STRIPE_READ_POOL_MIN_SIZE, + localRef = ecReconstructExecutor; + if (localRef == null) { + localRef = createThreadPoolExecutor(EC_RECONSTRUCT_STRIPE_READ_POOL_MIN_SIZE, clientConfig.getEcReconstructStripeReadPoolLimit(), - 60, TimeUnit.SECONDS, new SynchronousQueue<>(), - new ThreadFactoryBuilder() - .setNameFormat("ec-reconstruct-reader-TID-%d") - .build(), - new ThreadPoolExecutor.CallerRunsPolicy()); - executor = ecReconstructExecutor; + "ec-reconstruct-reader-TID-%d"); + ecReconstructExecutor = localRef; + } + } + } + return localRef; + } + + public ExecutorService getWriteThreadPool() { + ExecutorService localRef = writeExecutor; + if (localRef == null) { + synchronized (this) { + localRef = writeExecutor; + if (localRef == null) { + localRef = createThreadPoolExecutor(WRITE_POOL_MIN_SIZE, + Integer.MAX_VALUE, + "client-write-TID-%d"); + writeExecutor = localRef; } } } Review Comment: Use `MemoizedSupplier`: ```java diff --git a/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/rpc/RpcClient.java b/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/rpc/RpcClient.java index 94d6ae9769..abb981caa9 100644 --- a/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/rpc/RpcClient.java +++ b/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/rpc/RpcClient.java @@ -145,6 +145,7 @@ import org.apache.hadoop.security.UserGroupInformation; import org.apache.hadoop.security.token.Token; import org.apache.ratis.protocol.ClientId; +import org.apache.ratis.util.MemoizedSupplier; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -195,6 +196,9 @@ public class RpcClient implements ClientProtocol { // for reconstruction. private static final int EC_RECONSTRUCT_STRIPE_READ_POOL_MIN_SIZE = 3; + // TODO: Adjusts to the appropriate value when the writeThreadPool is used. + private static final int WRITE_POOL_MIN_SIZE = 0; + private final ConfigurationSource conf; private final OzoneManagerClientProtocol ozoneManagerClient; private final XceiverClientFactory xceiverClientManager; @@ -213,7 +217,8 @@ public class RpcClient implements ClientProtocol { private final ByteBufferPool byteBufferPool; private final BlockInputStreamFactory blockInputStreamFactory; private final OzoneManagerVersion omVersion; - private volatile ExecutorService ecReconstructExecutor; + private final MemoizedSupplier<ExecutorService> ecReconstructExecutor; + private final MemoizedSupplier<ExecutorService> writeExecutor; private final ContainerClientMetrics clientMetrics; private final AtomicBoolean isS3GRequest = new AtomicBoolean(false); @@ -237,6 +242,12 @@ public RpcClient(ConfigurationSource conf, String omServiceId) this.groupRights = aclConfig.getGroupDefaultRights(); this.clientConfig = conf.getObject(OzoneClientConfig.class); + this.ecReconstructExecutor = MemoizedSupplier.valueOf(() -> createThreadPoolExecutor( + EC_RECONSTRUCT_STRIPE_READ_POOL_MIN_SIZE, + clientConfig.getEcReconstructStripeReadPoolLimit(), + "ec-reconstruct-reader-TID-%d")); + this.writeExecutor = MemoizedSupplier.valueOf(() -> createThreadPoolExecutor( + WRITE_POOL_MIN_SIZE, Integer.MAX_VALUE, "client-write-TID-%d")); OmTransport omTransport = createOmTransport(omServiceId); OzoneManagerProtocolClientSideTranslatorPB @@ -311,7 +322,7 @@ public void onRemoval( }).build(); this.byteBufferPool = new ElasticByteBufferPool(); this.blockInputStreamFactory = BlockInputStreamFactoryImpl - .getInstance(byteBufferPool, this::getECReconstructExecutor); + .getInstance(byteBufferPool, ecReconstructExecutor); this.clientMetrics = ContainerClientMetrics.acquire(); } @@ -1752,9 +1763,11 @@ private OmKeyInfo getKeyInfo(OmKeyArgs keyArgs) throws IOException { @Override public void close() throws IOException { - if (ecReconstructExecutor != null) { - ecReconstructExecutor.shutdownNow(); - ecReconstructExecutor = null; + if (ecReconstructExecutor.isInitialized()) { + ecReconstructExecutor.get().shutdownNow(); + } + if (writeExecutor.isInitialized()) { + writeExecutor.get().shutdownNow(); } IOUtils.cleanupWithLogger(LOG, ozoneManagerClient, xceiverClientManager); keyProviderCache.invalidateAll(); @@ -2496,26 +2509,11 @@ public void setTimes(OzoneObj obj, String keyName, long mtime, long atime) ozoneManagerClient.setTimes(builder.build(), mtime, atime); } - public ExecutorService getECReconstructExecutor() { - // local ref to a volatile to ensure access - // to a completed initialized object - ExecutorService executor = ecReconstructExecutor; - if (executor == null) { - synchronized (this) { - executor = ecReconstructExecutor; - if (executor == null) { - ecReconstructExecutor = new ThreadPoolExecutor( - EC_RECONSTRUCT_STRIPE_READ_POOL_MIN_SIZE, - clientConfig.getEcReconstructStripeReadPoolLimit(), - 60, TimeUnit.SECONDS, new SynchronousQueue<>(), - new ThreadFactoryBuilder() - .setNameFormat("ec-reconstruct-reader-TID-%d") - .build(), - new ThreadPoolExecutor.CallerRunsPolicy()); - executor = ecReconstructExecutor; - } - } - } - return executor; + private static ExecutorService createThreadPoolExecutor( + int corePoolSize, int maximumPoolSize, String threadNameFormat) { + return new ThreadPoolExecutor(corePoolSize, maximumPoolSize, + 60, TimeUnit.SECONDS, new SynchronousQueue<>(), + new ThreadFactoryBuilder().setNameFormat(threadNameFormat).build(), + new ThreadPoolExecutor.CallerRunsPolicy()); } } ``` -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. To unsubscribe, e-mail: [email protected] For queries about this service, please contact Infrastructure at: [email protected] --------------------------------------------------------------------- To unsubscribe, e-mail: [email protected] For additional commands, e-mail: [email protected]
