[
https://issues.apache.org/jira/browse/KAFKA-7745?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
]
Andrey Pustovetov updated KAFKA-7745:
-------------------------------------
Description:
*Test case*
# Create a connector:
{noformat}
curl -i -XPOST -H "Accept:application/json" -H "Content-Type:application/json"
http://localhost:8083/connectors/ -d'
{
"name": "postgres-connector",
"config": {
"connector.class": "io.debezium.connector.postgresql.PostgresConnector",
"database.hostname": "postgresql",
"database.port": "5432",
"database.user": "postgres",
"database.password": "postgres",
"database.dbname": "postgres",
"database.server.name": "test",
"slot.drop_on_stop": "true"
}
}'
{noformat}
# Delete the connector:
{noformat}
curl -XDELETE http://localhost:8083/connectors/postgres-connector
{noformat}
# Check your config topic of Kafka Connect:
{noformat}
./bin/kafka-console-consumer.sh --bootstrap-server kafka:9092 --topic
my-connect-config --from-beginning --property print.key=true
{noformat}
AR:
The following records are read from the config topic after the connector is
created:
||Key||Value||
|connector-postgres-connector|{{\{"properties":\{"connector.class":"io.debezium.connector.postgresql.PostgresConnector","database.hostname":"postgresql","database.port":"5432","database.user":"postgres","database.password":"postgres","database.dbname":"postgres","database.server.name":"test","slot.drop_on_stop":"true","name":"postgres-connector"\}\}}}|
|task-postgres-connector-0|{{\{"properties":\{"connector.class":"io.debezium.connector.postgresql.PostgresConnector","slot.drop_on_stop":"true","database.user":"postgres","database.dbname":"postgres","task.class":"io.debezium.connector.postgresql.PostgresConnectorTask","database.hostname":"postgresql","database.password":"postgres","name":"postgres-connector","database.server.name":"test","database.port":"5432"\}\}}}|
|commit-postgres-connector|{{\{"tasks":1\}}}|
The following records are read from the config topic after the connector is
deleted:
||Key||Value||
|connector-postgres-connector|{{null}}|
|target-state-postgres-connector|{{null}}|
Note that the tombstone record is missing for the {{task-postgres-connector-0}}
key.
Additionally, the following fields of
{{org.apache.kafka.connect.storage.KafkaConfigBackingStore}} class are never
cleared during connector removal:
* {{connectorTaskCounts}}
* {{taskConfigs
* deferredTaskUpdates
and can lead to memory leaks.
was:
*Test case*
# Create a connector:
{noformat}
curl -i -XPOST -H "Accept:application/json" -H "Content-Type:application/json"
http://localhost:8083/connectors/ -d'
{
"name": "postgres-connector",
"config": {
"connector.class": "io.debezium.connector.postgresql.PostgresConnector",
"database.hostname": "postgresql",
"database.port": "5432",
"database.user": "postgres",
"database.password": "postgres",
"database.dbname": "postgres",
"database.server.name": "test",
"slot.drop_on_stop": "true"
}
}'
{noformat}
# Delete the connector:
{noformat}
curl -XDELETE http://localhost:8083/connectors/postgres-connector
{noformat}
# Check your config topic:
{noformat}
./bin/kafka-console-consumer.sh --bootstrap-server kafka:9092 --topic
my-connect-config --from-beginning --property print.key=true
{noformat}
AR:
The following records are read from the config topic after the connector is
created:
||Key||Value||
|connector-postgres-connector|{{\{"properties":\{"connector.class":"io.debezium.connector.postgresql.PostgresConnector","database.hostname":"postgresql","database.port":"5432","database.user":"postgres","database.password":"postgres","database.dbname":"postgres","database.server.name":"test","slot.drop_on_stop":"true","name":"postgres-connector"\}\}}}|
|task-postgres-connector-0|{{\{"properties":\{"connector.class":"io.debezium.connector.postgresql.PostgresConnector","slot.drop_on_stop":"true","database.user":"postgres","database.dbname":"postgres","task.class":"io.debezium.connector.postgresql.PostgresConnectorTask","database.hostname":"postgresql","database.password":"postgres","name":"postgres-connector","database.server.name":"test","database.port":"5432"\}\}}}|
|commit-postgres-connector|{{\{"tasks":1\}}}|
The following records are read from the config topic after the connector is
deleted:
||Key||Value||
|connector-postgres-connector|{{null}}|
|target-state-postgres-connector|{{null}}|
Note that the tombstone record is missing for the {{task-postgres-connector-0}}
key.
Additionally, the following fields of
{{org.apache.kafka.connect.storage.KafkaConfigBackingStore}} class are never
cleared during connector removal:
* {{connectorTaskCounts}}
* {{taskConfigs
* deferredTaskUpdates
and can lead to memory leaks.
> Kafka Connect doesn't create tombstone record for tasks of deleted connector
> ----------------------------------------------------------------------------
>
> Key: KAFKA-7745
> URL: https://issues.apache.org/jira/browse/KAFKA-7745
> Project: Kafka
> Issue Type: Bug
> Components: KafkaConnect
> Affects Versions: 1.1.1
> Reporter: Andrey Pustovetov
> Priority: Critical
>
> *Test case*
> # Create a connector:
> {noformat}
> curl -i -XPOST -H "Accept:application/json" -H
> "Content-Type:application/json" http://localhost:8083/connectors/ -d'
> {
> "name": "postgres-connector",
> "config": {
> "connector.class": "io.debezium.connector.postgresql.PostgresConnector",
> "database.hostname": "postgresql",
> "database.port": "5432",
> "database.user": "postgres",
> "database.password": "postgres",
> "database.dbname": "postgres",
> "database.server.name": "test",
> "slot.drop_on_stop": "true"
> }
> }'
> {noformat}
> # Delete the connector:
> {noformat}
> curl -XDELETE http://localhost:8083/connectors/postgres-connector
> {noformat}
> # Check your config topic of Kafka Connect:
> {noformat}
> ./bin/kafka-console-consumer.sh --bootstrap-server kafka:9092 --topic
> my-connect-config --from-beginning --property print.key=true
> {noformat}
> AR:
> The following records are read from the config topic after the connector is
> created:
> ||Key||Value||
> |connector-postgres-connector|{{\{"properties":\{"connector.class":"io.debezium.connector.postgresql.PostgresConnector","database.hostname":"postgresql","database.port":"5432","database.user":"postgres","database.password":"postgres","database.dbname":"postgres","database.server.name":"test","slot.drop_on_stop":"true","name":"postgres-connector"\}\}}}|
> |task-postgres-connector-0|{{\{"properties":\{"connector.class":"io.debezium.connector.postgresql.PostgresConnector","slot.drop_on_stop":"true","database.user":"postgres","database.dbname":"postgres","task.class":"io.debezium.connector.postgresql.PostgresConnectorTask","database.hostname":"postgresql","database.password":"postgres","name":"postgres-connector","database.server.name":"test","database.port":"5432"\}\}}}|
> |commit-postgres-connector|{{\{"tasks":1\}}}|
> The following records are read from the config topic after the connector is
> deleted:
> ||Key||Value||
> |connector-postgres-connector|{{null}}|
> |target-state-postgres-connector|{{null}}|
> Note that the tombstone record is missing for the
> {{task-postgres-connector-0}} key.
> Additionally, the following fields of
> {{org.apache.kafka.connect.storage.KafkaConfigBackingStore}} class are never
> cleared during connector removal:
> * {{connectorTaskCounts}}
> * {{taskConfigs
> * deferredTaskUpdates
> and can lead to memory leaks.
--
This message was sent by Atlassian JIRA
(v7.6.3#76005)