-
Notifications
You must be signed in to change notification settings - Fork 13.8k
[FLINK-38819] Add debug logs for serialisation errors #27370
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
base: master
Are you sure you want to change the base?
Changes from all commits
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
There are no files selected for viewing
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -36,6 +36,9 @@ | |
|
|
||
| import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.JsonNode; | ||
|
|
||
| import org.slf4j.Logger; | ||
| import org.slf4j.LoggerFactory; | ||
|
|
||
| import javax.annotation.Nullable; | ||
|
|
||
| import java.io.IOException; | ||
|
|
@@ -62,6 +65,8 @@ | |
| public final class CanalJsonDeserializationSchema implements DeserializationSchema<RowData> { | ||
| private static final long serialVersionUID = 1L; | ||
|
|
||
| private static final Logger LOG = LoggerFactory.getLogger(CanalJsonDeserializationSchema.class); | ||
|
|
||
| private static final String FIELD_OLD = "old"; | ||
| private static final String OP_INSERT = "INSERT"; | ||
| private static final String OP_UPDATE = "UPDATE"; | ||
|
|
@@ -288,13 +293,22 @@ public void deserialize(@Nullable byte[] message, Collector<RowData> out) throws | |
| "Unknown \"type\" value \"%s\". The Canal JSON message is '%s'", | ||
|
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. nit: the IOException is the same as the debug content - can we use the same variable?
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. nit to the nit: It is not the same content.
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. fair point - lets not do this nit. . |
||
| type, new String(message))); | ||
| } | ||
| if (LOG.isDebugEnabled()) { | ||
| LOG.debug( | ||
| "Unknown \"type\" value '{}'. The Canal JSON message is '{}'.", | ||
| type, | ||
| new String(message)); | ||
| } | ||
| } | ||
| } catch (Throwable t) { | ||
| // a big try catch to protect the processing. | ||
| if (!ignoreParseErrors) { | ||
| throw new IOException( | ||
| format("Corrupt Canal JSON message '%s'.", new String(message)), t); | ||
| } | ||
| if (LOG.isDebugEnabled()) { | ||
| LOG.debug("Corrupt Canal JSON message '{}'.", new String(message), t); | ||
| } | ||
| } | ||
| for (GenericRowData genericRowData : genericRowDataList) { | ||
| out.collect(genericRowData); | ||
|
|
||
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -28,9 +28,11 @@ | |
| import org.apache.flink.table.types.DataType; | ||
| import org.apache.flink.table.types.logical.RowType; | ||
| import org.apache.flink.table.types.utils.DataTypeUtils; | ||
| import org.apache.flink.testutils.logging.LoggerAuditingExtension; | ||
| import org.apache.flink.util.Collector; | ||
|
|
||
| import org.junit.jupiter.api.Test; | ||
| import org.junit.jupiter.api.extension.RegisterExtension; | ||
|
|
||
| import java.io.File; | ||
| import java.io.IOException; | ||
|
|
@@ -54,10 +56,15 @@ | |
| import static org.apache.flink.table.api.DataTypes.STRING; | ||
| import static org.assertj.core.api.Assertions.assertThat; | ||
| import static org.assertj.core.api.Assertions.assertThatThrownBy; | ||
| import static org.slf4j.event.Level.DEBUG; | ||
|
|
||
| /** Tests for {@link CanalJsonSerializationSchema} and {@link CanalJsonDeserializationSchema}. */ | ||
| class CanalJsonSerDeSchemaTest { | ||
|
|
||
| @RegisterExtension | ||
| public final LoggerAuditingExtension loggerExtension = | ||
| new LoggerAuditingExtension(CanalJsonDeserializationSchema.class, DEBUG); | ||
|
|
||
| private static final DataType PHYSICAL_DATA_TYPE = | ||
| ROW( | ||
| FIELD("id", INT().notNull()), | ||
|
|
@@ -105,6 +112,57 @@ void testIgnoreParseErrors() throws Exception { | |
| "An error occurred while collecting data.")); | ||
| } | ||
|
|
||
| @Test | ||
| void testIgnoreParseErrorsLogsDebugMessage() throws Exception { | ||
|
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. could you add a test case for fail on missing as well please. This also comes out of |
||
| String corruptMessage = | ||
| "{\"data\":null,\"old\":null,\"type\":\"UNKNOWN_TYPE\",\"database\":\"test\",\"table\":\"test\"}"; | ||
| CanalJsonDeserializationSchema deserializationSchema = | ||
| CanalJsonDeserializationSchema.builder( | ||
| PHYSICAL_DATA_TYPE, | ||
| Collections.emptyList(), | ||
| InternalTypeInfo.of(PHYSICAL_DATA_TYPE.getLogicalType())) | ||
| .setIgnoreParseErrors(true) | ||
| .build(); | ||
| open(deserializationSchema); | ||
|
|
||
| SimpleCollector collector = new SimpleCollector(); | ||
| deserializationSchema.deserialize( | ||
| corruptMessage.getBytes(StandardCharsets.UTF_8), collector); | ||
|
|
||
| // Verify no records were collected | ||
| assertThat(collector.list).isEmpty(); | ||
|
|
||
| // Verify the error was logged at DEBUG level | ||
| assertThat(loggerExtension.getMessages()) | ||
| .anyMatch( | ||
| msg -> | ||
| msg.contains("Unknown \"type\" value") | ||
| || msg.contains("Corrupt Canal JSON")); | ||
| } | ||
|
|
||
| @Test | ||
| void testDeserializationThrowsExceptionWhenIgnoreParseErrorsIsFalse() { | ||
| String corruptMessage = "{\"type\":\"INVALID\"}"; | ||
|
|
||
| CanalJsonDeserializationSchema deserializationSchema = | ||
| CanalJsonDeserializationSchema.builder( | ||
| PHYSICAL_DATA_TYPE, | ||
| Collections.emptyList(), | ||
| InternalTypeInfo.of(PHYSICAL_DATA_TYPE.getLogicalType())) | ||
| .setIgnoreParseErrors(false) | ||
| .build(); | ||
| open(deserializationSchema); | ||
|
|
||
| SimpleCollector collector = new SimpleCollector(); | ||
|
|
||
| assertThatThrownBy( | ||
| () -> | ||
| deserializationSchema.deserialize( | ||
| corruptMessage.getBytes(StandardCharsets.UTF_8), collector)) | ||
| .isInstanceOf(IOException.class) | ||
| .hasMessageContaining("Unknown \"type\" value"); | ||
| } | ||
|
|
||
| @Test | ||
| void testDeserializeNullRow() throws Exception { | ||
| final List<ReadableMetadata> requestedMetadata = Arrays.asList(ReadableMetadata.values()); | ||
|
|
||
Uh oh!
There was an error while loading. Please reload this page.