|
| 1 | +/* |
| 2 | + * Copyright 2025, AutoMQ HK Limited. |
| 3 | + * |
| 4 | + * Licensed to the Apache Software Foundation (ASF) under one or more |
| 5 | + * contributor license agreements. See the NOTICE file distributed with |
| 6 | + * this work for additional information regarding copyright ownership. |
| 7 | + * The ASF licenses this file to You under the Apache License, Version 2.0 |
| 8 | + * (the "License"); you may not use this file except in compliance with |
| 9 | + * the License. You may obtain a copy of the License at |
| 10 | + * |
| 11 | + * http://www.apache.org/licenses/LICENSE-2.0 |
| 12 | + * |
| 13 | + * Unless required by applicable law or agreed to in writing, software |
| 14 | + * distributed under the License is distributed on an "AS IS" BASIS, |
| 15 | + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. |
| 16 | + * See the License for the specific language governing permissions and |
| 17 | + * limitations under the License. |
| 18 | + */ |
| 19 | + |
| 20 | +package kafka.automq.zerozone; |
| 21 | + |
| 22 | +import kafka.server.KafkaConfig; |
| 23 | + |
| 24 | +import org.apache.kafka.clients.ApiVersions; |
| 25 | +import org.apache.kafka.clients.ClientRequest; |
| 26 | +import org.apache.kafka.clients.ClientResponse; |
| 27 | +import org.apache.kafka.clients.ManualMetadataUpdater; |
| 28 | +import org.apache.kafka.clients.MetadataRecoveryStrategy; |
| 29 | +import org.apache.kafka.clients.NetworkClient; |
| 30 | +import org.apache.kafka.clients.NetworkClientUtils; |
| 31 | +import org.apache.kafka.clients.RequestCompletionHandler; |
| 32 | +import org.apache.kafka.common.Node; |
| 33 | +import org.apache.kafka.common.metrics.Metrics; |
| 34 | +import org.apache.kafka.common.network.ChannelBuilder; |
| 35 | +import org.apache.kafka.common.network.ChannelBuilders; |
| 36 | +import org.apache.kafka.common.network.NetworkReceive; |
| 37 | +import org.apache.kafka.common.network.Selectable; |
| 38 | +import org.apache.kafka.common.network.Selector; |
| 39 | +import org.apache.kafka.common.requests.AbstractRequest; |
| 40 | +import org.apache.kafka.common.security.JaasContext; |
| 41 | +import org.apache.kafka.common.utils.LogContext; |
| 42 | +import org.apache.kafka.common.utils.Time; |
| 43 | + |
| 44 | +import com.automq.stream.utils.Threads; |
| 45 | + |
| 46 | +import org.slf4j.Logger; |
| 47 | +import org.slf4j.LoggerFactory; |
| 48 | + |
| 49 | +import java.net.SocketTimeoutException; |
| 50 | +import java.util.Collections; |
| 51 | +import java.util.Map; |
| 52 | +import java.util.Queue; |
| 53 | +import java.util.concurrent.CompletableFuture; |
| 54 | +import java.util.concurrent.ConcurrentHashMap; |
| 55 | +import java.util.concurrent.ConcurrentLinkedQueue; |
| 56 | +import java.util.concurrent.ConcurrentMap; |
| 57 | +import java.util.concurrent.ExecutorService; |
| 58 | +import java.util.concurrent.atomic.AtomicBoolean; |
| 59 | + |
| 60 | +public interface AsyncSender { |
| 61 | + |
| 62 | + <T extends AbstractRequest> CompletableFuture<ClientResponse> sendRequest( |
| 63 | + Node node, |
| 64 | + AbstractRequest.Builder<T> requestBuilder |
| 65 | + ); |
| 66 | + |
| 67 | + void initiateClose(); |
| 68 | + |
| 69 | + void close(); |
| 70 | + |
| 71 | + class BrokersAsyncSender implements AsyncSender { |
| 72 | + private static final Logger LOGGER = LoggerFactory.getLogger(BrokersAsyncSender.class); |
| 73 | + private final NetworkClient networkClient; |
| 74 | + private final Time time; |
| 75 | + private final ExecutorService executorService; |
| 76 | + private final AtomicBoolean shouldRun = new AtomicBoolean(true); |
| 77 | + |
| 78 | + public BrokersAsyncSender( |
| 79 | + KafkaConfig brokerConfig, |
| 80 | + Metrics metrics, |
| 81 | + String metricGroupPrefix, |
| 82 | + Time time, |
| 83 | + String clientId, |
| 84 | + LogContext logContext |
| 85 | + ) { |
| 86 | + |
| 87 | + ChannelBuilder channelBuilder = ChannelBuilders.clientChannelBuilder( |
| 88 | + brokerConfig.interBrokerSecurityProtocol(), |
| 89 | + JaasContext.Type.SERVER, |
| 90 | + brokerConfig, |
| 91 | + brokerConfig.interBrokerListenerName(), |
| 92 | + brokerConfig.saslMechanismInterBrokerProtocol(), |
| 93 | + time, |
| 94 | + brokerConfig.saslInterBrokerHandshakeRequestEnable(), |
| 95 | + logContext |
| 96 | + ); |
| 97 | + Selector selector = new Selector( |
| 98 | + NetworkReceive.UNLIMITED, |
| 99 | + brokerConfig.connectionsMaxIdleMs(), |
| 100 | + metrics, |
| 101 | + time, |
| 102 | + metricGroupPrefix, |
| 103 | + Collections.emptyMap(), |
| 104 | + false, |
| 105 | + channelBuilder, |
| 106 | + logContext |
| 107 | + ); |
| 108 | + this.networkClient = new NetworkClient( |
| 109 | + selector, |
| 110 | + new ManualMetadataUpdater(), |
| 111 | + clientId, |
| 112 | + 5, |
| 113 | + 0, |
| 114 | + 0, |
| 115 | + Selectable.USE_DEFAULT_BUFFER_SIZE, |
| 116 | + brokerConfig.replicaSocketReceiveBufferBytes(), |
| 117 | + brokerConfig.requestTimeoutMs(), |
| 118 | + brokerConfig.connectionSetupTimeoutMs(), |
| 119 | + brokerConfig.connectionSetupTimeoutMaxMs(), |
| 120 | + time, |
| 121 | + false, |
| 122 | + new ApiVersions(), |
| 123 | + logContext, |
| 124 | + MetadataRecoveryStrategy.REBOOTSTRAP |
| 125 | + ); |
| 126 | + this.time = time; |
| 127 | + executorService = Threads.newFixedThreadPoolWithMonitor(1, metricGroupPrefix, true, LOGGER); |
| 128 | + executorService.submit(this::run); |
| 129 | + } |
| 130 | + |
| 131 | + private final ConcurrentMap<Node, Queue<Request>> waitingSendRequests = new ConcurrentHashMap<>(); |
| 132 | + |
| 133 | + @Override |
| 134 | + public <T extends AbstractRequest> CompletableFuture<ClientResponse> sendRequest(Node node, |
| 135 | + AbstractRequest.Builder<T> requestBuilder) { |
| 136 | + CompletableFuture<ClientResponse> cf = new CompletableFuture<>(); |
| 137 | + waitingSendRequests.compute(node, (n, queue) -> { |
| 138 | + if (queue == null) { |
| 139 | + queue = new ConcurrentLinkedQueue<>(); |
| 140 | + } |
| 141 | + queue.add(new Request(requestBuilder, cf)); |
| 142 | + return queue; |
| 143 | + }); |
| 144 | + return cf; |
| 145 | + } |
| 146 | + |
| 147 | + private void run() { |
| 148 | + Map<Node, Long> lastUnreadyTimes = new ConcurrentHashMap<>(); |
| 149 | + while (shouldRun.get()) { |
| 150 | + // TODO: graceful shutdown |
| 151 | + try { |
| 152 | + long now = time.milliseconds(); |
| 153 | + waitingSendRequests.forEach((node, queue) -> { |
| 154 | + if (queue.isEmpty()) { |
| 155 | + return; |
| 156 | + } |
| 157 | + if (NetworkClientUtils.isReady(networkClient, node, now)) { |
| 158 | + lastUnreadyTimes.remove(node); |
| 159 | + Request request = queue.poll(); |
| 160 | + ClientRequest clientRequest = networkClient.newClientRequest(Integer.toString(node.id()), request.requestBuilder, now, true, 3000, new RequestCompletionHandler() { |
| 161 | + @Override |
| 162 | + public void onComplete(ClientResponse response) { |
| 163 | + request.cf.complete(response); |
| 164 | + } |
| 165 | + }); |
| 166 | + networkClient.send(clientRequest, now); |
| 167 | + } else { |
| 168 | + Long lastUnreadyTime = lastUnreadyTimes.get(node); |
| 169 | + if (lastUnreadyTime == null) { |
| 170 | + networkClient.ready(node, now); |
| 171 | + lastUnreadyTimes.put(node, now); |
| 172 | + } else { |
| 173 | + if (now - lastUnreadyTime > 3000) { |
| 174 | + for (; ; ) { |
| 175 | + Request request = queue.poll(); |
| 176 | + if (request == null) { |
| 177 | + break; |
| 178 | + } |
| 179 | + request.cf.completeExceptionally(new SocketTimeoutException(String.format("Cannot connect to node=%s", node))); |
| 180 | + } |
| 181 | + lastUnreadyTimes.remove(node); |
| 182 | + } else { |
| 183 | + // attempt to reconnect |
| 184 | + networkClient.ready(node, now); |
| 185 | + } |
| 186 | + } |
| 187 | + } |
| 188 | + }); |
| 189 | + networkClient.poll(1, now); |
| 190 | + } catch (Throwable e) { |
| 191 | + LOGGER.error("Processor get uncaught exception", e); |
| 192 | + } |
| 193 | + } |
| 194 | + } |
| 195 | + |
| 196 | + @Override |
| 197 | + public void initiateClose() { |
| 198 | + networkClient.initiateClose(); |
| 199 | + } |
| 200 | + |
| 201 | + @Override |
| 202 | + public void close() { |
| 203 | + networkClient.close(); |
| 204 | + shouldRun.set(false); |
| 205 | + } |
| 206 | + } |
| 207 | + |
| 208 | + class Request { |
| 209 | + final AbstractRequest.Builder<?> requestBuilder; |
| 210 | + final CompletableFuture<ClientResponse> cf; |
| 211 | + |
| 212 | + public Request(AbstractRequest.Builder<?> requestBuilder, CompletableFuture<ClientResponse> cf) { |
| 213 | + this.requestBuilder = requestBuilder; |
| 214 | + this.cf = cf; |
| 215 | + } |
| 216 | + } |
| 217 | + |
| 218 | +} |
0 commit comments