szetszwo commented on code in PR #1306: URL: https://github.com/apache/ratis/pull/1306#discussion_r2524395030
########## ratis-grpc/src/main/java/org/apache/ratis/grpc/server/GrpcStubPool.java: ########## @@ -0,0 +1,112 @@ +/* + * 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.ratis.grpc.server; + +import org.apache.ratis.protocol.RaftPeer; +import org.apache.ratis.thirdparty.io.grpc.ManagedChannel; +import org.apache.ratis.thirdparty.io.grpc.netty.NegotiationType; +import org.apache.ratis.thirdparty.io.grpc.netty.NettyChannelBuilder; +import org.apache.ratis.thirdparty.io.grpc.stub.AbstractStub; +import org.apache.ratis.thirdparty.io.netty.channel.ChannelOption; +import org.apache.ratis.thirdparty.io.netty.channel.WriteBufferWaterMark; +import org.apache.ratis.thirdparty.io.netty.handler.ssl.SslContext; +import org.apache.ratis.util.JavaUtils; +import org.apache.ratis.util.MemoizedSupplier; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.util.ArrayList; +import java.util.Collections; +import java.util.List; +import java.util.concurrent.Semaphore; +import java.util.concurrent.ThreadLocalRandom; +import java.util.concurrent.TimeUnit; +import java.util.function.Function; + +final class GrpcStubPool<S extends AbstractStub<S>> { + public static final Logger LOG = LoggerFactory.getLogger(GrpcStubPool.class); + + static final class PooledStub<S extends AbstractStub<S>> { + private final ManagedChannel ch; + private final S stub; + private final Semaphore permits; + + PooledStub(ManagedChannel ch, S stub, int maxInflight) { + this.ch = ch; + this.stub = stub; + this.permits = new Semaphore(maxInflight); + } + + S getStub() { + return stub; + } + + void release() { + permits.release(); + } + } + + private final List<MemoizedSupplier<PooledStub<S>>> pool; + private final int size; + + GrpcStubPool(RaftPeer target, int n, Function<ManagedChannel, S> stubFactory, SslContext sslContext) { + this(target, n, stubFactory, sslContext, 16); + } + + GrpcStubPool(RaftPeer target, int n, Function<ManagedChannel, S> stubFactory, SslContext sslContext, + int maxInflightPerConn) { + ArrayList<MemoizedSupplier<PooledStub<S>>> tmp = new ArrayList<>(n); + for (int i = 0; i < n; i++) { + NettyChannelBuilder channelBuilder = NettyChannelBuilder.forTarget(target.getAddress()) + .keepAliveTime(10, TimeUnit.MINUTES) + .keepAliveWithoutCalls(false) + .idleTimeout(30, TimeUnit.MINUTES) + .withOption(ChannelOption.WRITE_BUFFER_WATER_MARK, new WriteBufferWaterMark(64 << 10, 128 << 10)); + if (sslContext != null) { + LOG.debug("Setting TLS for {}", target.getAddress()); + channelBuilder.useTransportSecurity().sslContext(sslContext); + } else { + channelBuilder.negotiationType(NegotiationType.PLAINTEXT); + } + ManagedChannel ch = channelBuilder.build(); + tmp.add(JavaUtils.memoize(() -> new PooledStub<>(ch, stubFactory.apply(ch), maxInflightPerConn))); + ch.getState(true); + } + this.pool = Collections.unmodifiableList(tmp); + this.size = n; + } + + PooledStub<S> acquire() throws InterruptedException { + final int start = ThreadLocalRandom.current().nextInt(size); + for (int k = 0; k < size; k++) { + PooledStub<S> p = pool.get((start + k) % size).get(); + if (p.permits.tryAcquire()) { + return p; + } + } + final PooledStub<S> p = pool.get(start).get(); + p.permits.acquire(); + return p; + } + + public void close() { + for (MemoizedSupplier<PooledStub<S>> p : pool) { + p.get().ch.shutdown(); Review Comment: We should check it is already initialized. If not, it should not be initialized and then shut down. -- 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]
