Skip to content

disconnect on repeated errors #97

New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

Open
wants to merge 2 commits into
base: 2.10_ds
Choose a base branch
from
Open
Changes from all commits
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 @@ -16,6 +16,9 @@
import static com.google.common.base.Preconditions.checkArgument;
import static io.streamnative.pulsar.handlers.kop.KafkaRequestHandler.newNode;

import com.google.common.cache.CacheBuilder;
import com.google.common.cache.CacheLoader;
import com.google.common.cache.LoadingCache;
import io.netty.buffer.ByteBuf;
import io.netty.channel.ChannelHandlerContext;
import io.netty.channel.EventLoopGroup;
Expand Down Expand Up @@ -63,6 +66,7 @@
import lombok.extern.slf4j.Slf4j;
import org.apache.commons.lang.StringUtils;
import org.apache.commons.lang3.NotImplementedException;
import org.apache.kafka.common.KafkaException;
import org.apache.kafka.common.Node;
import org.apache.kafka.common.TopicPartition;
import org.apache.kafka.common.acl.AclOperation;
Expand Down Expand Up @@ -204,6 +208,16 @@ public class KafkaProxyRequestHandler extends KafkaCommandDecoder {
private AtomicInteger dummyCorrelationIdGenerator = new AtomicInteger(-1);
private volatile boolean coordinatorNamespaceExists = false;

LoadingCache<String, AtomicInteger> errorCounts = CacheBuilder.newBuilder()
.expireAfterWrite(1, TimeUnit.MINUTES)
.build(new CacheLoader<>() {
@Override
public AtomicInteger load(String key) {
return new AtomicInteger(0);
}
});


public KafkaProxyRequestHandler(String id, KafkaProtocolProxyMain.PulsarAdminProvider pulsarAdmin,
AuthenticationService authenticationService,
AuthorizationService authorizationService,
Expand Down Expand Up @@ -571,6 +585,9 @@ protected void handleProduceRequest(KafkaHeaderAndRequest produceHar,
.stream()
.collect(Collectors.toMap(Function.identity(),
p -> new PartitionResponse(Errors.REQUEST_TIMED_OUT)));
incErrorCountAndMaybeDiscardConnectionToBroker(badError,
broker.node.host(),
broker.node.port());
resultFuture.complete(new ProduceResponse(errorsMap));
return null;
});
Expand Down Expand Up @@ -713,6 +730,9 @@ protected void handleProduceRequest(KafkaHeaderAndRequest produceHar,
.setErrorMessage(Errors.REQUEST_TIMED_OUT.message()));
});
});
incErrorCountAndMaybeDiscardConnectionToBroker(badError,
kopBroker.host(),
kopBroker.port());
return null;
}).whenComplete((ignore1, ignore2) -> {
singlePartitionRequest.close();
Expand Down Expand Up @@ -815,14 +835,17 @@ protected void handleFetchRequest(KafkaHeaderAndRequest fetch,
// all the partitions are owned by one single broker,
// we can forward the whole request to the only broker
log.debug("forward FULL fetch of {} parts to {}", numPartitions, first);
grabConnectionToBroker(first.node.host(), first.node.port()).
final String host = first.node.host();
final int port = first.node.port();
grabConnectionToBroker(host, port).
forwardRequest(fetch)
.thenAccept(response -> {
resultFuture.complete(response);
}).exceptionally(badError -> {
log.error("bad error for FULL fetch", badError);
FetchResponse fetchResponse = buildFetchErrorResponse(fetchRequest,
fetchData, Errors.UNKNOWN_SERVER_ERROR);
incErrorCountAndMaybeDiscardConnectionToBroker(badError, host, port);
resultFuture.complete(fetchResponse);
return null;
});
Expand Down Expand Up @@ -963,6 +986,9 @@ protected void handleFetchRequest(KafkaHeaderAndRequest fetch,
fetchData.keySet().forEach(topicPartition ->
errorsConsumer.accept(topicPartition, Errors.UNKNOWN_SERVER_ERROR)
);
incErrorCountAndMaybeDiscardConnectionToBroker(badError,
kopBroker.host(),
kopBroker.port());
return null;
}).whenComplete((ignore1, ignore2) -> {
singlePartitionRequest.close();
Expand Down Expand Up @@ -1237,6 +1263,7 @@ R extends AbstractResponse> void sendRequestToAllTopicOwners(
}).exceptionally(err -> {
log.error("Error sending coordinator request to {} :{}",
node, err);
incErrorCountAndMaybeDiscardConnectionToBroker(err, node.host(), node.port());
responseFromBroker.complete(errorBuilder.apply(request, err));
return null;
});
Expand Down Expand Up @@ -1621,7 +1648,9 @@ protected void handleDeleteRecords(KafkaHeaderAndRequest deleteRecords,
// all the partitions are owned by one single broker,
// we can forward the whole request to the only broker
log.debug("forward FULL DeleteRecords of {} parts to {}", numPartitions, first);
grabConnectionToBroker(first.node.host(), first.node.port()).
final String host = first.node.host();
final int port = first.node.port();
grabConnectionToBroker(host, port).
forwardRequest(deleteRecords)
.thenAccept(response -> {
resultFuture.complete(response);
Expand All @@ -1633,6 +1662,7 @@ protected void handleDeleteRecords(KafkaHeaderAndRequest deleteRecords,
.stream()
.collect(Collectors.toMap(Function.identity(),
p -> Errors.UNKNOWN_SERVER_ERROR));
incErrorCountAndMaybeDiscardConnectionToBroker(badError, host, port);
resultFuture.complete(KafkaResponseUtils.newDeleteRecords(errorsMap));
return null;
});
Expand Down Expand Up @@ -1777,6 +1807,9 @@ protected void handleDeleteRecords(KafkaHeaderAndRequest deleteRecords,
errorsConsumer.accept(topicPartition, Errors.UNKNOWN_SERVER_ERROR);
});
});
incErrorCountAndMaybeDiscardConnectionToBroker(badError,
kopBroker.host(),
kopBroker.port());
return null;
}).whenComplete((ignore1, ignore2) -> {
singlePartitionRequest.close();
Expand Down Expand Up @@ -2031,6 +2064,9 @@ protected void handleListOffsetRequestV0(KafkaHeaderAndRequest listOffset,
}).exceptionally(err -> {
ListOffsetsResponseData responseData =
buildDummyListOffsetsResponseData(entry.getKey());
incErrorCountAndMaybeDiscardConnectionToBroker(err,
brokerAddress.node.host(),
brokerAddress.node.port());
onResponse.accept(fullPartitionName, responseData);
return null;
}).whenComplete((ignore1, ignore2) -> {
Expand Down Expand Up @@ -2161,6 +2197,9 @@ protected void handleListOffsetRequestV1(KafkaHeaderAndRequest listOffset,
}).exceptionally(err -> {
ListOffsetsResponseData responseData =
buildDummyListOffsetsResponseData(topicPartition);
incErrorCountAndMaybeDiscardConnectionToBroker(err,
brokerAddress.node.host(),
brokerAddress.node.port());
onResponse.accept(fullPartitionName, responseData);
return null;
}).whenComplete((ignore1, ignore2) -> {
Expand Down Expand Up @@ -2342,6 +2381,8 @@ R extends AbstractResponse> void handleRequestWithCoordinator(
}).exceptionally(err -> {
log.error("Error sending {} coordinator for id {} request to {} :{}",
coordinatorType, transactionalId, metadata.node, err);
incErrorCountAndMaybeDiscardConnectionToBroker(err, metadata.node.host(),
metadata.node.port());
resultFuture.complete(errorBuilder.apply(request, err));
return null;
});
Expand Down Expand Up @@ -2479,6 +2520,7 @@ R extends AbstractResponse> void sendRequestToAllCoordinators(
}).exceptionally(err -> {
log.error("Error sending {} coordinator request to {} :{}",
coordinatorType, node, err);
incErrorCountAndMaybeDiscardConnectionToBroker(err, node.host(), node.port());
responseFromBroker.complete(errorBuilder.apply(request, err));
return null;
});
Expand Down Expand Up @@ -2719,7 +2761,39 @@ void discardConnectionToBroker(ConnectionToBroker connectionToBroker) {
});
}

void forgetMetadataForFailedBroker(String brokerHost, int brokerPort) {
void incErrorCountAndMaybeDiscardConnectionToBroker(Throwable error, String brokerHost, int brokerPort) {
if (error instanceof KafkaException) {
if (log.isDebugEnabled()) {
log.debug("Skipping internal kafka exceptions for {}:{}", brokerHost, brokerPort, error);
}
return;
}

String connectionKey = brokerHost + ":" + brokerPort;
int errorCount;
try {
errorCount = errorCounts.get(connectionKey).incrementAndGet();
} catch (ExecutionException e) {
errorCount = 0;
}
if (errorCount > 10) {
log.info("Closing connection to {}:{} due to having {} errors/min; last error:",
brokerHost, brokerPort, errorCount, error);
discardConnectionToBroker(connectionKey);
}
}

void discardConnectionToBroker(String connectionKey) {
connectionsToBrokers.compute(connectionKey, (key, existing) -> {
if (existing != null) {
existing.close();
}
return null;
});

}

void forgetMetadataForFailedBroker(String brokerHost, int brokerPort) {
Collection<String> keysToRemove = topicsLeaders
.entrySet()
.stream()
Expand Down