diff --git a/CHANGELOG.md b/CHANGELOG.md
index ff7f2611..2a8bb663 100644
--- a/CHANGELOG.md
+++ b/CHANGELOG.md
@@ -1,4 +1,7 @@
## Release History
+### 1.15.0 (2024-04-18)
+#### Key Bug Fixes
+* Fixed an issue where using `CosmosDBSinkConnector` in bulk mode failed to write items for container with nested partition key path - [PR 565](https://github.com/microsoft/kafka-connect-cosmosdb/pull/565)
### 1.14.2 (2024-03-12)
#### Key Bug Fixes
diff --git a/pom.xml b/pom.xml
index bd35896f..b921ee85 100644
--- a/pom.xml
+++ b/pom.xml
@@ -7,7 +7,7 @@
com.azure.cosmos.kafka
kafka-connect-cosmos
- 1.14.2
+ 1.15.0
kafka-connect-cosmos
https://github.com/microsoft/kafka-connect-cosmosdb
@@ -48,7 +48,7 @@
com.azure
azure-cosmos
- 4.56.0
+ 4.58.0
com.jayway.jsonpath
diff --git a/src/main/java/com/azure/cosmos/kafka/connect/sink/BulkWriter.java b/src/main/java/com/azure/cosmos/kafka/connect/sink/BulkWriter.java
index 5088e703..6fb96be7 100644
--- a/src/main/java/com/azure/cosmos/kafka/connect/sink/BulkWriter.java
+++ b/src/main/java/com/azure/cosmos/kafka/connect/sink/BulkWriter.java
@@ -22,10 +22,10 @@
import java.util.ArrayList;
import java.util.Collections;
+import java.util.LinkedHashMap;
import java.util.List;
import java.util.Map;
import java.util.Objects;
-import java.util.LinkedHashMap;
import static com.azure.cosmos.implementation.guava25.base.Preconditions.checkArgument;
import static com.azure.cosmos.implementation.guava25.base.Preconditions.checkNotNull;
@@ -120,16 +120,8 @@ protected SinkWriteResponse writeCore(List sinkRecords) {
private PartitionKey getPartitionKeyValue(Object recordValue) {
checkArgument(recordValue instanceof Map, "Argument 'recordValue' is not valid map format.");
- //TODO: examine the code here for sub-partition
- String partitionKeyPath = StringUtils.join(this.partitionKeyDefinition.getPaths(), "");
Map recordMap = (Map) recordValue;
- Object partitionKeyValue = recordMap.get(partitionKeyPath.substring(1));
- PartitionKeyInternal partitionKeyInternal = PartitionKeyInternal.fromObjectArray(Collections.singletonList(partitionKeyValue), false);
-
- return ImplementationBridgeHelpers
- .PartitionKeyHelper
- .getPartitionKeyAccessor()
- .toPartitionKey(partitionKeyInternal);
+ return PartitionKey.fromItem(recordMap, this.partitionKeyDefinition);
}
BulkOperationFailedException handleErrorStatusCode(
diff --git a/src/test/java/com/azure/cosmos/kafka/connect/sink/BulkWriterTests.java b/src/test/java/com/azure/cosmos/kafka/connect/sink/BulkWriterTests.java
index 7ed57a90..529927c4 100644
--- a/src/test/java/com/azure/cosmos/kafka/connect/sink/BulkWriterTests.java
+++ b/src/test/java/com/azure/cosmos/kafka/connect/sink/BulkWriterTests.java
@@ -8,6 +8,7 @@
import com.azure.cosmos.implementation.BadRequestException;
import com.azure.cosmos.implementation.HttpConstants;
import com.azure.cosmos.implementation.RequestTimeoutException;
+import com.azure.cosmos.implementation.Utils;
import com.azure.cosmos.models.CosmosBulkItemResponse;
import com.azure.cosmos.models.CosmosBulkOperationResponse;
import com.azure.cosmos.models.CosmosBulkOperations;
@@ -16,6 +17,9 @@
import com.azure.cosmos.models.CosmosItemOperation;
import com.azure.cosmos.models.PartitionKey;
import com.azure.cosmos.models.PartitionKeyDefinition;
+import com.azure.cosmos.models.PartitionKind;
+import com.fasterxml.jackson.core.type.TypeReference;
+import com.fasterxml.jackson.databind.node.ObjectNode;
import org.apache.kafka.common.record.TimestampType;
import org.apache.kafka.connect.data.ConnectSchema;
import org.apache.kafka.connect.data.Schema;
@@ -31,11 +35,13 @@
import java.util.Iterator;
import java.util.List;
import java.util.Map;
-import java.util.UUID;
import java.util.Random;
+import java.util.UUID;
import java.util.concurrent.atomic.AtomicInteger;
+import static junit.framework.Assert.assertNotNull;
import static junit.framework.TestCase.assertEquals;
+import static junit.framework.TestCase.assertFalse;
import static junit.framework.TestCase.assertTrue;
import static org.mockito.ArgumentMatchers.any;
import static org.mockito.Mockito.times;
@@ -59,6 +65,7 @@ public void setup(){
PartitionKeyDefinition mockedPartitionKeyDefinition = Mockito.mock(PartitionKeyDefinition.class);
Mockito.when(mockedContainerProperties.getPartitionKeyDefinition()).thenReturn(mockedPartitionKeyDefinition);
Mockito.when(mockedPartitionKeyDefinition.getPaths()).thenReturn(Arrays.asList("/id"));
+ Mockito.when(mockedPartitionKeyDefinition.getKind()).thenReturn(PartitionKind.HASH);
bulkWriter = new BulkWriter(container, MAX_RETRY_COUNT, COMPRESSION_ENABLED);
}
@@ -207,6 +214,65 @@ public void testBulkWriteFailedWithTransientException() {
assertEquals(HttpConstants.StatusCodes.REQUEST_TIMEOUT, ((CosmosException)response.getFailedRecordResponses().get(0).getException()).getStatusCode());
}
+ @Test
+ public void testBulkWriteForContainerWithNestedPartitionKey() {
+ CosmosContainer containerWithNestedPartitionKey = Mockito.mock(CosmosContainer.class);
+
+ CosmosContainerResponse mockedContainerResponse = Mockito.mock(CosmosContainerResponse.class);
+ Mockito.when(containerWithNestedPartitionKey.read()).thenReturn(mockedContainerResponse);
+ CosmosContainerProperties mockedContainerProperties = Mockito.mock(CosmosContainerProperties.class);
+ Mockito.when(mockedContainerResponse.getProperties()).thenReturn(mockedContainerProperties);
+ PartitionKeyDefinition mockedPartitionKeyDefinition = Mockito.mock(PartitionKeyDefinition.class);
+ Mockito.when(mockedContainerProperties.getPartitionKeyDefinition()).thenReturn(mockedPartitionKeyDefinition);
+ Mockito.when(mockedPartitionKeyDefinition.getPaths()).thenReturn(Arrays.asList("/location/city/zipCode"));
+ Mockito.when(mockedPartitionKeyDefinition.getKind()).thenReturn(PartitionKind.HASH);
+
+ BulkWriter testWriter = new BulkWriter(containerWithNestedPartitionKey, MAX_RETRY_COUNT, COMPRESSION_ENABLED);
+
+ String itemId = UUID.randomUUID().toString();
+ String pkValue = "1234";
+
+ ObjectNode objectNode = Utils.getSimpleObjectMapper().createObjectNode();
+ objectNode.put("id", itemId);
+
+ ObjectNode locationNode = Utils.getSimpleObjectMapper().createObjectNode();
+ ObjectNode cityNode = Utils.getSimpleObjectMapper().createObjectNode();
+ cityNode.put("zipCode", pkValue);
+ locationNode.put("city", cityNode);
+ objectNode.put("location", locationNode);
+
+ SinkRecord sinkRecord =
+ new SinkRecord(
+ TOPIC_NAME,
+ 1,
+ new ConnectSchema(org.apache.kafka.connect.data.Schema.Type.STRING),
+ objectNode.get("id"),
+ new ConnectSchema(org.apache.kafka.connect.data.Schema.Type.MAP),
+ Utils.getSimpleObjectMapper().convertValue(objectNode, new TypeReference