@ -16,12 +16,17 @@
package com.ververica.cdc.connectors.mysql.debezium.task.context ;
import com.ververica.cdc.connectors.mysql.debezium.task.context.exception.SchemaOutOfSyncException ;
import com.ververica.cdc.connectors.mysql.source.config.MySqlSourceConfig ;
import com.ververica.cdc.connectors.mysql.table.StartupMode ;
import io.debezium.DebeziumException ;
import io.debezium.connector.base.ChangeEventQueue ;
import io.debezium.connector.mysql.MySqlConnector ;
import io.debezium.connector.mysql.MySqlTaskContext ;
import io.debezium.pipeline.ErrorHandler ;
import io.debezium.relational.TableId ;
import org.apache.commons.lang3.exception.ExceptionUtils ;
import org.apache.kafka.connect.errors.ConnectException ;
import org.slf4j.Logger ;
import org.slf4j.LoggerFactory ;
@ -38,12 +43,17 @@ public class MySqlErrorHandler extends ErrorHandler {
Pattern . compile (
"Encountered change event for table (.+)\\.(.+) whose schema isn't known to this connector" ) ;
MySqlTaskContext context ;
private final MySqlTaskContext context ;
private final MySqlSourceConfig sourceConfig ;
public MySqlErrorHandler (
String logicalName , ChangeEventQueue < ? > queue , MySqlTaskContext context ) {
String logicalName ,
ChangeEventQueue < ? > queue ,
MySqlTaskContext context ,
MySqlSourceConfig sourceConfig ) {
super ( MySqlConnector . class , logicalName , queue ) ;
this . context = context ;
this . sourceConfig = sourceConfig ;
}
@Override
@ -53,20 +63,56 @@ public class MySqlErrorHandler extends ErrorHandler {
@Override
public void setProducerThrowable ( Throwable producerThrowable ) {
if ( producerThrowable . getCause ( ) instanceof DebeziumException ) {
DebeziumException e = ( DebeziumException ) producerThrowable . getCause ( ) ;
String detailMessage = e . getMessage ( ) ;
Matcher matcher = NOT_FOUND_TABLE_MSG_PATTERN . matcher ( detailMessage ) ;
if ( matcher . find ( ) ) {
String databaseName = matcher . group ( 1 ) ;
String tableName = matcher . group ( 2 ) ;
TableId tableId = new TableId ( databaseName , null , tableName ) ;
if ( context . getSchema ( ) . schemaFor ( tableId ) = = null ) {
LOG . warn ( "Schema for table " + tableId + " is null" ) ;
return ;
}
if ( isTableNotFoundException ( producerThrowable ) ) {
Matcher matcher =
NOT_FOUND_TABLE_MSG_PATTERN . matcher ( producerThrowable . getCause ( ) . getMessage ( ) ) ;
String databaseName = matcher . group ( 1 ) ;
String tableName = matcher . group ( 2 ) ;
TableId tableId = new TableId ( databaseName , null , tableName ) ;
if ( context . getSchema ( ) . schemaFor ( tableId ) = = null ) {
LOG . warn ( "Schema for table " + tableId + " is null" ) ;
return ;
}
}
if ( isSchemaOutOfSyncException ( producerThrowable ) ) {
super . setProducerThrowable (
new SchemaOutOfSyncException (
"Internal schema representation is probably out of sync with real database schema. "
+ "The reason could be that the table schema was changed after the starting "
+ "binlog offset, which is not supported when startup mode is set to "
+ sourceConfig . getStartupOptions ( ) . startupMode ,
producerThrowable ) ) ;
return ;
}
super . setProducerThrowable ( producerThrowable ) ;
}
private boolean isTableNotFoundException ( Throwable t ) {
if ( ! ( t . getCause ( ) instanceof DebeziumException ) ) {
return false ;
}
DebeziumException e = ( DebeziumException ) t . getCause ( ) ;
String detailMessage = e . getMessage ( ) ;
Matcher matcher = NOT_FOUND_TABLE_MSG_PATTERN . matcher ( detailMessage ) ;
return matcher . find ( ) ;
}
private boolean isSchemaOutOfSyncException ( Throwable t ) {
Throwable rootCause = ExceptionUtils . getRootCause ( t ) ;
return rootCause instanceof ConnectException
& & rootCause
. getMessage ( )
. endsWith (
"internal schema representation is probably out of sync with real database schema" )
& & isSettingStartingOffset ( ) ;
}
private boolean isSettingStartingOffset ( ) {
StartupMode startupMode = sourceConfig . getStartupOptions ( ) . startupMode ;
return startupMode = = StartupMode . EARLIEST_OFFSET
| | startupMode = = StartupMode . TIMESTAMP
| | startupMode = = StartupMode . SPECIFIC_OFFSETS ;
}
}