-
Notifications
You must be signed in to change notification settings - Fork 1k
Commit
This commit does not belong to any branch on this repository, and may belong to a fork outside of the repository.
feat: add serialization exceptions to processing logger (#6084)
- Loading branch information
Showing
12 changed files
with
607 additions
and
49 deletions.
There are no files selected for viewing
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
57 changes: 57 additions & 0 deletions
57
ksqldb-serde/src/main/java/io/confluent/ksql/logging/processing/LoggingSerializer.java
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,57 @@ | ||
/* | ||
* Copyright 2020 Confluent Inc. | ||
* | ||
* Licensed under the Confluent Community License (the "License"); you may not use | ||
* this file except in compliance with the License. You may obtain a copy of the | ||
* License at | ||
* | ||
* http://www.confluent.io/confluent-community-license | ||
* | ||
* Unless required by applicable law or agreed to in writing, software | ||
* distributed under the License is distributed on an "AS IS" BASIS, WITHOUT | ||
* WARRANTIES OF ANY KIND, either express or implied. See the License for the | ||
* specific language governing permissions and limitations under the License. | ||
*/ | ||
|
||
package io.confluent.ksql.logging.processing; | ||
|
||
import static java.util.Objects.requireNonNull; | ||
|
||
import java.util.Map; | ||
import java.util.Optional; | ||
import org.apache.kafka.common.serialization.Serializer; | ||
|
||
public final class LoggingSerializer<T> implements Serializer<T> { | ||
|
||
private final Serializer<T> delegate; | ||
private final ProcessingLogger processingLogger; | ||
|
||
public LoggingSerializer( | ||
final Serializer<T> delegate, | ||
final ProcessingLogger processingLogger | ||
) { | ||
this.delegate = requireNonNull(delegate, "delegate"); | ||
this.processingLogger = requireNonNull(processingLogger, "processingLogger"); | ||
} | ||
|
||
@Override | ||
public void configure(final Map<String, ?> configs, final boolean isKey) { | ||
delegate.configure(configs, isKey); | ||
} | ||
|
||
@Override | ||
public byte[] serialize(final String topic, final T data) { | ||
try { | ||
return delegate.serialize(topic, data); | ||
} catch (final RuntimeException e) { | ||
processingLogger.error(new SerializationError<>(e, Optional.of(data), topic)); | ||
throw e; | ||
} | ||
} | ||
|
||
@Override | ||
public void close() { | ||
delegate.close(); | ||
} | ||
|
||
} |
101 changes: 101 additions & 0 deletions
101
ksqldb-serde/src/main/java/io/confluent/ksql/logging/processing/SerializationError.java
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,101 @@ | ||
/* | ||
* Copyright 2020 Confluent Inc. | ||
* | ||
* Licensed under the Confluent Community License (the "License"); you may not use | ||
* this file except in compliance with the License. You may obtain a copy of the | ||
* License at | ||
* | ||
* http://www.confluent.io/confluent-community-license | ||
* | ||
* Unless required by applicable law or agreed to in writing, software | ||
* distributed under the License is distributed on an "AS IS" BASIS, WITHOUT | ||
* WARRANTIES OF ANY KIND, either express or implied. See the License for the | ||
* specific language governing permissions and limitations under the License. | ||
*/ | ||
|
||
package io.confluent.ksql.logging.processing; | ||
|
||
import static java.util.Objects.requireNonNull; | ||
|
||
import io.confluent.ksql.logging.processing.ProcessingLogMessageSchema.MessageType; | ||
import io.confluent.ksql.util.ErrorMessageUtil; | ||
import java.util.List; | ||
import java.util.Objects; | ||
import java.util.Optional; | ||
import org.apache.kafka.connect.data.SchemaAndValue; | ||
import org.apache.kafka.connect.data.Struct; | ||
|
||
public class SerializationError<T> implements ProcessingLogger.ErrorMessage { | ||
|
||
private final Throwable exception; | ||
private final Optional<T> record; | ||
private final String topic; | ||
|
||
public SerializationError( | ||
final Throwable exception, | ||
final Optional<T> record, | ||
final String topic | ||
) { | ||
this.exception = requireNonNull(exception, "exception"); | ||
this.record = requireNonNull(record, "record"); | ||
this.topic = requireNonNull(topic, "topic"); | ||
} | ||
|
||
@Override | ||
public SchemaAndValue get(final ProcessingLogConfig config) { | ||
final Struct struct = new Struct(ProcessingLogMessageSchema.PROCESSING_LOG_SCHEMA) | ||
.put(ProcessingLogMessageSchema.TYPE, MessageType.SERIALIZATION_ERROR.getTypeId()) | ||
.put(ProcessingLogMessageSchema.SERIALIZATION_ERROR, serializationError(config)); | ||
|
||
return new SchemaAndValue(ProcessingLogMessageSchema.PROCESSING_LOG_SCHEMA, struct); | ||
} | ||
|
||
@Override | ||
public boolean equals(final Object o) { | ||
if (this == o) { | ||
return true; | ||
} | ||
if (o == null || getClass() != o.getClass()) { | ||
return false; | ||
} | ||
final SerializationError<?> that = (SerializationError) o; | ||
return Objects.equals(exception, that.exception) | ||
&& Objects.equals(record, that.record) | ||
&& Objects.equals(topic, that.topic); | ||
} | ||
|
||
@Override | ||
public int hashCode() { | ||
return Objects.hash(exception, record, topic); | ||
} | ||
|
||
private Struct serializationError(final ProcessingLogConfig config) { | ||
final Struct serializationError = new Struct(MessageType.SERIALIZATION_ERROR.getSchema()) | ||
.put( | ||
ProcessingLogMessageSchema.SERIALIZATION_ERROR_FIELD_MESSAGE, | ||
exception.getMessage()) | ||
.put( | ||
ProcessingLogMessageSchema.SERIALIZATION_ERROR_FIELD_CAUSE, | ||
getCause() | ||
) | ||
.put( | ||
ProcessingLogMessageSchema.SERIALIZATION_ERROR_FIELD_TOPIC, | ||
topic | ||
); | ||
|
||
if (config.getBoolean(ProcessingLogConfig.INCLUDE_ROWS)) { | ||
serializationError.put( | ||
ProcessingLogMessageSchema.SERIALIZATION_ERROR_FIELD_RECORD, | ||
record.map(Object::toString).orElse(null) | ||
); | ||
} | ||
|
||
return serializationError; | ||
} | ||
|
||
private List<String> getCause() { | ||
final List<String> cause = ErrorMessageUtil.getErrorMessages(exception); | ||
cause.remove(0); | ||
return cause; | ||
} | ||
} |
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Oops, something went wrong.