yashmayya commented on code in PR #12984:
URL: https://github.com/apache/kafka/pull/12984#discussion_r1084960389
##########
connect/runtime/src/main/java/org/apache/kafka/connect/storage/KafkaConfigBackingStore.java:
##########
@@ -712,8 +733,16 @@ KafkaBasedLog<String, byte[]>
setupAndCreateKafkaBasedLog(String topic, final Wo
}
private void sendPrivileged(String key, byte[] value) {
+ sendPrivileged(key, value, null);
+ }
+
+ private void sendPrivileged(String key, byte[] value, Callback<Void>
callback) {
if (!usesFencableWriter) {
- configLog.send(key, value);
Review Comment:
Yeah, that's right, the API response will only include the generic top level
message.
> I think it'd be nice to include more detail on the cause of the failure
I strongly agree, and this was discussed in some more detail on the [other
thread](https://github.com/apache/kafka/pull/12984#discussion_r1064077119).
> We wouldn't be making it more vague. The message would state that the
write to the config topic failed which is the cause for failure. Since the
exception mapper used by Connect's REST server only writes the [top level
exception's
message](https://github.com/apache/kafka/blob/d798ec779c25dba31fa5ee9384d159ed54c6e07b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/rest/errors/ConnectExceptionMapper.java#L72)
to the response (i.e. nested exceptions aren't surfaced via the REST API
response), I think it makes sense to keep the top level exception's message
generic and allow users to debug further via the worker logs (where the entire
exception chain's stack trace will be visible).
...
The reasoning here is that since a Connect user may not even know that
Connect uses a producer under the hood to write certain requests to the config
topic for asynchronous processing, it would make more sense to have an
informative Connect specific exception message rather than directly throwing
the producer exception which may or may not contain enough details to be
relevant to a Connect user.
> Another option for the above issue could be changing the exception mapper
to concatenate all the exception messages from the exception chain.
> Yet another option for this could be to simply append a "Check the worker
logs for more details on the error" to the top level exception's message in the
REST API response (the worker logs will have the entire exception chain).
Thoughts?
What do you think about modifying the exception mapper to be more
informative (either in this PR or a separate one)?
##########
connect/runtime/src/main/java/org/apache/kafka/connect/storage/KafkaConfigBackingStore.java:
##########
@@ -711,9 +742,9 @@ KafkaBasedLog<String, byte[]>
setupAndCreateKafkaBasedLog(String topic, final Wo
return createKafkaBasedLog(topic, producerProps, consumerProps, new
ConsumeCallback(), topicDescription, adminSupplier);
}
- private void sendPrivileged(String key, byte[] value) {
+ private void sendPrivileged(String key, byte[] value) throws
ExecutionException, InterruptedException {
if (!usesFencableWriter) {
- configLog.send(key, value);
+ configLog.send(key, value).get();
Review Comment:
Thanks Chris, both great points. The `get` without timeout here was
definitely a miss on my part. I've addressed both of your raised concerns in
the latest patch (including batching multiple sends in a single transaction for
the EOS enabled case).
--
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]