CompletableFuture<LookupDataResult> newLookup(ByteBuf request, long requestId) { CompletableFuture<LookupDataResult> future = new CompletableFuture<>(); if (addPendingLookupRequests(requestId, future)) { ctx.writeAndFlush(request).addListener(writeFuture -> { if (!writeFuture.isSuccess()) { log.warn("{} Failed to send request {} to broker: {}", ctx.channel(), requestId, writeFuture.cause().getMessage()); getAndRemovePendingLookupRequest(requestId); future.completeExceptionally(writeFuture.cause()); } }); } else { if (log.isDebugEnabled()) { log.debug("{} Failed to add lookup-request into pending queue", requestId); } future.completeExceptionally(new PulsarClientException.TooManyRequestsException( "Failed due to too many pending lookup requests")); } return future; }
private void resendMessages(ClientCnx cnx) { cnx.ctx().channel().eventLoop().execute(() -> { synchronized (this) { if (getState() == State.Closing || getState() == State.Closed) { cnx.channel().close(); return; cnx.channel().close(); return; final boolean stripChecksum = cnx.getRemoteEndpointProtocolVersion() < brokerChecksumSupportedVersion(); for (OpSendMsg op : pendingMessages) { if (log.isDebugEnabled()) { log.debug("[{}] [{}] Re-Sending message in cnx {}, sequenceId {}", topic, producerName, cnx.channel(), op.sequenceId); cnx.ctx().write(op.cmd, cnx.ctx().voidPromise()); stats.updateNumMsgsSent(op.numMessagesInBatch, op.batchSizeByte); cnx.ctx().flush(); if (!changeToReadyState()) { cnx.channel().close(); return;
cnx.connectionFuture().thenRun(() -> { if (log.isDebugEnabled()) { log.debug("[{}] Connection handshake completed", cnx.channel()); log.warn("[{}] Connection handshake failed: {}", cnx.channel(), exception.getMessage()); cnxFuture.completeExceptionally(exception); cleanupConnection(address, connectionKey, cnxFuture); cnx.ctx().close(); return null; });
cnx.registerProducer(producerId, this); log.info("[{}] [{}] Creating producer on cnx {}", topic, producerName, cnx.ctx().channel()); cnx.sendRequestWithId(Commands.newProducer(topic, producerId, requestId, producerName), requestId) .thenAccept(producerName -> { cnx.removeProducer(producerId); cnx.channel().close(); return; log.info("[{}] [{}] Created producer on cnx {}", topic, producerName, cnx.ctx().channel()); connectionId = cnx.ctx().channel().toString(); connectedSince = DATE_FORMAT.format(Instant.now()); cnx.removeProducer(producerId); if (getState() == State.Closing || getState() == State.Closed) { cnx.channel().close(); return null;
@Override void connectionOpened(final ClientCnx cnx) { setClientCnx(cnx); cnx.registerConsumer(consumerId, this); log.info("[{}][{}] Subscribing to topic on cnx {}", topic, subscription, cnx.ctx().channel()); cnx.sendRequestWithId(request, requestId).thenRun(() -> { synchronized (ConsumerImpl.this) { if (changeToReadyState()) { log.info("[{}][{}] Subscribed to topic on {} -- consumer: {}", topic, subscription, cnx.channel().remoteAddress(), consumerId); cnx.removeConsumer(consumerId); cnx.channel().close(); return; cnx.removeConsumer(consumerId); if (getState() == State.Closing || getState() == State.Closed) { cnx.channel().close(); return null; cnx.channel().remoteAddress()); if (e.getCause() instanceof PulsarClientException && isRetriableError((PulsarClientException) e.getCause())
@Override protected void handlePartitionResponse(CommandPartitionedTopicMetadataResponse lookupResult) { log.info("Received Broker Partition response: {}", lookupResult.getPartitions()); long requestId = lookupResult.getRequestId(); CompletableFuture<LookupDataResult> requestFuture = getAndRemovePendingLookupRequest(requestId); if (requestFuture != null) { // Complete future with exception if : Result.response=fail/null if (!lookupResult.hasResponse() || CommandPartitionedTopicMetadataResponse.LookupType.Failed.equals(lookupResult.getResponse())) { if (lookupResult.hasError()) { checkServerError(lookupResult.getError(), lookupResult.getMessage()); requestFuture.completeExceptionally( getPulsarClientException(lookupResult.getError(), lookupResult.getMessage())); } else { requestFuture .completeExceptionally(new PulsarClientException.LookupException("Empty lookup response")); } } else { // return LookupDataResult when Result.response = success/redirect requestFuture.complete(new LookupDataResult(lookupResult.getPartitions())); } } else { log.warn("{} Received unknown request id from server: {}", ctx.channel(), lookupResult.getRequestId()); } }
cnx.sendRequestWithId(cmd, requestId).handle((v, exception) -> { cnx.removeProducer(producerId); if (exception == null || !cnx.ctx().channel().isActive()) {
cnx.sendRequestWithId(cmd, requestId).handle((v, exception) -> { cnx.removeConsumer(consumerId); if (exception == null || !cnx.ctx().channel().isActive()) { log.info("[{}] [{}] Closed consumer", topic, subscription); setState(State.Closed);
@Override public void redeliverUnacknowledgedMessages() { ClientCnx cnx = cnx(); if (isConnected() && cnx.getRemoteEndpointProtocolVersion() >= ProtocolVersion.v2.getNumber()) { int currentSize = 0; synchronized (this) { currentSize = incomingMessages.size(); incomingMessages.clear(); unAckedMessageTracker.clear(); batchMessageAckTracker.clear(); } cnx.ctx().writeAndFlush(Commands.newRedeliverUnacknowledgedMessages(consumerId), cnx.ctx().voidPromise()); if (currentSize > 0) { increaseAvailablePermits(cnx, currentSize); } if (log.isDebugEnabled()) { log.debug("[{}] [{}] [{}] Redeliver unacked messages and send {} permits", subscription, topic, consumerName, currentSize); } return; } if (cnx == null || (getState() == State.Connecting)) { log.warn("[{}] Client Connection needs to be establised for redelivery of unacknowledged messages", this); } else { log.warn("[{}] Reconnecting the client to redeliver the messages.", this); cnx.ctx().close(); } }
public boolean isWritable() { ClientCnx cnx = getClientCnx(); return cnx != null && cnx.channel().isWritable(); }
private CompletableFuture<PartitionedTopicMetadata> getPartitionedTopicMetadata(InetSocketAddress socketAddress, DestinationName destination) { CompletableFuture<PartitionedTopicMetadata> partitionFuture = new CompletableFuture<PartitionedTopicMetadata>(); client.getCnxPool().getConnection(socketAddress).thenAccept(clientCnx -> { long requestId = client.newRequestId(); ByteBuf request = Commands.newPartitionMetadataRequest(destination.toString(), requestId); clientCnx.newLookup(request, requestId).thenAccept(lookupDataResult -> { try { partitionFuture.complete(new PartitionedTopicMetadata(lookupDataResult.partitions)); } catch (Exception e) { partitionFuture.completeExceptionally(new PulsarClientException.LookupException( format("Failed to parse partition-response redirect=%s , partitions with %s", lookupDataResult.redirect, lookupDataResult.partitions, e.getMessage()))); } }).exceptionally((e) -> { log.warn("[{}] failed to get Partitioned metadata : {}", destination.toString(), e.getCause().getMessage(), e); partitionFuture.completeExceptionally(e); return null; }); }).exceptionally(connectionException -> { partitionFuture.completeExceptionally(connectionException); return null; }); return partitionFuture; }
@Override protected void handleError(CommandError error) { checkArgument(state == State.Ready); log.warn("{} Received error from server: {}", ctx.channel(), error.getMessage()); long requestId = error.getRequestId(); if (error.getError() == ServerError.ProducerBlockedQuotaExceededError) { log.warn("{} Producer creation has been blocked because backlog quota exceeded for producer topic", ctx.channel()); } CompletableFuture<String> requestFuture = pendingRequests.remove(requestId); if (requestFuture != null) { requestFuture.completeExceptionally(getPulsarClientException(error.getError(), error.getMessage())); } else { log.warn("{} Received unknown request id from server: {}", ctx.channel(), error.getRequestId()); } }
private ByteBuf sendMessage(long producerId, long sequenceId, int numMessages, MessageMetadata msgMetadata, ByteBuf compressedPayload) throws IOException { ChecksumType checksumType; if (getClientCnx() == null || getClientCnx().getRemoteEndpointProtocolVersion() >= brokerChecksumSupportedVersion()) { checksumType = ChecksumType.Crc32c; } else { checksumType = ChecksumType.None; } return Commands.newSend(producerId, sequenceId, numMessages, checksumType, msgMetadata, compressedPayload); }
public void initChannel(SocketChannel ch) throws Exception { ClientConfiguration clientConfig = client.getConfiguration(); if (clientConfig.isUseTls()) { SslContextBuilder builder = SslContextBuilder.forClient(); if (clientConfig.isTlsAllowInsecureConnection()) { builder.trustManager(InsecureTrustManagerFactory.INSTANCE); } else { if (clientConfig.getTlsTrustCertsFilePath().isEmpty()) { // Use system default builder.trustManager((File) null); } else { File trustCertCollection = new File(clientConfig.getTlsTrustCertsFilePath()); builder.trustManager(trustCertCollection); } } // Set client certificate if available AuthenticationDataProvider authData = clientConfig.getAuthentication().getAuthData(); if (authData.hasDataForTls()) { builder.keyManager(authData.getTlsPrivateKey(), (X509Certificate[]) authData.getTlsCertificates()); } SslContext sslCtx = builder.build(); ch.pipeline().addLast(TLS_HANDLER, sslCtx.newHandler(ch.alloc())); } ch.pipeline().addLast("frameDecoder", new PulsarLengthFieldFrameDecoder(MaxMessageSize, 0, 4, 0, 4)); ch.pipeline().addLast("handler", new ClientCnx(client)); } });
@Override protected void handleLookupResponse(CommandLookupTopicResponse lookupResult) { log.info("Received Broker lookup response: {}", lookupResult.getResponse()); long requestId = lookupResult.getRequestId(); CompletableFuture<LookupDataResult> requestFuture = getAndRemovePendingLookupRequest(requestId); if (requestFuture != null) { // Complete future with exception if : Result.response=fail/null if (!lookupResult.hasResponse() || CommandLookupTopicResponse.LookupType.Failed.equals(lookupResult.getResponse())) { if (lookupResult.hasError()) { checkServerError(lookupResult.getError(), lookupResult.getMessage()); requestFuture.completeExceptionally( getPulsarClientException(lookupResult.getError(), lookupResult.getMessage())); } else { requestFuture .completeExceptionally(new PulsarClientException.LookupException("Empty lookup response")); } } else { // return LookupDataResult when Result.response = connect/redirect boolean redirect = CommandLookupTopicResponse.LookupType.Redirect.equals(lookupResult.getResponse()); requestFuture.complete(new LookupDataResult(lookupResult.getBrokerServiceUrl(), lookupResult.getBrokerServiceUrlTls(), redirect, lookupResult.getAuthoritative())); } } else { log.warn("{} Received unknown request id from server: {}", ctx.channel(), lookupResult.getRequestId()); } }
if (isConnected() && cnx.getRemoteEndpointProtocolVersion() >= ProtocolVersion.v2.getNumber()) { int messagesFromQueue = removeExpiredMessagesFromQueue(messageIds); Iterable<List<MessageIdImpl>> batches = Iterables.partition(messageIds, MAX_REDELIVER_UNACKNOWLEDGED); }).collect(Collectors.toList()); ByteBuf cmd = Commands.newRedeliverUnacknowledgedMessages(consumerId, messageIdDatas); cnx.ctx().writeAndFlush(cmd, cnx.ctx().voidPromise()); messageIdDatas.forEach(MessageIdData::recycle); }); } else { log.warn("[{}] Reconnecting the client to redeliver the messages.", this); cnx.ctx().close();
/** * send the flow command to have the broker start pushing messages */ void sendFlowPermitsToBroker(ClientCnx cnx, int numMessages) { if (cnx != null) { if (log.isDebugEnabled()) { log.debug("[{}] [{}] Adding {} additional permits", topic, subscription, numMessages); } cnx.ctx().writeAndFlush(Commands.newFlow(consumerId, numMessages), cnx.ctx().voidPromise()); } }
expectedSequenceId, sequenceId, pendingMessages.size()); cnx.channel().close(); } else if (sequenceId < expectedSequenceId) {
long requestId = client.newRequestId(); ByteBuf request = Commands.newLookup(destination.toString(), authoritative, requestId); clientCnx.newLookup(request, requestId).thenAccept(lookupDataResult -> { URI uri = null; try {