Skip to content

KAFKA-438: Create DeleteOneTombstoneBusinessKeyStrategy.java #175

New issue

Have a question about this project? # for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “#”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? # to your account

Merged
merged 6 commits into from
Apr 11, 2025
Merged
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
@@ -0,0 +1,61 @@
/*
* Copyright 2008-present MongoDB, Inc.
*
* Licensed under the Apache License, Version 2.0 (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.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/

package com.mongodb.kafka.connect.sink.writemodel.strategy;

import static com.mongodb.kafka.connect.sink.writemodel.strategy.WriteModelHelper.flattenKeys;

import org.apache.kafka.connect.errors.ConnectException;
import org.apache.kafka.connect.errors.DataException;

import org.bson.BsonDocument;

import com.mongodb.client.model.DeleteOneModel;
import com.mongodb.client.model.WriteModel;

import com.mongodb.kafka.connect.sink.Configurable;
import com.mongodb.kafka.connect.sink.MongoSinkTopicConfig;
import com.mongodb.kafka.connect.sink.converter.SinkDocument;
import com.mongodb.kafka.connect.sink.processor.id.strategy.IdStrategy;
import com.mongodb.kafka.connect.sink.processor.id.strategy.PartialKeyStrategy;

public class DeleteOneTombstoneBusinessKeyStrategy implements WriteModelStrategy, Configurable {
Copy link
Collaborator Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I ended up opting to just create a new write model instead of updating DeleteOneBusinessKeyStrategy, the risk would be if we add the ability to start deleting on key values when tombstone events are sent, documents might start being deleted when they weren't in previous versions.

private IdStrategy idStrategy;

@Override
public WriteModel<BsonDocument> createWriteModel(final SinkDocument document) {
document
.getKeyDoc()
.orElseThrow(
() ->
new DataException(
"Could not build the WriteModel,the key document was missing unexpectedly"));

if (!(idStrategy instanceof PartialKeyStrategy)) {
throw new ConnectException(
"DeleteOneTombstoneBusinessKeyStrategy expects PartialKeyStrategy to be defined");
}

BsonDocument businessKey = idStrategy.generateId(document, null).asDocument();
businessKey = flattenKeys(businessKey);
return new DeleteOneModel<>(businessKey);
}

@Override
public void configure(final MongoSinkTopicConfig configuration) {
idStrategy = configuration.getIdStrategy();
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -108,6 +108,7 @@
import com.mongodb.kafka.connect.sink.writemodel.strategy.DefaultWriteModelStrategy;
import com.mongodb.kafka.connect.sink.writemodel.strategy.DeleteOneBusinessKeyStrategy;
import com.mongodb.kafka.connect.sink.writemodel.strategy.DeleteOneDefaultStrategy;
import com.mongodb.kafka.connect.sink.writemodel.strategy.DeleteOneTombstoneBusinessKeyStrategy;
import com.mongodb.kafka.connect.sink.writemodel.strategy.InsertOneDefaultStrategy;
import com.mongodb.kafka.connect.sink.writemodel.strategy.ReplaceOneBusinessKeyStrategy;
import com.mongodb.kafka.connect.sink.writemodel.strategy.ReplaceOneDefaultStrategy;
Expand Down Expand Up @@ -712,6 +713,9 @@ Collection<DynamicTest> testGetSingleValidWriteModelStrategy() {
UpdateOneBusinessKeyTimestampStrategy.class.getName(),
UpdateOneBusinessKeyTimestampStrategy.class);
put(DeleteOneBusinessKeyStrategy.class.getName(), DeleteOneBusinessKeyStrategy.class);
put(
DeleteOneTombstoneBusinessKeyStrategy.class.getName(),
DeleteOneTombstoneBusinessKeyStrategy.class);
}
};

Expand Down Expand Up @@ -796,6 +800,9 @@ Collection<DynamicTest> testGetSingleValidDeleteWriteModelStrategy() {
UpdateOneBusinessKeyTimestampStrategy.class.getName(),
UpdateOneBusinessKeyTimestampStrategy.class);
put(DeleteOneBusinessKeyStrategy.class.getName(), DeleteOneBusinessKeyStrategy.class);
put(
DeleteOneTombstoneBusinessKeyStrategy.class.getName(),
DeleteOneTombstoneBusinessKeyStrategy.class);
}
};

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -29,6 +29,7 @@

import java.util.Map;

import org.apache.kafka.connect.errors.ConnectException;
import org.apache.kafka.connect.errors.DataException;
import org.junit.jupiter.api.DisplayName;
import org.junit.jupiter.api.Test;
Expand Down Expand Up @@ -68,6 +69,10 @@ class WriteModelStrategyTest {
private static final DeleteOneBusinessKeyStrategy DELETE_ONE_BUSINESS_KEY_STRATEGY =
new DeleteOneBusinessKeyStrategy();
private static final DeleteOneBusinessKeyStrategy DELETE_ONE_BUSINESS_KEY_PARTIAL_STRATEGY;
private static final DeleteOneTombstoneBusinessKeyStrategy
DELETE_ONE_TOMBSTONE_BUSINESS_KEY_STRATEGY = new DeleteOneTombstoneBusinessKeyStrategy();
private static final DeleteOneTombstoneBusinessKeyStrategy
DELETE_ONE_TOMBSTONE_BUSINESS_KEY_PARTIAL_STRATEGY;
private static final SinkDocument SINK_DOCUMENT_NULL_VALUE =
new SinkDocument(new BsonDocument(), null);
private static final SinkDocument SINK_DOCUMENT_NULL_KEY =
Expand All @@ -81,6 +86,9 @@ class WriteModelStrategyTest {
MongoSinkTopicConfig.DOCUMENT_ID_STRATEGY_CONFIG, PartialKeyStrategy.class.getName());
configMap.put(
MongoSinkTopicConfig.DOCUMENT_ID_STRATEGY_PARTIAL_KEY_PROJECTION_TYPE_CONFIG, "AllowList");
configMap.put(
MongoSinkTopicConfig.DOCUMENT_ID_STRATEGY_PARTIAL_KEY_PROJECTION_LIST_CONFIG,
"a.a1,b.b1,b.b2");

MongoSinkTopicConfig partialKeyConfig =
new MongoSinkConfig(configMap).getMongoSinkTopicConfig(TEST_TOPIC);
Expand All @@ -93,6 +101,9 @@ class WriteModelStrategyTest {
UPDATE_ONE_BUSINESS_KEY_TIMESTAMPS_PARTIAL_STRATEGY.configure(partialKeyConfig);
DELETE_ONE_BUSINESS_KEY_PARTIAL_STRATEGY = new DeleteOneBusinessKeyStrategy();
DELETE_ONE_BUSINESS_KEY_PARTIAL_STRATEGY.configure(partialKeyConfig);
DELETE_ONE_TOMBSTONE_BUSINESS_KEY_PARTIAL_STRATEGY =
new DeleteOneTombstoneBusinessKeyStrategy();
DELETE_ONE_TOMBSTONE_BUSINESS_KEY_PARTIAL_STRATEGY.configure(partialKeyConfig);
}

private static final BsonDocument VALUE_DOC =
Expand Down Expand Up @@ -379,7 +390,7 @@ void testUpdateOneBusinessKeyTimestampsStrategyPartialWithValidSinkDocument() {

@Test
@DisplayName(
"when sink document is valid for UpdateOneBusinessKeyTimestampStrategy then correct UpdateOneModel")
"when sink document is valid for DeleteOneBusinessKeyTimestampStrategy then correct DeleteOneModel")
void testDeleteOneBusinessKeyStrategyWithValidSinkDocument() {
WriteModel<BsonDocument> result =
DELETE_ONE_BUSINESS_KEY_STRATEGY.createWriteModel(
Expand All @@ -392,7 +403,7 @@ void testDeleteOneBusinessKeyStrategyWithValidSinkDocument() {

@Test
@DisplayName(
"when sink document is valid for UpdateOneBusinessKeyTimestampStrategy with partial id strategy then correct UpdateOneModel")
"when sink document is valid for DeleteOneBusinessKeyTimestampStrategy with partial id strategy then correct DeleteOneModel")
void testDeleteOneBusinessKeyStrategyStrategyPartialWithValidSinkDocument() {
WriteModel<BsonDocument> result =
DELETE_ONE_BUSINESS_KEY_PARTIAL_STRATEGY.createWriteModel(
Expand All @@ -403,6 +414,22 @@ void testDeleteOneBusinessKeyStrategyStrategyPartialWithValidSinkDocument() {
assertEquals(BUSINESS_KEY_FLATTENED_FILTER, writeModel.getFilter());
}

@Test
@DisplayName(
"when sink document is valid for DeleteOneTombstoneBusinessKeyTimestampStrategy with partial id strategy then correct DeleteOneModel")
void testDeleteOneTombstoneBusinessKeyStrategyStrategyPartialWithValidSinkDocument() {
BsonDocument keyDoc =
BsonDocument.parse(
"{_id: {a: {a1: 0}, b: {b1: 0, b2: 0}}, a: {a1: 0}, b: {b1: 0, b2: 0, c1: 0}}");
WriteModel<BsonDocument> result =
DELETE_ONE_TOMBSTONE_BUSINESS_KEY_PARTIAL_STRATEGY.createWriteModel(
new SinkDocument(keyDoc, null));
assertTrue(result instanceof DeleteOneModel, "result expected to be of type DeleteOneModel");

DeleteOneModel<BsonDocument> writeModel = (DeleteOneModel<BsonDocument>) result;
assertEquals(BsonDocument.parse("{'a.a1': 0, 'b.b1': 0, 'b.b2': 0}"), writeModel.getFilter());
}

@Test
@DisplayName("Test handling empty or missing sink document data")
void testIEmptyOrMissingSinkDocumentData() {
Expand Down Expand Up @@ -493,7 +520,24 @@ void testIEmptyOrMissingSinkDocumentData() {
assertThrows(
DataException.class,
() ->
DELETE_ONE_BUSINESS_KEY_PARTIAL_STRATEGY.createWriteModel(
SINK_DOCUMENT_EMPTY)));
DELETE_ONE_BUSINESS_KEY_PARTIAL_STRATEGY.createWriteModel(SINK_DOCUMENT_EMPTY)),
() ->
assertThrows(
DataException.class,
() ->
DELETE_ONE_TOMBSTONE_BUSINESS_KEY_STRATEGY.createWriteModel(
SINK_DOCUMENT_NULL_KEY)),
() ->
assertThrows(
ConnectException.class,
() ->
DELETE_ONE_TOMBSTONE_BUSINESS_KEY_STRATEGY.createWriteModel(
SINK_DOCUMENT_EMPTY)),
() ->
assertThrows(
DataException.class,
() ->
DELETE_ONE_TOMBSTONE_BUSINESS_KEY_PARTIAL_STRATEGY.createWriteModel(
SINK_DOCUMENT_NULL_KEY)));
}
}