Skip to content

Commit

Permalink
[C++] Add detail logs for schema related messages (#9544)
Browse files Browse the repository at this point in the history
### Motivation

If broker failed to parse the schema, the logs of C++/Python client is poor because when C++ client handles the `Error` response, it doesn't print the `message` field that is filled by broker. On the other hand, if the `InvalidSchemaDataException` is thrown by `StructSchemaDataValidator#throwInvalidSchemaDataException`, the detail error message will be written to the exception's cause. The String that `getMessage()` returns is not enough

### Modifications

- When `tryAddSchema` failed, add the exception cause's message to the error message that's sent to client.
- Print `Error` response's `message` field to logs.

### Verifying this change

- [ ] Make sure that the change passes the CI checks.

This change is a trivial rework / code cleanup without any test coverage. Take #9483 (which is to be fixed) as the example, before this PR, the client side's log is like

> Received error response from server: IncompatibleSchema -- req_id: 0

After this PR, the client side's log will be

> Received error response from server: IncompatibleSchema (Invalid schema definition data for AVRO schema caused by org.apache.avro.SchemaParseException: Undefined name: "array") -- req_id: 0
  • Loading branch information
BewareMyPower authored Feb 11, 2021
1 parent fb0f3e3 commit d2d5fb6
Show file tree
Hide file tree
Showing 2 changed files with 11 additions and 2 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -1167,9 +1167,13 @@ protected void handleProducer(final CommandProducer cmdProducer) {
CompletableFuture<SchemaVersion> schemaVersionFuture = tryAddSchema(topic, schema);

schemaVersionFuture.exceptionally(exception -> {
String message = exception.getMessage();
if (exception.getCause() != null) {
message += (" caused by " + exception.getCause());
}
commandSender.sendErrorResponse(requestId,
BrokerServiceException.getClientErrorCode(exception),
exception.getMessage());
message);
producers.remove(producerId, producerFuture);
return null;
});
Expand Down Expand Up @@ -1727,7 +1731,11 @@ protected void handleGetOrCreateSchema(CommandGetOrCreateSchema commandGetOrCrea
CompletableFuture<SchemaVersion> schemaVersionFuture = tryAddSchema(topic, schema);
schemaVersionFuture.exceptionally(ex -> {
ServerError errorCode = BrokerServiceException.getClientErrorCode(ex);
commandSender.sendGetOrCreateSchemaErrorResponse(requestId, errorCode, ex.getMessage());
String message = ex.getMessage();
if (ex.getCause() != null) {
message += (" caused by " + ex.getCause());
}
commandSender.sendGetOrCreateSchemaErrorResponse(requestId, errorCode, message);
return null;
}).thenAccept(schemaVersion -> {
commandSender.sendGetOrCreateSchemaResponse(requestId, schemaVersion);
Expand Down
1 change: 1 addition & 0 deletions pulsar-client-cpp/lib/ClientConnection.cc
Original file line number Diff line number Diff line change
Expand Up @@ -976,6 +976,7 @@ void ClientConnection::handleIncomingCommand() {
const CommandError& error = incomingCmd_.error();
Result result = getResult(error.error());
LOG_WARN(cnxString_ << "Received error response from server: " << result
<< (error.has_message() ? (" (" + error.message() + ")") : "")
<< " -- req_id: " << error.request_id());

Lock lock(mutex_);
Expand Down

0 comments on commit d2d5fb6

Please sign in to comment.