From 31078773b786cafdca7814b19f7730a815590f54 Mon Sep 17 00:00:00 2001 From: emrehizal Date: Mon, 20 May 2024 16:48:45 +0200 Subject: [PATCH 01/14] test: add composite key test for node pattern --- .../kafka/sink/Neo4jNodePatternIT.kt | 136 ++++++++++++++++++ .../kafka/sink/SinkConfiguration.kt | 3 +- .../kafka/sink/strategy/NodePatternHandler.kt | 5 +- .../kafka/sink/strategy/PatternHandler.kt | 13 +- .../strategy/RelationshipPatternHandler.kt | 12 +- .../sink/strategy/NodePatternHandlerTest.kt | 20 ++- .../RelationshipPatternHandlerTest.kt | 18 +-- 7 files changed, 183 insertions(+), 24 deletions(-) create mode 100644 sink-connector/src/test/kotlin/org/neo4j/connectors/kafka/sink/Neo4jNodePatternIT.kt diff --git a/sink-connector/src/test/kotlin/org/neo4j/connectors/kafka/sink/Neo4jNodePatternIT.kt b/sink-connector/src/test/kotlin/org/neo4j/connectors/kafka/sink/Neo4jNodePatternIT.kt new file mode 100644 index 000000000..a5486c654 --- /dev/null +++ b/sink-connector/src/test/kotlin/org/neo4j/connectors/kafka/sink/Neo4jNodePatternIT.kt @@ -0,0 +1,136 @@ +/* + * Copyright (c) "Neo4j" + * Neo4j Sweden AB [https://neo4j.com] + * + * 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 org.neo4j.connectors.kafka.sink + +import io.kotest.assertions.nondeterministic.eventually +import io.kotest.matchers.should +import io.kotest.matchers.shouldBe +import kotlin.time.Duration.Companion.seconds +import org.apache.kafka.connect.data.Schema +import org.junit.jupiter.api.Test +import org.neo4j.connectors.kafka.testing.TestSupport.runTest +import org.neo4j.connectors.kafka.testing.kafka.ConvertingKafkaProducer +import org.neo4j.connectors.kafka.testing.sink.Neo4jSink +import org.neo4j.connectors.kafka.testing.sink.NodePatternStrategy +import org.neo4j.connectors.kafka.testing.sink.TopicProducer +import org.neo4j.driver.Session + +class Neo4jNodePatternIT { + companion object { + const val TOPIC = "test" + } + + @Neo4jSink(nodePattern = [NodePatternStrategy(TOPIC, "(:User{!id,name,surname})", false)]) + @Test + fun `should create node`( + @TopicProducer(TOPIC) producer: ConvertingKafkaProducer, + session: Session + ) = runTest { + producer.publish( + valueSchema = Schema.STRING_SCHEMA, + value = """{"id": 1, "name": "john", "surname": "doe"}""") + + eventually(30.seconds) { session.run("MATCH (n:User) RETURN n", emptyMap()).single() } + .get("n") + .asNode() should + { + it.labels() shouldBe listOf("User") + it.asMap() shouldBe mapOf("id" to 1L, "name" to "john", "surname" to "doe") + } + } + + @Neo4jSink( + nodePattern = + [ + NodePatternStrategy( + TOPIC, "(:User{!id: old_id,name: first_name,surname: last_name})", false)]) + @Test + fun `should create node with aliases`( + @TopicProducer(TOPIC) producer: ConvertingKafkaProducer, + session: Session + ) = runTest { + producer.publish( + valueSchema = Schema.STRING_SCHEMA, + value = """{"old_id": 1, "first_name": "john", "last_name": "doe"}""") + + eventually(30.seconds) { session.run("MATCH (n:User) RETURN n", emptyMap()).single() } + .get("n") + .asNode() should + { + it.labels() shouldBe listOf("User") + it.asMap() shouldBe mapOf("id" to 1L, "name" to "john", "surname" to "doe") + } + } + + @Neo4jSink( + nodePattern = + [ + NodePatternStrategy( + TOPIC, + "(:User{!id: __value.old_id,name: __key.first_name,surname: last_name})", + false)]) + @Test + fun `should create and delete node with aliases`( + @TopicProducer(TOPIC) producer: ConvertingKafkaProducer, + session: Session + ) = runTest { + producer.publish( + keySchema = Schema.STRING_SCHEMA, + key = """{"first_name": "john"}""", + valueSchema = Schema.STRING_SCHEMA, + value = """{"old_id": 1, "last_name": "doe"}""") + eventually(30.seconds) { session.run("MATCH (n:User) RETURN n", emptyMap()).single() } + .get("n") + .asNode() should + { + it.labels() shouldBe listOf("User") + it.asMap() shouldBe mapOf("id" to 1L, "name" to "john", "surname" to "doe") + } + + producer.publish( + keySchema = Schema.STRING_SCHEMA, + key = """{"old_id": 1}""", + ) + eventually(30.seconds) { + session + .run("MATCH (n:User) RETURN count(n) AS count", emptyMap()) + .single() + .get("count") + .asLong() shouldBe 0 + } + } + + @Neo4jSink(nodePattern = [NodePatternStrategy(TOPIC, "(:User{!id,!name,surname})", false)]) + @Test + fun `should create node with compositeKey`( + @TopicProducer(TOPIC) producer: ConvertingKafkaProducer, + session: Session + ) = runTest { + producer.publish( + keySchema = Schema.STRING_SCHEMA, + key = """{"id": 1, "name": "john"}""", + valueSchema = Schema.STRING_SCHEMA, + value = """{"surname": "doe"}""") + eventually(30.seconds) { session.run("MATCH (n:User) RETURN n", emptyMap()).single() } + .get("n") + .asNode() should + { + it.labels() shouldBe listOf("User") + it.asMap() shouldBe mapOf("id" to 1L, "name" to "john", "surname" to "doe") + } + } +} diff --git a/sink/src/main/kotlin/org/neo4j/connectors/kafka/sink/SinkConfiguration.kt b/sink/src/main/kotlin/org/neo4j/connectors/kafka/sink/SinkConfiguration.kt index 717478945..39deb0090 100644 --- a/sink/src/main/kotlin/org/neo4j/connectors/kafka/sink/SinkConfiguration.kt +++ b/sink/src/main/kotlin/org/neo4j/connectors/kafka/sink/SinkConfiguration.kt @@ -122,7 +122,8 @@ class SinkConfiguration(originals: Map) : ?: emptyList() init { - validateAllTopics(originals) + // TODO: reinstate and fix issue + // validateAllTopics(originals) } override fun userAgentComment(): String = diff --git a/sink/src/main/kotlin/org/neo4j/connectors/kafka/sink/strategy/NodePatternHandler.kt b/sink/src/main/kotlin/org/neo4j/connectors/kafka/sink/strategy/NodePatternHandler.kt index 08547f7d6..45227ba7f 100644 --- a/sink/src/main/kotlin/org/neo4j/connectors/kafka/sink/strategy/NodePatternHandler.kt +++ b/sink/src/main/kotlin/org/neo4j/connectors/kafka/sink/strategy/NodePatternHandler.kt @@ -71,7 +71,8 @@ class NodePatternHandler( val flattened = flattenMessage(it) val used = mutableSetOf() - val keys = extractKeys(pattern, flattened, used, bindValueAs, bindKeyAs) + val keys = + extractKeys(pattern, flattened, isTombstoneMessage, used, bindValueAs, bindKeyAs) val mapped = if (isTombstoneMessage) { listOf("D", mapOf("keys" to keys)) @@ -108,7 +109,7 @@ class NodePatternHandler( .named("n") return renderer.render( - Cypher.unwind(Cypher.parameter("messages")) + Cypher.unwind(Cypher.parameter("events")) .`as`(event) .call( Cypher.with(event) diff --git a/sink/src/main/kotlin/org/neo4j/connectors/kafka/sink/strategy/PatternHandler.kt b/sink/src/main/kotlin/org/neo4j/connectors/kafka/sink/strategy/PatternHandler.kt index b77ce8395..e2cf8e311 100644 --- a/sink/src/main/kotlin/org/neo4j/connectors/kafka/sink/strategy/PatternHandler.kt +++ b/sink/src/main/kotlin/org/neo4j/connectors/kafka/sink/strategy/PatternHandler.kt @@ -72,6 +72,7 @@ abstract class PatternHandler( protected fun extractKeys( pattern: Pattern, flattened: Map, + isTombstone: Boolean, usedTracker: MutableSet, vararg prefixes: String ): Map = @@ -79,8 +80,9 @@ abstract class PatternHandler( .associateBy { it.to } .mapValues { (_, mapping) -> if (isExplicit(mapping.from)) { - usedTracker += mapping.from - return@mapValues flattened[mapping.from] + val newKey = if (isTombstone) replaceValueWithKey(mapping.from) else mapping.from + usedTracker += newKey + return@mapValues flattened[newKey] } for (prefix in prefixes) { @@ -93,6 +95,13 @@ abstract class PatternHandler( } } + private fun replaceValueWithKey(mapping: String): String { + if (!mapping.startsWith("${bindValueAs}.")) { + return mapping + } + return mapping.replace("${bindValueAs}.", "${bindKeyAs}.") + } + /** * Extracts properties from flattened message properties, excluding previously used keys from the * computed properties. diff --git a/sink/src/main/kotlin/org/neo4j/connectors/kafka/sink/strategy/RelationshipPatternHandler.kt b/sink/src/main/kotlin/org/neo4j/connectors/kafka/sink/strategy/RelationshipPatternHandler.kt index 12a39dde6..3bd14bf6f 100644 --- a/sink/src/main/kotlin/org/neo4j/connectors/kafka/sink/strategy/RelationshipPatternHandler.kt +++ b/sink/src/main/kotlin/org/neo4j/connectors/kafka/sink/strategy/RelationshipPatternHandler.kt @@ -72,9 +72,13 @@ class RelationshipPatternHandler( val flattened = flattenMessage(it) val used = mutableSetOf() - val startKeys = extractKeys(pattern.start, flattened, used, bindValueAs, bindKeyAs) - val endKeys = extractKeys(pattern.end, flattened, used, bindValueAs, bindKeyAs) - val keys = extractKeys(pattern, flattened, used, bindValueAs, bindKeyAs) + val startKeys = + extractKeys( + pattern.start, flattened, isTombstoneMessage, used, bindValueAs, bindKeyAs) + val endKeys = + extractKeys(pattern.end, flattened, isTombstoneMessage, used, bindValueAs, bindKeyAs) + val keys = + extractKeys(pattern, flattened, isTombstoneMessage, used, bindValueAs, bindKeyAs) val mapped = if (isTombstoneMessage) { listOf( @@ -136,7 +140,7 @@ class RelationshipPatternHandler( .named("relationship") return renderer.render( - Cypher.unwind(Cypher.parameter("messages")) + Cypher.unwind(Cypher.parameter("events")) .`as`(event) .call( Cypher.with(event) diff --git a/sink/src/test/kotlin/org/neo4j/connectors/kafka/sink/strategy/NodePatternHandlerTest.kt b/sink/src/test/kotlin/org/neo4j/connectors/kafka/sink/strategy/NodePatternHandlerTest.kt index 2f8dd28d3..45809b510 100644 --- a/sink/src/test/kotlin/org/neo4j/connectors/kafka/sink/strategy/NodePatternHandlerTest.kt +++ b/sink/src/test/kotlin/org/neo4j/connectors/kafka/sink/strategy/NodePatternHandlerTest.kt @@ -46,7 +46,7 @@ class NodePatternHandlerTest : HandlerTest() { handler.query shouldBe CypherParser.parse( """ - UNWIND ${'$'}messages AS event + UNWIND ${'$'}events AS event CALL { WITH event WITH event WHERE event[0] = 'C' WITH event[1] AS event @@ -80,7 +80,7 @@ class NodePatternHandlerTest : HandlerTest() { handler.query shouldBe CypherParser.parse( """ - UNWIND ${'$'}messages AS event + UNWIND ${'$'}events AS event CALL { WITH event WITH event WHERE event[0] = 'C' WITH event[1] AS event @@ -114,7 +114,7 @@ class NodePatternHandlerTest : HandlerTest() { handler.query shouldBe CypherParser.parse( """ - UNWIND ${'$'}messages AS event + UNWIND ${'$'}events AS event CALL { WITH event WITH event WHERE event[0] = 'C' WITH event[1] AS event @@ -148,7 +148,7 @@ class NodePatternHandlerTest : HandlerTest() { handler.query shouldBe CypherParser.parse( """ - UNWIND ${'$'}messages AS event + UNWIND ${'$'}events AS event CALL { WITH event WITH event WHERE event[0] = 'C' WITH event[1] AS event @@ -183,7 +183,7 @@ class NodePatternHandlerTest : HandlerTest() { handler.query shouldBe CypherParser.parse( """ - UNWIND ${'$'}messages AS event + UNWIND ${'$'}events AS event CALL { WITH event WITH event WHERE event[0] = 'C' WITH event[1] AS event @@ -256,6 +256,14 @@ class NodePatternHandlerTest : HandlerTest() { "name" to "john", "surname" to "doe", "dob" to "2000-01-01"))))) } + @Test + fun `should remap key properties from message value fields to message key fields for tombstone messages`() { + assertQueryAndParameters( + "(:ALabel{!id: __value.old_id})", + key = """{"old_id": 1}""", + expected = listOf(listOf("D", mapOf("keys" to mapOf("id" to 1))))) + } + @Test fun `should include all properties with structs`() { val schema = @@ -629,7 +637,7 @@ class NodePatternHandlerTest : HandlerTest() { Query( CypherParser.parse( """ - UNWIND ${'$'}messages AS event + UNWIND ${'$'}events AS event CALL { WITH event WITH event WHERE event[0] = 'C' WITH event[1] AS event diff --git a/sink/src/test/kotlin/org/neo4j/connectors/kafka/sink/strategy/RelationshipPatternHandlerTest.kt b/sink/src/test/kotlin/org/neo4j/connectors/kafka/sink/strategy/RelationshipPatternHandlerTest.kt index 193cf21f5..c5daa41d0 100644 --- a/sink/src/test/kotlin/org/neo4j/connectors/kafka/sink/strategy/RelationshipPatternHandlerTest.kt +++ b/sink/src/test/kotlin/org/neo4j/connectors/kafka/sink/strategy/RelationshipPatternHandlerTest.kt @@ -44,7 +44,7 @@ class RelationshipPatternHandlerTest : HandlerTest() { handler.query shouldBe CypherParser.parse( """ - UNWIND ${'$'}messages AS event + UNWIND ${'$'}events AS event CALL { WITH event WITH event WHERE event[0] = 'C' WITH event[1] AS event @@ -83,7 +83,7 @@ class RelationshipPatternHandlerTest : HandlerTest() { handler.query shouldBe CypherParser.parse( """ - UNWIND ${'$'}messages AS event + UNWIND ${'$'}events AS event CALL { WITH event WITH event WHERE event[0] = 'C' WITH event[1] AS event @@ -122,7 +122,7 @@ class RelationshipPatternHandlerTest : HandlerTest() { handler.query shouldBe CypherParser.parse( """ - UNWIND ${'$'}messages AS event + UNWIND ${'$'}events AS event CALL { WITH event WITH event WHERE event[0] = 'C' WITH event[1] AS event @@ -161,7 +161,7 @@ class RelationshipPatternHandlerTest : HandlerTest() { handler.query shouldBe CypherParser.parse( """ - UNWIND ${'$'}messages AS event + UNWIND ${'$'}events AS event CALL { WITH event WITH event WHERE event[0] = 'C' WITH event[1] AS event @@ -202,7 +202,7 @@ class RelationshipPatternHandlerTest : HandlerTest() { handler.query shouldBe CypherParser.parse( """ - UNWIND ${'$'}messages AS event + UNWIND ${'$'}events AS event CALL { WITH event WITH event WHERE event[0] = 'C' WITH event[1] AS event @@ -242,7 +242,7 @@ class RelationshipPatternHandlerTest : HandlerTest() { handler.query shouldBe CypherParser.parse( """ - UNWIND ${'$'}messages AS event + UNWIND ${'$'}events AS event CALL { WITH event WITH event WHERE event[0] = 'C' WITH event[1] AS event @@ -284,7 +284,7 @@ class RelationshipPatternHandlerTest : HandlerTest() { handler.query shouldBe CypherParser.parse( """ - UNWIND ${'$'}messages AS event + UNWIND ${'$'}events AS event CALL { WITH event WITH event WHERE event[0] = 'C' WITH event[1] AS event @@ -323,7 +323,7 @@ class RelationshipPatternHandlerTest : HandlerTest() { handler.query shouldBe CypherParser.parse( """ - UNWIND ${'$'}messages AS event + UNWIND ${'$'}events AS event CALL { WITH event WITH event WHERE event[0] = 'C' WITH event[1] AS event @@ -829,7 +829,7 @@ class RelationshipPatternHandlerTest : HandlerTest() { Query( CypherParser.parse( """ - UNWIND ${'$'}messages AS event + UNWIND ${'$'}events AS event CALL { WITH event WITH event WHERE event[0] = 'C' WITH event[1] AS event From b1ac9a67144deb15a237c031ee1b6d7e28849277 Mon Sep 17 00:00:00 2001 From: emrehizal Date: Tue, 21 May 2024 10:44:45 +0200 Subject: [PATCH 02/14] test: add more integration tests for node pattern --- .../kafka/sink/Neo4jNodePatternIT.kt | 69 +++++++++++++++++++ 1 file changed, 69 insertions(+) diff --git a/sink-connector/src/test/kotlin/org/neo4j/connectors/kafka/sink/Neo4jNodePatternIT.kt b/sink-connector/src/test/kotlin/org/neo4j/connectors/kafka/sink/Neo4jNodePatternIT.kt index a5486c654..d37b3e7bd 100644 --- a/sink-connector/src/test/kotlin/org/neo4j/connectors/kafka/sink/Neo4jNodePatternIT.kt +++ b/sink-connector/src/test/kotlin/org/neo4j/connectors/kafka/sink/Neo4jNodePatternIT.kt @@ -19,6 +19,7 @@ package org.neo4j.connectors.kafka.sink import io.kotest.assertions.nondeterministic.eventually import io.kotest.matchers.should import io.kotest.matchers.shouldBe +import io.kotest.matchers.shouldNotBe import kotlin.time.Duration.Companion.seconds import org.apache.kafka.connect.data.Schema import org.junit.jupiter.api.Test @@ -133,4 +134,72 @@ class Neo4jNodePatternIT { it.asMap() shouldBe mapOf("id" to 1L, "name" to "john", "surname" to "doe") } } + + @Neo4jSink(nodePattern = [NodePatternStrategy(TOPIC, "(:User{!id})", false)]) + @Test + fun `should create node with nested properties`( + @TopicProducer(TOPIC) producer: ConvertingKafkaProducer, + session: Session + ) = runTest { + producer.publish( + valueSchema = Schema.STRING_SCHEMA, + value = + """{"id": 1, "name": "john", "surname": "doe", "address": {"city": "london", "country": "uk"}}""") + eventually(30.seconds) { session.run("MATCH (n:User) RETURN n", emptyMap()).single() } + .get("n") + .asNode() should + { + it.labels() shouldBe listOf("User") + it.asMap() shouldBe + mapOf( + "id" to 1L, + "name" to "john", + "surname" to "doe", + "address.city" to "london", + "address.country" to "uk") + } + } + + @Neo4jSink(nodePattern = [NodePatternStrategy(TOPIC, "(:User{!id, -name, -surname})", false)]) + @Test + fun `should create node with excluded properties`( + @TopicProducer(TOPIC) producer: ConvertingKafkaProducer, + session: Session + ) = runTest { + producer.publish( + valueSchema = Schema.STRING_SCHEMA, + value = + """{"id": 1, "name": "john", "surname": "doe", "dob": "2000-01-01", "address": {"city": "london", "country": "uk"}}""") + eventually(30.seconds) { session.run("MATCH (n:User) RETURN n", emptyMap()).single() } + .get("n") + .asNode() should + { + it.labels() shouldBe listOf("User") + it.asMap() shouldBe + mapOf( + "id" to 1L, + "dob" to "2000-01-01", + "address.city" to "london", + "address.country" to "uk") + } + } + + @Neo4jSink( + nodePattern = [NodePatternStrategy(TOPIC, "(:User{!id, created_at: __timestamp})", false)]) + @Test + fun `should create node with createdAt`( + @TopicProducer(TOPIC) producer: ConvertingKafkaProducer, + session: Session + ) = runTest { + producer.publish( + valueSchema = Schema.STRING_SCHEMA, + value = """{"id": 1, "name": "john", "surname": "doe"}""") + eventually(30.seconds) { session.run("MATCH (n:User) RETURN n", emptyMap()).single() } + .get("n") + .asNode() should + { + it.labels() shouldBe listOf("User") + it.asMap()["created_at"] shouldNotBe null + } + } } From 55a83b5d4daedae813a3cb9d65844d0cf634cfaa Mon Sep 17 00:00:00 2001 From: emrehizal Date: Wed, 22 May 2024 12:48:09 +0200 Subject: [PATCH 03/14] test: add composite key test for node pattern extended --- .../kafka/sink/Neo4jNodePatternIT.kt | 269 +++++++++++++++++- 1 file changed, 268 insertions(+), 1 deletion(-) diff --git a/sink-connector/src/test/kotlin/org/neo4j/connectors/kafka/sink/Neo4jNodePatternIT.kt b/sink-connector/src/test/kotlin/org/neo4j/connectors/kafka/sink/Neo4jNodePatternIT.kt index d37b3e7bd..65e4d507b 100644 --- a/sink-connector/src/test/kotlin/org/neo4j/connectors/kafka/sink/Neo4jNodePatternIT.kt +++ b/sink-connector/src/test/kotlin/org/neo4j/connectors/kafka/sink/Neo4jNodePatternIT.kt @@ -16,12 +16,15 @@ */ package org.neo4j.connectors.kafka.sink +import com.fasterxml.jackson.databind.ObjectMapper import io.kotest.assertions.nondeterministic.eventually import io.kotest.matchers.should import io.kotest.matchers.shouldBe import io.kotest.matchers.shouldNotBe import kotlin.time.Duration.Companion.seconds import org.apache.kafka.connect.data.Schema +import org.apache.kafka.connect.data.SchemaBuilder +import org.apache.kafka.connect.data.Struct import org.junit.jupiter.api.Test import org.neo4j.connectors.kafka.testing.TestSupport.runTest import org.neo4j.connectors.kafka.testing.kafka.ConvertingKafkaProducer @@ -33,11 +36,13 @@ import org.neo4j.driver.Session class Neo4jNodePatternIT { companion object { const val TOPIC = "test" + const val TOPIC_1 = "test-1" + const val TOPIC_2 = "test-2" } @Neo4jSink(nodePattern = [NodePatternStrategy(TOPIC, "(:User{!id,name,surname})", false)]) @Test - fun `should create node`( + fun `should create node from json string`( @TopicProducer(TOPIC) producer: ConvertingKafkaProducer, session: Session ) = runTest { @@ -54,6 +59,53 @@ class Neo4jNodePatternIT { } } + @Neo4jSink(nodePattern = [NodePatternStrategy(TOPIC, "(:User{!id,name,surname})", false)]) + @Test + fun `should create node from struct`( + @TopicProducer(TOPIC) producer: ConvertingKafkaProducer, + session: Session + ) = runTest { + SchemaBuilder.struct() + .field("id", Schema.INT64_SCHEMA) + .field("name", Schema.STRING_SCHEMA) + .field("surname", Schema.STRING_SCHEMA) + .build() + .let { schema -> + producer.publish( + valueSchema = schema, + value = Struct(schema).put("id", 1L).put("name", "john").put("surname", "doe")) + } + + eventually(30.seconds) { session.run("MATCH (n:User) RETURN n", emptyMap()).single() } + .get("n") + .asNode() should + { + it.labels() shouldBe listOf("User") + it.asMap() shouldBe mapOf("id" to 1L, "name" to "john", "surname" to "doe") + } + } + + @Neo4jSink(nodePattern = [NodePatternStrategy(TOPIC, "(:User{!id,name,surname})", false)]) + @Test + fun `should create node from json byte array`( + @TopicProducer(TOPIC) producer: ConvertingKafkaProducer, + session: Session + ) = runTest { + producer.publish( + valueSchema = Schema.BYTES_SCHEMA, + value = + ObjectMapper() + .writeValueAsBytes(mapOf("id" to 1L, "name" to "john", "surname" to "doe"))) + + eventually(30.seconds) { session.run("MATCH (n:User) RETURN n", emptyMap()).single() } + .get("n") + .asNode() should + { + it.labels() shouldBe listOf("User") + it.asMap() shouldBe mapOf("id" to 1L, "name" to "john", "surname" to "doe") + } + } + @Neo4jSink( nodePattern = [ @@ -115,6 +167,67 @@ class Neo4jNodePatternIT { } } + @Neo4jSink(nodePattern = [NodePatternStrategy(TOPIC, "(:User{!id,name,surname})", false)]) + @Test + fun `should create, delete and recreate node`( + @TopicProducer(TOPIC) producer: ConvertingKafkaProducer, + session: Session + ) = runTest { + producer.publish( + keySchema = Schema.STRING_SCHEMA, + key = """{"id": 1}""", + valueSchema = Schema.STRING_SCHEMA, + value = """{"name": "john", "surname": "doe"}""") + producer.publish( + keySchema = Schema.STRING_SCHEMA, + key = """{"id": 1}""", + ) + producer.publish( + keySchema = Schema.STRING_SCHEMA, + key = """{"id": 1}""", + valueSchema = Schema.STRING_SCHEMA, + value = """{"name": "john-new", "surname": "doe-new"}""") + + eventually(30.seconds) { + session.run("MATCH (n:User) RETURN n", emptyMap()).single().get("n").asNode() should + { + it.labels() shouldBe listOf("User") + it.asMap() shouldBe mapOf("id" to 1L, "name" to "john-new", "surname" to "doe-new") + } + } + } + + @Neo4jSink(nodePattern = [NodePatternStrategy(TOPIC, "(:User{!id,name,surname})", false)]) + @Test + fun `should create multiple nodes`( + @TopicProducer(TOPIC) producer: ConvertingKafkaProducer, + session: Session + ) = runTest { + producer.publish( + keySchema = Schema.STRING_SCHEMA, + key = """{"id": 1}""", + valueSchema = Schema.STRING_SCHEMA, + value = """{"name": "john", "surname": "doe"}""") + producer.publish( + keySchema = Schema.STRING_SCHEMA, + key = """{"id": 2}""", + valueSchema = Schema.STRING_SCHEMA, + value = """{"name": "mary", "surname": "doe"}""") + + eventually(30.seconds) { + session.run("MATCH (n:User) RETURN n", emptyMap()).list { r -> + r.get("n").asNode().let { mapOf("labels" to it.labels(), "properties" to it.asMap()) } + } shouldBe + listOf( + mapOf( + "labels" to listOf("User"), + "properties" to mapOf("id" to 1, "name" to "john", "surname" to "doe")), + mapOf( + "labels" to listOf("User"), + "properties" to mapOf("id" to 2, "name" to "mary", "surname" to "doe"))) + } + } + @Neo4jSink(nodePattern = [NodePatternStrategy(TOPIC, "(:User{!id,!name,surname})", false)]) @Test fun `should create node with compositeKey`( @@ -202,4 +315,158 @@ class Neo4jNodePatternIT { it.asMap()["created_at"] shouldNotBe null } } + + @Neo4jSink( + nodePattern = + [ + NodePatternStrategy( + TOPIC, + "(:User{!id: __key.old_id, name: __key.first_name, surname: __key.last_name})", + false)]) + @Test + fun `should create and delete node with all keys pattern`( + @TopicProducer(TOPIC) producer: ConvertingKafkaProducer, + session: Session + ) = runTest { + producer.publish( + keySchema = Schema.STRING_SCHEMA, + key = """{"old_id": 1, "first_name": "john", "last_name": "doe"}""", + valueSchema = Schema.STRING_SCHEMA, + value = """{}""") + eventually(30.seconds) { session.run("MATCH (n:User) RETURN n", emptyMap()).single() } + .get("n") + .asNode() should + { + it.labels() shouldBe listOf("User") + it.asMap() shouldBe mapOf("id" to 1L, "name" to "john", "surname" to "doe") + } + + producer.publish( + keySchema = Schema.STRING_SCHEMA, + key = """{"old_id": 1}""", + ) + eventually(30.seconds) { + session + .run("MATCH (n:User) RETURN count(n) AS count", emptyMap()) + .single() + .get("count") + .asLong() shouldBe 0 + } + } + + @Neo4jSink(nodePattern = [NodePatternStrategy(TOPIC, "(:User{!id})", false)]) + @Test + fun `should create and update node`( + @TopicProducer(TOPIC) producer: ConvertingKafkaProducer, + session: Session + ) = runTest { + producer.publish( + keySchema = Schema.STRING_SCHEMA, + key = """{"id": 1}""", + valueSchema = Schema.STRING_SCHEMA, + value = """{"name": "john", "surname": "doe"}""") + + eventually(30.seconds) { session.run("MATCH (n:User) RETURN n", emptyMap()).single() } + .get("n") + .asNode() should + { + it.labels() shouldBe listOf("User") + it.asMap() shouldBe mapOf("id" to 1L, "name" to "john", "surname" to "doe") + } + + producer.publish( + keySchema = Schema.STRING_SCHEMA, + key = """{"id": 1}""", + valueSchema = Schema.STRING_SCHEMA, + value = """{"name": "john-updated", "surname": "doe-updated"}""") + + eventually(30.seconds) { + session.run("MATCH (n:User) RETURN n", emptyMap()).single().get("n").asNode() should + { + it.labels() shouldBe listOf("User") + it.asMap() shouldBe + mapOf("id" to 1L, "name" to "john-updated", "surname" to "doe-updated") + } + } + } + + @Neo4jSink( + nodePattern = + [ + NodePatternStrategy( + TOPIC, "(:User{!id: old_id, name: first_name, surname: last_name})", false)]) + @Test + fun `should create and update node with aliases`( + @TopicProducer(TOPIC) producer: ConvertingKafkaProducer, + session: Session + ) = runTest { + producer.publish( + keySchema = Schema.STRING_SCHEMA, + key = """{"old_id": 1}""", + valueSchema = Schema.STRING_SCHEMA, + value = """{"first_name": "john", "last_name": "doe"}""") + + eventually(30.seconds) { session.run("MATCH (n:User) RETURN n", emptyMap()).single() } + .get("n") + .asNode() should + { + it.labels() shouldBe listOf("User") + it.asMap() shouldBe mapOf("id" to 1L, "name" to "john", "surname" to "doe") + } + + producer.publish( + keySchema = Schema.STRING_SCHEMA, + key = """{"old_id": 1}""", + valueSchema = Schema.STRING_SCHEMA, + value = """{"first_name": "john-updated", "last_name": "doe-updated"}""") + + eventually(30.seconds) { + session.run("MATCH (n:User) RETURN n", emptyMap()).single().get("n").asNode() should + { + it.labels() shouldBe listOf("User") + it.asMap() shouldBe + mapOf("id" to 1L, "name" to "john-updated", "surname" to "doe-updated") + } + } + } + + @Neo4jSink( + nodePattern = + [ + NodePatternStrategy(TOPIC_1, "(:User{!id})", false), + NodePatternStrategy(TOPIC_2, "(:Account{!id})", false)]) + @Test + fun `should create nodes from multiple topics`( + @TopicProducer(TOPIC_1) producer1: ConvertingKafkaProducer, + @TopicProducer(TOPIC_2) producer2: ConvertingKafkaProducer, + session: Session + ) = runTest { + producer1.publish( + keySchema = Schema.STRING_SCHEMA, + key = """{"id": 1}""", + valueSchema = Schema.STRING_SCHEMA, + value = """{"name": "john", "surname": "doe"}""") + + eventually(30.seconds) { session.run("MATCH (n:User) RETURN n", emptyMap()).single() } + .get("n") + .asNode() should + { + it.labels() shouldBe listOf("User") + it.asMap() shouldBe mapOf("id" to 1L, "name" to "john", "surname" to "doe") + } + + producer2.publish( + keySchema = Schema.STRING_SCHEMA, + key = """{"id": 1}""", + valueSchema = Schema.STRING_SCHEMA, + value = """{"email": "john@doe.com"}""") + + eventually(30.seconds) { session.run("MATCH (n:Account) RETURN n", emptyMap()).single() } + .get("n") + .asNode() should + { + it.labels() shouldBe listOf("Account") + it.asMap() shouldBe mapOf("id" to 1L, "email" to "john@doe.com") + } + } } From 503a124d4a30998db2b00d69a8e45c27dcf114b2 Mon Sep 17 00:00:00 2001 From: Ali Ince Date: Fri, 31 May 2024 13:01:35 +0100 Subject: [PATCH 04/14] refactor: use constants for literals --- .../kafka/sink/strategy/NodePatternHandler.kt | 108 +++++----- .../kafka/sink/strategy/PatternHandler.kt | 15 ++ .../strategy/RelationshipPatternHandler.kt | 199 ++++++++++-------- 3 files changed, 183 insertions(+), 139 deletions(-) diff --git a/sink/src/main/kotlin/org/neo4j/connectors/kafka/sink/strategy/NodePatternHandler.kt b/sink/src/main/kotlin/org/neo4j/connectors/kafka/sink/strategy/NodePatternHandler.kt index 45227ba7f..b736fe570 100644 --- a/sink/src/main/kotlin/org/neo4j/connectors/kafka/sink/strategy/NodePatternHandler.kt +++ b/sink/src/main/kotlin/org/neo4j/connectors/kafka/sink/strategy/NodePatternHandler.kt @@ -23,6 +23,9 @@ import org.neo4j.connectors.kafka.sink.SinkStrategy import org.neo4j.connectors.kafka.sink.strategy.pattern.NodePattern import org.neo4j.connectors.kafka.sink.strategy.pattern.Pattern import org.neo4j.cypherdsl.core.Cypher +import org.neo4j.cypherdsl.core.Literal +import org.neo4j.cypherdsl.core.Node +import org.neo4j.cypherdsl.core.SymbolicName import org.neo4j.cypherdsl.core.renderer.Renderer import org.neo4j.driver.Query import org.slf4j.Logger @@ -75,13 +78,11 @@ class NodePatternHandler( extractKeys(pattern, flattened, isTombstoneMessage, used, bindValueAs, bindKeyAs) val mapped = if (isTombstoneMessage) { - listOf("D", mapOf("keys" to keys)) + listOf(DELETE, mapOf(KEYS to keys)) } else { listOf( - "C", - mapOf( - "keys" to keys, - "properties" to computeProperties(pattern, flattened, used))) + CREATE, + mapOf(KEYS to keys, PROPERTIES to computeProperties(pattern, flattened, used))) } logger.trace("message '{}' mapped to: '{}'", it, mapped) @@ -89,65 +90,74 @@ class NodePatternHandler( mapped } .chunked(batchSize) - .map { listOf(ChangeQuery(null, null, Query(query, mapOf("events" to it)))) } + .map { listOf(ChangeQuery(null, null, Query(query, mapOf(EVENTS to it)))) } .onEach { logger.trace("mapped messages: '{}'", it) } .toList() } private fun buildStatement(): String { - val event = Cypher.name("event") - val created = Cypher.name("created") - val deleted = Cypher.name("deleted") - val createOperation = Cypher.literalOf("C") - val deleteOperation = Cypher.literalOf("D") + val createOperation = Cypher.literalOf(CREATE) + val deleteOperation = Cypher.literalOf(DELETE) val node = Cypher.node(pattern.labels.first(), pattern.labels.drop(1)) .withProperties( - pattern.keyProperties.associate { it.to to event.property("keys").property(it.to) }, + pattern.keyProperties.associate { + it.to to NAME_EVENT.property(KEYS).property(it.to) + }, ) .named("n") return renderer.render( - Cypher.unwind(Cypher.parameter("events")) - .`as`(event) - .call( - Cypher.with(event) - .with(event) - .where(Cypher.valueAt(event, 0).eq(createOperation)) - .with(Cypher.valueAt(event, 1).`as`(event)) - .merge(node) - .let { - if (mergeProperties) { - it.mutate( - node.asExpression(), - Cypher.property("event", "properties"), - ) - } else { - it.set( - node.asExpression(), - Cypher.property("event", "properties"), - ) - .mutate(node.asExpression(), Cypher.property("event", "keys")) - } - } - .returning( - Cypher.raw("count(${'$'}E)", node.requiredSymbolicName).`as`(created)) - .build()) - .call( - Cypher.with(event) - .with(event) - .where(Cypher.valueAt(event, 0).eq(deleteOperation)) - .with(Cypher.valueAt(event, 1).`as`(event)) - .match(node) - .detachDelete(node) - .returning( - Cypher.raw("count(${'$'}E)", node.requiredSymbolicName).`as`(deleted)) - .build()) + Cypher.unwind(Cypher.parameter(EVENTS)) + .`as`(NAME_EVENT) + .call(buildCreateStatement(NAME_EVENT, createOperation, node)) + .call(buildDeleteStatement(NAME_EVENT, deleteOperation, node)) .returning( - Cypher.raw("sum(${'$'}E)", created).`as`(created), - Cypher.raw("sum(${'$'}E)", deleted).`as`(deleted)) + Cypher.raw("sum(${'$'}E)", NAME_CREATED).`as`(NAME_CREATED), + Cypher.raw("sum(${'$'}E)", NAME_DELETED).`as`(NAME_DELETED)) .build(), ) } + + private fun buildDeleteStatement( + event: SymbolicName, + deleteOperation: Literal, + node: Node, + ) = + Cypher.with(event) + .with(event) + .where(Cypher.valueAt(event, 0).eq(deleteOperation)) + .with(Cypher.valueAt(event, 1).`as`(event)) + .match(node) + .detachDelete(node) + .returning(Cypher.raw("count(${'$'}E)", node.requiredSymbolicName).`as`(NAME_DELETED)) + .build() + + private fun buildCreateStatement( + event: SymbolicName, + createOperation: Literal, + node: Node, + ) = + Cypher.with(event) + .with(event) + .where(Cypher.valueAt(event, 0).eq(createOperation)) + .with(Cypher.valueAt(event, 1).`as`(event)) + .merge(node) + .let { + if (mergeProperties) { + it.mutate( + node.asExpression(), + Cypher.property(NAME_EVENT, PROPERTIES), + ) + } else { + it.set( + node.asExpression(), + Cypher.property(NAME_EVENT, PROPERTIES), + ) + } + } + .mutate(node.asExpression(), Cypher.property(NAME_EVENT, KEYS)) + .returning(Cypher.raw("count(${'$'}E)", node.requiredSymbolicName).`as`(NAME_CREATED)) + .build() } diff --git a/sink/src/main/kotlin/org/neo4j/connectors/kafka/sink/strategy/PatternHandler.kt b/sink/src/main/kotlin/org/neo4j/connectors/kafka/sink/strategy/PatternHandler.kt index e2cf8e311..450107217 100644 --- a/sink/src/main/kotlin/org/neo4j/connectors/kafka/sink/strategy/PatternHandler.kt +++ b/sink/src/main/kotlin/org/neo4j/connectors/kafka/sink/strategy/PatternHandler.kt @@ -24,6 +24,7 @@ import org.neo4j.connectors.kafka.sink.SinkConfiguration import org.neo4j.connectors.kafka.sink.SinkMessage import org.neo4j.connectors.kafka.sink.SinkStrategyHandler import org.neo4j.connectors.kafka.sink.strategy.pattern.Pattern +import org.neo4j.cypherdsl.core.Cypher abstract class PatternHandler( protected val bindTimestampAs: String = SinkConfiguration.DEFAULT_BIND_TIMESTAMP_ALIAS, @@ -31,6 +32,20 @@ abstract class PatternHandler( protected val bindKeyAs: String = SinkConfiguration.DEFAULT_BIND_KEY_ALIAS, protected val bindValueAs: String = SinkConfiguration.DEFAULT_BIND_VALUE_ALIAS, ) : SinkStrategyHandler { + companion object { + internal const val CREATE = "C" + internal const val DELETE = "D" + internal const val EVENTS = "events" + internal const val KEYS = "keys" + internal const val PROPERTIES = "properties" + internal const val START = "start" + internal const val END = "end" + + internal val NAME_EVENT = Cypher.name("event") + internal val NAME_CREATED = Cypher.name("created") + internal val NAME_DELETED = Cypher.name("deleted") + } + abstract val pattern: T @Suppress("UNCHECKED_CAST") diff --git a/sink/src/main/kotlin/org/neo4j/connectors/kafka/sink/strategy/RelationshipPatternHandler.kt b/sink/src/main/kotlin/org/neo4j/connectors/kafka/sink/strategy/RelationshipPatternHandler.kt index 3bd14bf6f..291e61fe4 100644 --- a/sink/src/main/kotlin/org/neo4j/connectors/kafka/sink/strategy/RelationshipPatternHandler.kt +++ b/sink/src/main/kotlin/org/neo4j/connectors/kafka/sink/strategy/RelationshipPatternHandler.kt @@ -23,6 +23,9 @@ import org.neo4j.connectors.kafka.sink.SinkStrategy import org.neo4j.connectors.kafka.sink.strategy.pattern.Pattern import org.neo4j.connectors.kafka.sink.strategy.pattern.RelationshipPattern import org.neo4j.cypherdsl.core.Cypher +import org.neo4j.cypherdsl.core.Literal +import org.neo4j.cypherdsl.core.Node +import org.neo4j.cypherdsl.core.Relationship import org.neo4j.cypherdsl.core.renderer.Renderer import org.neo4j.driver.Query import org.slf4j.Logger @@ -82,21 +85,21 @@ class RelationshipPatternHandler( val mapped = if (isTombstoneMessage) { listOf( - "D", + DELETE, mapOf( - "start" to mapOf("keys" to startKeys), - "end" to mapOf("keys" to endKeys), - "keys" to keys)) + START to mapOf(KEYS to startKeys), + END to mapOf(KEYS to endKeys), + KEYS to keys)) } else { val startProperties = computeProperties(pattern.start, flattened, used) val endProperties = computeProperties(pattern.end, flattened, used) listOf( - "C", + CREATE, mapOf( - "start" to mapOf("keys" to startKeys, "properties" to startProperties), - "end" to mapOf("keys" to endKeys, "properties" to endProperties), - "keys" to keys, - "properties" to computeProperties(pattern, flattened, used))) + START to mapOf(KEYS to startKeys, PROPERTIES to startProperties), + END to mapOf(KEYS to endKeys, PROPERTIES to endProperties), + KEYS to keys, + PROPERTIES to computeProperties(pattern, flattened, used))) } logger.trace("message '{}' mapped to: '{}'", it, mapped) @@ -104,114 +107,130 @@ class RelationshipPatternHandler( mapped } .chunked(batchSize) - .map { listOf(ChangeQuery(null, null, Query(query, mapOf("events" to it)))) } + .map { listOf(ChangeQuery(null, null, Query(query, mapOf(EVENTS to it)))) } .onEach { logger.trace("mapped messages: '{}'", it) } .toList() } private fun buildStatement(): String { - val event = Cypher.name("event") - val created = Cypher.name("created") - val deleted = Cypher.name("deleted") - val createOperation = Cypher.literalOf("C") - val deleteOperation = Cypher.literalOf("D") + val createOperation = Cypher.literalOf(CREATE) + val deleteOperation = Cypher.literalOf(DELETE) val startNode = Cypher.node(pattern.start.labels.first(), pattern.start.labels.drop(1)) .withProperties( pattern.start.keyProperties.associate { - it.to to event.property("start", "keys").property(it.to) + it.to to NAME_EVENT.property(START, KEYS).property(it.to) }, ) - .named("start") + .named(START) val endNode = Cypher.node(pattern.end.labels.first(), pattern.end.labels.drop(1)) .withProperties( pattern.end.keyProperties.associate { - it.to to event.property("end", "keys").property(it.to) + it.to to NAME_EVENT.property(END, KEYS).property(it.to) }, ) - .named("end") + .named(END) val relationship = startNode .relationshipTo(endNode, pattern.type) .withProperties( - pattern.keyProperties.associate { it.to to event.property("keys").property(it.to) }) + pattern.keyProperties.associate { + it.to to NAME_EVENT.property(KEYS).property(it.to) + }) .named("relationship") return renderer.render( - Cypher.unwind(Cypher.parameter("events")) - .`as`(event) - .call( - Cypher.with(event) - .with(event) - .where(Cypher.valueAt(event, 0).eq(createOperation)) - .with(Cypher.valueAt(event, 1).`as`(event)) - .merge(startNode) - .let { - if (mergeNodeProperties) { - it.mutate( - startNode.asExpression(), - Cypher.property("event", "start", "properties"), - ) - } else { - it.set( - startNode.asExpression(), - Cypher.property("event", "start", "properties"), - ) - .mutate( - startNode.asExpression(), Cypher.property("event", "start", "keys")) - } - } - .merge(endNode) - .let { - if (mergeNodeProperties) { - it.mutate( - endNode.asExpression(), - Cypher.property("event", "end", "properties"), - ) - } else { - it.set( - endNode.asExpression(), - Cypher.property("event", "end", "properties"), - ) - .mutate(endNode.asExpression(), Cypher.property("event", "end", "keys")) - } - } - .merge(relationship) - .let { - if (mergeRelationshipProperties) { - it.mutate( - relationship.asExpression(), - Cypher.property("event", "properties"), - ) - } else { - it.set( - relationship.asExpression(), - Cypher.property("event", "properties"), - ) - .mutate(relationship.asExpression(), Cypher.property("event", "keys")) - } - } - .returning( - Cypher.raw("count(${'$'}E)", relationship.requiredSymbolicName) - .`as`(created)) - .build()) - .call( - Cypher.with(event) - .with(event) - .where(Cypher.valueAt(event, 0).eq(deleteOperation)) - .with(Cypher.valueAt(event, 1).`as`(event)) - .match(relationship) - .delete(relationship) - .returning( - Cypher.raw("count(${'$'}E)", relationship.requiredSymbolicName) - .`as`(deleted)) - .build()) + Cypher.unwind(Cypher.parameter(EVENTS)) + .`as`(NAME_EVENT) + .call(buildCreateStatement(startNode, endNode, relationship, createOperation)) + .call(buildDeleteStatement(relationship, deleteOperation)) .returning( - Cypher.raw("sum(${'$'}E)", created).`as`(created), - Cypher.raw("sum(${'$'}E)", deleted).`as`(deleted)) + Cypher.raw("sum(${'$'}E)", NAME_CREATED).`as`(NAME_CREATED), + Cypher.raw("sum(${'$'}E)", NAME_DELETED).`as`(NAME_DELETED)) .build(), ) } + + private fun buildDeleteStatement( + relationship: Relationship, + deleteOperation: Literal, + ) = + Cypher.with(NAME_EVENT) + .with(NAME_EVENT) + .where(Cypher.valueAt(NAME_EVENT, 0).eq(deleteOperation)) + .with(Cypher.valueAt(NAME_EVENT, 1).`as`(NAME_EVENT)) + .match(relationship) + .delete(relationship) + .returning( + Cypher.raw("count(${'$'}E)", relationship.requiredSymbolicName).`as`(NAME_DELETED), + ) + .build() + + private fun buildCreateStatement( + startNode: Node, + endNode: Node, + relationship: Relationship, + createOperation: Literal, + ) = + Cypher.with(NAME_EVENT) + .with(NAME_EVENT) + .where(Cypher.valueAt(NAME_EVENT, 0).eq(createOperation)) + .with(Cypher.valueAt(NAME_EVENT, 1).`as`(NAME_EVENT)) + .merge(startNode) + .let { + if (mergeNodeProperties) { + it.mutate( + startNode.asExpression(), + Cypher.property(NAME_EVENT, START, PROPERTIES), + ) + } else { + it.set( + startNode.asExpression(), + Cypher.property(NAME_EVENT, START, PROPERTIES), + ) + } + } + .mutate( + startNode.asExpression(), + Cypher.property(NAME_EVENT, START, KEYS), + ) + .merge(endNode) + .let { + if (mergeNodeProperties) { + it.mutate( + endNode.asExpression(), + Cypher.property(NAME_EVENT, END, PROPERTIES), + ) + } else { + it.set( + endNode.asExpression(), + Cypher.property(NAME_EVENT, END, PROPERTIES), + ) + } + } + .mutate( + endNode.asExpression(), + Cypher.property(NAME_EVENT, END, KEYS), + ) + .merge(relationship) + .let { + if (mergeRelationshipProperties) { + it.mutate( + relationship.asExpression(), + Cypher.property(NAME_EVENT, PROPERTIES), + ) + } else { + it.set( + relationship.asExpression(), + Cypher.property(NAME_EVENT, PROPERTIES), + ) + } + } + .mutate(relationship.asExpression(), Cypher.property(NAME_EVENT, KEYS)) + .returning( + Cypher.raw("count(${'$'}E)", relationship.requiredSymbolicName).`as`(NAME_CREATED), + ) + .build() } From 24308674b4e7e4e4ded9a2732f1f4f5efb8ddf2c Mon Sep 17 00:00:00 2001 From: Ali Ince Date: Fri, 31 May 2024 13:33:01 +0100 Subject: [PATCH 05/14] refactor: use new pattern parser for legacy pattern strategies --- .../sink/strategy/PatternConfiguration.kt | 277 --------- .../RelationshipPatternIngestionStrategy.kt | 151 ----- .../neo4j/connectors/kafka/utils/JSONUtils.kt | 17 +- .../sink/strategy/PatternConfigurationTest.kt | 580 ------------------ .../sink/DeprecatedNeo4jSinkConfiguration.kt | 2 +- .../kafka/sink/DeprecatedNeo4jSinkService.kt | 4 +- .../kafka/sink/DeprecatedNeo4jSinkTask.kt | 2 +- .../connectors/kafka/sink/EventBuilder.kt | 2 +- .../kafka/sink/Neo4jStrategyStorage.kt | 28 +- .../kafka/sink/SinkConfiguration.kt | 7 +- .../kafka/sink/strategy/CdcHandler.kt | 2 +- .../kafka/sink/strategy/CudHandler.kt | 2 +- .../kafka/sink/strategy/NodePatternHandler.kt | 8 +- .../kafka/sink/strategy/PatternHandler.kt | 2 +- .../kafka/sink/strategy/RedirectingHandler.kt | 2 +- .../strategy/RelationshipPatternHandler.kt | 36 +- .../strategy/legacy}/CUDIngestionStrategy.kt | 12 +- .../sink/strategy/legacy}/CommonExtensions.kt | 21 +- .../legacy}/CypherTemplateStrategy.kt | 3 +- .../strategy/legacy}/IngestionStrategy.kt | 3 +- .../sink/strategy/legacy}/IngestionUtils.kt | 6 +- .../legacy}/NodePatternIngestionStrategy.kt | 74 ++- .../RelationshipPatternIngestionStrategy.kt | 169 +++++ .../legacy}/SchemaIngestionStrategy.kt | 12 +- .../sink/strategy/legacy}/SchemaUtils.kt | 4 +- .../legacy}/SourceIdIngestionStrategy.kt | 7 +- .../strategy/legacy}/StreamsSinkService.kt | 4 +- .../sink/utils/ConnectExtensionFunctions.kt | 4 +- .../connectors/kafka/sink/utils/Topics.kt | 59 +- .../sink/DeprecatedNeo4jSinkTaskAuraTest.kt | 4 +- .../kafka/sink/DeprecatedNeo4jSinkTaskTest.kt | 9 +- .../kafka/sink/SinkConfigurationTest.kt | 2 +- .../sink/strategy/NodePatternHandlerTest.kt | 2 +- .../RelationshipPatternHandlerTest.kt | 2 +- .../legacy}/CUDIngestionStrategyTest.kt | 4 +- .../NodePatternIngestionStrategyTest.kt | 33 +- ...elationshipPatternIngestionStrategyTest.kt | 43 +- .../legacy}/SchemaIngestionStrategyTest.kt | 3 +- .../sink/strategy/legacy}/SchemaUtilsTest.kt | 4 +- .../legacy}/SourceIdIngestionStrategyTest.kt | 3 +- .../source/utils/ConnectExtensionFunctions.kt | 2 +- 41 files changed, 388 insertions(+), 1223 deletions(-) delete mode 100644 common/src/main/kotlin/org/neo4j/connectors/kafka/service/sink/strategy/PatternConfiguration.kt delete mode 100644 common/src/main/kotlin/org/neo4j/connectors/kafka/service/sink/strategy/RelationshipPatternIngestionStrategy.kt delete mode 100644 common/src/test/kotlin/org/neo4j/connectors/kafka/service/sink/strategy/PatternConfigurationTest.kt rename {common/src/main/kotlin/org/neo4j/connectors/kafka/service/sink/strategy => sink/src/main/kotlin/org/neo4j/connectors/kafka/sink/strategy/legacy}/CUDIngestionStrategy.kt (96%) rename {common/src/main/kotlin/org/neo4j/connectors/kafka/extensions => sink/src/main/kotlin/org/neo4j/connectors/kafka/sink/strategy/legacy}/CommonExtensions.kt (83%) rename {common/src/main/kotlin/org/neo4j/connectors/kafka/service/sink/strategy => sink/src/main/kotlin/org/neo4j/connectors/kafka/sink/strategy/legacy}/CypherTemplateStrategy.kt (92%) rename {common/src/main/kotlin/org/neo4j/connectors/kafka/service/sink/strategy => sink/src/main/kotlin/org/neo4j/connectors/kafka/sink/strategy/legacy}/IngestionStrategy.kt (94%) rename {common/src/main/kotlin/org/neo4j/connectors/kafka/utils => sink/src/main/kotlin/org/neo4j/connectors/kafka/sink/strategy/legacy}/IngestionUtils.kt (91%) rename {common/src/main/kotlin/org/neo4j/connectors/kafka/service/sink/strategy => sink/src/main/kotlin/org/neo4j/connectors/kafka/sink/strategy/legacy}/NodePatternIngestionStrategy.kt (52%) create mode 100644 sink/src/main/kotlin/org/neo4j/connectors/kafka/sink/strategy/legacy/RelationshipPatternIngestionStrategy.kt rename {common/src/main/kotlin/org/neo4j/connectors/kafka/service/sink/strategy => sink/src/main/kotlin/org/neo4j/connectors/kafka/sink/strategy/legacy}/SchemaIngestionStrategy.kt (95%) rename {common/src/main/kotlin/org/neo4j/connectors/kafka/utils => sink/src/main/kotlin/org/neo4j/connectors/kafka/sink/strategy/legacy}/SchemaUtils.kt (96%) rename {common/src/main/kotlin/org/neo4j/connectors/kafka/service/sink/strategy => sink/src/main/kotlin/org/neo4j/connectors/kafka/sink/strategy/legacy}/SourceIdIngestionStrategy.kt (95%) rename {common/src/main/kotlin/org/neo4j/connectors/kafka/service => sink/src/main/kotlin/org/neo4j/connectors/kafka/sink/strategy/legacy}/StreamsSinkService.kt (94%) rename {common/src/test/kotlin/org/neo4j/connectors/kafka/service/sink/strategy => sink/src/test/kotlin/org/neo4j/connectors/kafka/sink/strategy/legacy}/CUDIngestionStrategyTest.kt (99%) rename {common/src/test/kotlin/org/neo4j/connectors/kafka/service/sink/strategy => sink/src/test/kotlin/org/neo4j/connectors/kafka/sink/strategy/legacy}/NodePatternIngestionStrategyTest.kt (86%) rename {common/src/test/kotlin/org/neo4j/connectors/kafka/service/sink/strategy => sink/src/test/kotlin/org/neo4j/connectors/kafka/sink/strategy/legacy}/RelationshipPatternIngestionStrategyTest.kt (86%) rename {common/src/test/kotlin/org/neo4j/connectors/kafka/service/sink/strategy => sink/src/test/kotlin/org/neo4j/connectors/kafka/sink/strategy/legacy}/SchemaIngestionStrategyTest.kt (99%) rename {common/src/test/kotlin/org/neo4j/connectors/kafka/utils => sink/src/test/kotlin/org/neo4j/connectors/kafka/sink/strategy/legacy}/SchemaUtilsTest.kt (97%) rename {common/src/test/kotlin/org/neo4j/connectors/kafka/service/sink/strategy => sink/src/test/kotlin/org/neo4j/connectors/kafka/sink/strategy/legacy}/SourceIdIngestionStrategyTest.kt (99%) diff --git a/common/src/main/kotlin/org/neo4j/connectors/kafka/service/sink/strategy/PatternConfiguration.kt b/common/src/main/kotlin/org/neo4j/connectors/kafka/service/sink/strategy/PatternConfiguration.kt deleted file mode 100644 index cc11bed3a..000000000 --- a/common/src/main/kotlin/org/neo4j/connectors/kafka/service/sink/strategy/PatternConfiguration.kt +++ /dev/null @@ -1,277 +0,0 @@ -/* - * Copyright (c) "Neo4j" - * Neo4j Sweden AB [https://neo4j.com] - * - * 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 org.neo4j.connectors.kafka.service.sink.strategy - -import org.neo4j.connectors.kafka.extensions.quote - -enum class PatternConfigurationType { - ALL, - INCLUDE, - EXCLUDE -} - -private const val ID_PREFIX = "!" -private const val MINUS_PREFIX = "-" -private const val LABEL_SEPARATOR = ":" -private const val PROPERTIES_SEPARATOR = "," - -private fun getPatternConfiguredType(properties: List): PatternConfigurationType { - if (properties.isEmpty()) { - return PatternConfigurationType.ALL - } - return when (properties[0].trim()[0]) { - '*' -> PatternConfigurationType.ALL - '-' -> PatternConfigurationType.EXCLUDE - else -> PatternConfigurationType.INCLUDE - } -} - -private fun isHomogeneousPattern( - type: PatternConfigurationType, - properties: List, - pattern: String, - entityType: String -) { - val isHomogeneous = - when (type) { - PatternConfigurationType.INCLUDE -> properties.all { it.trim()[0].isJavaIdentifierStart() } - PatternConfigurationType.EXCLUDE -> properties.all { it.trim().startsWith(MINUS_PREFIX) } - PatternConfigurationType.ALL -> properties.isEmpty() || properties == listOf("*") - } - if (!isHomogeneous) { - throw IllegalArgumentException("The $entityType pattern $pattern is not homogeneous") - } -} - -private fun cleanProperties( - type: PatternConfigurationType, - properties: List -): List { - return when (type) { - PatternConfigurationType.INCLUDE -> properties.map { it.trim() } - PatternConfigurationType.EXCLUDE -> properties.map { it.trim().replace(MINUS_PREFIX, "") } - PatternConfigurationType.ALL -> emptyList() - } -} - -interface PatternConfiguration - -data class NodePatternConfiguration( - val keys: Set, - val type: PatternConfigurationType, - val labels: List, - val properties: List, - val mergeProperties: Boolean -) : PatternConfiguration { - companion object { - - // (:LabelA{!id,foo,bar}) - @JvmStatic - private val cypherNodePatternConfigured = - """\((:\w+\s*(?::\s*(?:\w+)\s*)*)\s*(?:\{\s*(-?[\w!\.]+\s*(?:,\s*-?[!\w\*\.]+\s*)*)\})?\)$""" - .toRegex() - // LabelA{!id,foo,bar} - @JvmStatic - private val simpleNodePatternConfigured = - """^(\w+\s*(?::\s*(?:\w+)\s*)*)\s*(?:\{\s*(-?[\w!\.]+\s*(?:,\s*-?[!\w\*\.]+\s*)*)\})?$""" - .toRegex() - - fun parse(pattern: String, mergeProperties: Boolean): NodePatternConfiguration { - val isCypherPattern = pattern.startsWith("(") - val regex = if (isCypherPattern) cypherNodePatternConfigured else simpleNodePatternConfigured - val matcher = regex.matchEntire(pattern) - if (matcher == null) { - throw IllegalArgumentException("The Node pattern $pattern is invalid") - } else { - val labels = - matcher.groupValues[1] - .split(LABEL_SEPARATOR) - .let { if (isCypherPattern) it.drop(1) else it } - .map { it.quote() } - val allProperties = matcher.groupValues[2].split(PROPERTIES_SEPARATOR) - val keys = - allProperties.filter { it.startsWith(ID_PREFIX) }.map { it.trim().substring(1) }.toSet() - if (keys.isEmpty()) { - throw IllegalArgumentException("The Node pattern $pattern must contain at least one key") - } - val properties = allProperties.filter { !it.startsWith(ID_PREFIX) } - val type = getPatternConfiguredType(properties) - isHomogeneousPattern(type, properties, pattern, "Node") - val cleanedProperties = cleanProperties(type, properties) - - return NodePatternConfiguration( - keys = keys, - type = type, - labels = labels, - properties = cleanedProperties, - mergeProperties) - } - } - } -} - -data class RelationshipPatternConfiguration( - val start: NodePatternConfiguration, - val end: NodePatternConfiguration, - val relType: String, - val type: PatternConfigurationType, - val properties: List, - val mergeProperties: Boolean -) : PatternConfiguration { - companion object { - - // we don't allow ALL for start/end nodes in rels - // it's public for testing purpose - fun getNodeConf(pattern: String, mergeProperties: Boolean): NodePatternConfiguration { - val start = NodePatternConfiguration.parse(pattern, mergeProperties) - return if (start.type == PatternConfigurationType.ALL) { - NodePatternConfiguration( - keys = start.keys, - type = PatternConfigurationType.INCLUDE, - labels = start.labels, - properties = start.properties, - mergeProperties) - } else { - start - } - } - - // (:Source{!id})-[:REL_TYPE{foo, -bar}]->(:Target{!targetId}) - private val cypherRelationshipPatternConfigured = - """^\(:(.*?)\)(<)?-\[(?::)([\w\_]+)(\{\s*(-?[\w\*\.]+\s*(?:,\s*-?[\w\*\.]+\s*)*)\})?\]-(>)?\(:(.*?)\)$""" - .toRegex() - // LabelA{!id} REL_TYPE{foo, -bar} LabelB{!targetId} - private val simpleRelationshipPatternConfigured = - """^(.*?) ([\w\_]+)(\{\s*(-?[\w\*\.]+\s*(?:,\s*-?[\w\*\.]+\s*)*)\})? (.*?)$""" - .toRegex() // """^\((.*?)\)-\[(?::)([\w\_]+)(\{\s*(-?[\w\*\.]+\s*(?:,\s*-?[\w\*\.]+\s*)*)\})?\]->\((.*?)\)$""".toRegex() - - data class RelationshipPatternMetaData( - val startPattern: String, - val endPattern: String, - val relType: String, - val properties: List - ) { - companion object { - - private fun toProperties(propGroup: String): List = - if (propGroup.isNullOrBlank()) { - emptyList() - } else { - propGroup.split(PROPERTIES_SEPARATOR) - } - - fun create( - isCypherPattern: Boolean, - isLeftToRight: Boolean, - groupValues: List - ): RelationshipPatternMetaData { - lateinit var start: String - lateinit var end: String - lateinit var relType: String - lateinit var props: List - - if (isCypherPattern) { - if (isLeftToRight) { - start = groupValues[1] - end = groupValues[7] - } else { - start = groupValues[7] - end = groupValues[1] - } - relType = groupValues[3] - props = toProperties(groupValues[5]) - } else { - if (isLeftToRight) { - start = groupValues[1] - end = groupValues[5] - } else { - start = groupValues[5] - end = groupValues[1] - } - relType = groupValues[2] - props = toProperties(groupValues[4]) - } - - return RelationshipPatternMetaData( - startPattern = start, endPattern = end, relType = relType, properties = props) - } - } - } - - fun parse( - pattern: String, - mergeNodeProps: Boolean, - mergeRelProps: Boolean - ): RelationshipPatternConfiguration { - val isCypherPattern = pattern.startsWith("(") - val regex = - if (isCypherPattern) { - cypherRelationshipPatternConfigured - } else { - simpleRelationshipPatternConfigured - } - val matcher = regex.matchEntire(pattern) - if (matcher == null) { - throw IllegalArgumentException("The Relationship pattern $pattern is invalid") - } else { - val isLeftToRight = - (!isCypherPattern || isUndirected(matcher) || isDirectedToRight(matcher)) - val isRightToLeft = if (isCypherPattern) isDirectedToLeft(matcher) else false - - if (!isLeftToRight && !isRightToLeft) { - throw IllegalArgumentException( - "The Relationship pattern $pattern has an invalid direction") - } - - val metadata = - RelationshipPatternMetaData.create(isCypherPattern, isLeftToRight, matcher.groupValues) - - val start = - try { - getNodeConf(metadata.startPattern, mergeNodeProps) - } catch (e: Exception) { - throw IllegalArgumentException("The Relationship pattern $pattern is invalid") - } - val end = - try { - getNodeConf(metadata.endPattern, mergeNodeProps) - } catch (e: Exception) { - throw IllegalArgumentException("The Relationship pattern $pattern is invalid") - } - val type = getPatternConfiguredType(metadata.properties) - isHomogeneousPattern(type, metadata.properties, pattern, "Relationship") - val cleanedProperties = cleanProperties(type, metadata.properties) - return RelationshipPatternConfiguration( - start = start, - end = end, - relType = metadata.relType, - properties = cleanedProperties, - type = type, - mergeProperties = mergeRelProps) - } - } - - private fun isDirectedToLeft(matcher: MatchResult) = - (matcher.groupValues[2] == "<" && matcher.groupValues[6] == "") - - private fun isDirectedToRight(matcher: MatchResult) = - (matcher.groupValues[2] == "" && matcher.groupValues[6] == ">") - - private fun isUndirected(matcher: MatchResult) = - (matcher.groupValues[2] == "" && matcher.groupValues[6] == "") - } -} diff --git a/common/src/main/kotlin/org/neo4j/connectors/kafka/service/sink/strategy/RelationshipPatternIngestionStrategy.kt b/common/src/main/kotlin/org/neo4j/connectors/kafka/service/sink/strategy/RelationshipPatternIngestionStrategy.kt deleted file mode 100644 index 5bbab857b..000000000 --- a/common/src/main/kotlin/org/neo4j/connectors/kafka/service/sink/strategy/RelationshipPatternIngestionStrategy.kt +++ /dev/null @@ -1,151 +0,0 @@ -/* - * Copyright (c) "Neo4j" - * Neo4j Sweden AB [https://neo4j.com] - * - * 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 org.neo4j.connectors.kafka.service.sink.strategy - -import org.neo4j.connectors.kafka.extensions.flatten -import org.neo4j.connectors.kafka.service.StreamsSinkEntity -import org.neo4j.connectors.kafka.utils.IngestionUtils.containsProp -import org.neo4j.connectors.kafka.utils.IngestionUtils.getLabelsAsString -import org.neo4j.connectors.kafka.utils.IngestionUtils.getNodeMergeKeys -import org.neo4j.connectors.kafka.utils.JSONUtils -import org.neo4j.connectors.kafka.utils.StreamsUtils - -class RelationshipPatternIngestionStrategy( - private val relationshipPatternConfiguration: RelationshipPatternConfiguration -) : IngestionStrategy { - - private val mergeRelationshipTemplate: String = - """ - |${StreamsUtils.UNWIND} - |MERGE (start${getLabelsAsString(relationshipPatternConfiguration.start.labels)}{${ - getNodeMergeKeys("start.keys", relationshipPatternConfiguration.start.keys) - }}) - |SET start ${if (relationshipPatternConfiguration.mergeProperties) "+" else ""}= event.start.properties - |SET start += event.start.keys - |MERGE (end${getLabelsAsString(relationshipPatternConfiguration.end.labels)}{${ - getNodeMergeKeys("end.keys", relationshipPatternConfiguration.end.keys) - }}) - |SET end ${if (relationshipPatternConfiguration.mergeProperties) "+" else ""}= event.end.properties - |SET end += event.end.keys - |MERGE (start)-[r:${relationshipPatternConfiguration.relType}]->(end) - |SET r ${if (relationshipPatternConfiguration.mergeProperties) "+" else ""}= event.properties - """ - .trimMargin() - - private val deleteRelationshipTemplate: String = - """ - |${StreamsUtils.UNWIND} - |MATCH (start${getLabelsAsString(relationshipPatternConfiguration.start.labels)}{${ - getNodeMergeKeys("start.keys", relationshipPatternConfiguration.start.keys) - }}) - |MATCH (end${getLabelsAsString(relationshipPatternConfiguration.end.labels)}{${ - getNodeMergeKeys("end.keys", relationshipPatternConfiguration.end.keys) - }}) - |MATCH (start)-[r:${relationshipPatternConfiguration.relType}]->(end) - |DELETE r - """ - .trimMargin() - - override fun mergeNodeEvents(events: Collection): List { - return emptyList() - } - - override fun deleteNodeEvents(events: Collection): List { - return emptyList() - } - - override fun mergeRelationshipEvents(events: Collection): List { - val data = - events - .mapNotNull { if (it.value != null) JSONUtils.asMap(it.value) else null } - .mapNotNull { props -> - val properties = props.flatten() - val containsKeys = - relationshipPatternConfiguration.start.keys.all { properties.containsKey(it) } && - relationshipPatternConfiguration.end.keys.all { properties.containsKey(it) } - if (containsKeys) { - val filteredProperties = - when (relationshipPatternConfiguration.type) { - PatternConfigurationType.ALL -> - properties.filterKeys { isRelationshipProperty(it) } - PatternConfigurationType.EXCLUDE -> - properties.filterKeys { - val containsProp = - containsProp(it, relationshipPatternConfiguration.properties) - isRelationshipProperty(it) && !containsProp - } - PatternConfigurationType.INCLUDE -> - properties.filterKeys { - val containsProp = - containsProp(it, relationshipPatternConfiguration.properties) - isRelationshipProperty(it) && containsProp - } - } - val startConf = relationshipPatternConfiguration.start - val endConf = relationshipPatternConfiguration.end - - val start = NodePatternIngestionStrategy.toData(startConf, props) - val end = NodePatternIngestionStrategy.toData(endConf, props) - - mapOf("start" to start, "end" to end, "properties" to filteredProperties) - } else { - null - } - } - return if (data.isEmpty()) { - emptyList() - } else { - listOf(QueryEvents(mergeRelationshipTemplate, data)) - } - } - - private fun isRelationshipProperty(propertyName: String): Boolean { - return (!relationshipPatternConfiguration.start.keys.contains(propertyName) && - !relationshipPatternConfiguration.start.properties.contains(propertyName) && - !relationshipPatternConfiguration.end.keys.contains(propertyName) && - !relationshipPatternConfiguration.end.properties.contains(propertyName)) - } - - override fun deleteRelationshipEvents(events: Collection): List { - val data = - events - .filter { it.value == null && it.key != null } - .mapNotNull { if (it.key != null) JSONUtils.asMap(it.key) else null } - .mapNotNull { props -> - val properties = props.flatten() - val containsKeys = - relationshipPatternConfiguration.start.keys.all { properties.containsKey(it) } && - relationshipPatternConfiguration.end.keys.all { properties.containsKey(it) } - if (containsKeys) { - val startConf = relationshipPatternConfiguration.start - val endConf = relationshipPatternConfiguration.end - - val start = NodePatternIngestionStrategy.toData(startConf, props) - val end = NodePatternIngestionStrategy.toData(endConf, props) - - mapOf("start" to start, "end" to end) - } else { - null - } - } - return if (data.isEmpty()) { - emptyList() - } else { - listOf(QueryEvents(deleteRelationshipTemplate, data)) - } - } -} diff --git a/common/src/main/kotlin/org/neo4j/connectors/kafka/utils/JSONUtils.kt b/common/src/main/kotlin/org/neo4j/connectors/kafka/utils/JSONUtils.kt index 6fb750597..de65200c4 100644 --- a/common/src/main/kotlin/org/neo4j/connectors/kafka/utils/JSONUtils.kt +++ b/common/src/main/kotlin/org/neo4j/connectors/kafka/utils/JSONUtils.kt @@ -45,7 +45,6 @@ import org.neo4j.connectors.kafka.events.Schema import org.neo4j.connectors.kafka.events.StreamsTransactionEvent import org.neo4j.connectors.kafka.events.StreamsTransactionNodeEvent import org.neo4j.connectors.kafka.events.StreamsTransactionRelationshipEvent -import org.neo4j.connectors.kafka.extensions.asStreamsMap import org.neo4j.driver.Value import org.neo4j.driver.Values import org.neo4j.driver.internal.value.PointValue @@ -119,6 +118,22 @@ class TemporalAccessorSerializer : JsonSerializer() { } } +fun Node.asStreamsMap(): Map { + val nodeMap = this.asMap().toMutableMap() + nodeMap[""] = this.id() + nodeMap[""] = this.labels() + return nodeMap +} + +fun Relationship.asStreamsMap(): Map { + val relMap = this.asMap().toMutableMap() + relMap[""] = this.id() + relMap[""] = this.type() + relMap[""] = this.startNodeId() + relMap[""] = this.endNodeId() + return relMap +} + class DriverNodeSerializer : JsonSerializer() { @Throws(IOException::class, JsonProcessingException::class) override fun serialize(value: Node?, jgen: JsonGenerator, provider: SerializerProvider) { diff --git a/common/src/test/kotlin/org/neo4j/connectors/kafka/service/sink/strategy/PatternConfigurationTest.kt b/common/src/test/kotlin/org/neo4j/connectors/kafka/service/sink/strategy/PatternConfigurationTest.kt deleted file mode 100644 index 98c27b3fb..000000000 --- a/common/src/test/kotlin/org/neo4j/connectors/kafka/service/sink/strategy/PatternConfigurationTest.kt +++ /dev/null @@ -1,580 +0,0 @@ -/* - * Copyright (c) "Neo4j" - * Neo4j Sweden AB [https://neo4j.com] - * - * 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 org.neo4j.connectors.kafka.service.sink.strategy - -import kotlin.test.assertEquals -import kotlin.test.assertFailsWith -import org.junit.jupiter.api.Test - -class NodePatternConfigurationTest { - - @Test - fun `should extract all params`() { - // given - val pattern = "(:LabelA:LabelB{!id,*})" - - // when - val result = NodePatternConfiguration.parse(pattern, false) - - // then - val expected = - NodePatternConfiguration( - keys = setOf("id"), - type = PatternConfigurationType.ALL, - labels = listOf("LabelA", "LabelB"), - properties = emptyList(), - false) - assertEquals(expected, result) - } - - @Test - fun `should extract all fixed params`() { - // given - val pattern = "(:LabelA{!id,foo,bar})" - - // when - val result = NodePatternConfiguration.parse(pattern, false) - - // then - val expected = - NodePatternConfiguration( - keys = setOf("id"), - type = PatternConfigurationType.INCLUDE, - labels = listOf("LabelA"), - properties = listOf("foo", "bar"), - false) - assertEquals(expected, result) - } - - @Test - fun `should extract complex params`() { - // given - val pattern = "(:LabelA{!id,foo.bar})" - - // when - val result = NodePatternConfiguration.parse(pattern, false) - - // then - val expected = - NodePatternConfiguration( - keys = setOf("id"), - type = PatternConfigurationType.INCLUDE, - labels = listOf("LabelA"), - properties = listOf("foo.bar"), - false) - assertEquals(expected, result) - } - - @Test - fun `should extract composite keys with fixed params`() { - // given - val pattern = "(:LabelA{!idA,!idB,foo,bar})" - - // when - val result = NodePatternConfiguration.parse(pattern, false) - - // then - val expected = - NodePatternConfiguration( - keys = setOf("idA", "idB"), - type = PatternConfigurationType.INCLUDE, - labels = listOf("LabelA"), - properties = listOf("foo", "bar"), - false) - assertEquals(expected, result) - } - - @Test - fun `should extract all excluded params`() { - // given - val pattern = "(:LabelA{!id,-foo,-bar})" - - // when - val result = NodePatternConfiguration.parse(pattern, false) - - // then - val expected = - NodePatternConfiguration( - keys = setOf("id"), - type = PatternConfigurationType.EXCLUDE, - labels = listOf("LabelA"), - properties = listOf("foo", "bar"), - false) - assertEquals(expected, result) - } - - @Test - fun `should throw an exception because of mixed configuration`() { - // given - val pattern = "(:LabelA{!id,-foo,bar})" - - val exception = - assertFailsWith(IllegalArgumentException::class) { - NodePatternConfiguration.parse(pattern, false) - } - - assertEquals("The Node pattern $pattern is not homogeneous", exception.message) - } - - @Test - fun `should throw an exception because of invalid pattern`() { - // given - val pattern = "(LabelA{!id,-foo,bar})" - val exception = - assertFailsWith(IllegalArgumentException::class) { - NodePatternConfiguration.parse(pattern, false) - } - - assertEquals("The Node pattern $pattern is invalid", exception.message) - } - - @Test - fun `should throw an exception because the pattern should contains a key`() { - // given - val pattern = "(:LabelA{id,-foo,bar})" - - val exception = - assertFailsWith(IllegalArgumentException::class) { - NodePatternConfiguration.parse(pattern, false) - } - - assertEquals("The Node pattern $pattern must contain at least one key", exception.message) - } - - @Test - fun `should extract all params - simple`() { - // given - val pattern = "LabelA:LabelB{!id,*}" - - // when - val result = NodePatternConfiguration.parse(pattern, false) - - // then - val expected = - NodePatternConfiguration( - keys = setOf("id"), - type = PatternConfigurationType.ALL, - labels = listOf("LabelA", "LabelB"), - properties = emptyList(), - false) - assertEquals(expected, result) - } - - @Test - fun `should extract all fixed params - simple`() { - // given - val pattern = "LabelA{!id,foo,bar}" - - // when - val result = NodePatternConfiguration.parse(pattern, false) - - // then - val expected = - NodePatternConfiguration( - keys = setOf("id"), - type = PatternConfigurationType.INCLUDE, - labels = listOf("LabelA"), - properties = listOf("foo", "bar"), - false) - assertEquals(expected, result) - } - - @Test - fun `should extract complex params - simple`() { - // given - val pattern = "LabelA{!id,foo.bar}" - - // when - val result = NodePatternConfiguration.parse(pattern, false) - - // then - val expected = - NodePatternConfiguration( - keys = setOf("id"), - type = PatternConfigurationType.INCLUDE, - labels = listOf("LabelA"), - properties = listOf("foo.bar"), - false) - assertEquals(expected, result) - } - - @Test - fun `should extract composite keys with fixed params - simple`() { - // given - val pattern = "LabelA{!idA,!idB,foo,bar}" - - // when - val result = NodePatternConfiguration.parse(pattern, false) - - // then - val expected = - NodePatternConfiguration( - keys = setOf("idA", "idB"), - type = PatternConfigurationType.INCLUDE, - labels = listOf("LabelA"), - properties = listOf("foo", "bar"), - false) - assertEquals(expected, result) - } - - @Test - fun `should extract all excluded params - simple`() { - // given - val pattern = "LabelA{!id,-foo,-bar}" - - // when - val result = NodePatternConfiguration.parse(pattern, false) - - // then - val expected = - NodePatternConfiguration( - keys = setOf("id"), - type = PatternConfigurationType.EXCLUDE, - labels = listOf("LabelA"), - properties = listOf("foo", "bar"), - false) - assertEquals(expected, result) - } - - @Test - fun `should throw an exception because of mixed configuration - simple`() { - // given - val pattern = "LabelA{!id,-foo,bar}" - - val exception = - assertFailsWith(IllegalArgumentException::class) { - NodePatternConfiguration.parse(pattern, false) - } - - assertEquals("The Node pattern $pattern is not homogeneous", exception.message) - } - - @Test - fun `should throw an exception because the pattern should contains a key - simple`() { - // given - val pattern = "LabelA{id,-foo,bar}" - - val exception = - assertFailsWith(IllegalArgumentException::class) { - NodePatternConfiguration.parse(pattern, false) - } - - assertEquals("The Node pattern $pattern must contain at least one key", exception.message) - } -} - -class RelationshipPatternConfigurationTest { - - @Test - fun `should extract all params`() { - // given - val startPattern = "LabelA{!id,aa}" - val endPattern = "LabelB{!idB,bb}" - val pattern = "(:$startPattern)-[:REL_TYPE]->(:$endPattern)" - - // when - val result = RelationshipPatternConfiguration.parse(pattern, false, false) - - // then - val start = NodePatternConfiguration.parse(startPattern, false) - val end = NodePatternConfiguration.parse(endPattern, false) - val properties = emptyList() - val relType = "REL_TYPE" - val expected = - RelationshipPatternConfiguration( - start = start, - end = end, - relType = relType, - properties = properties, - type = PatternConfigurationType.ALL, - mergeProperties = false) - assertEquals(expected, result) - } - - @Test - fun `should extract all params with reverse source and target`() { - // given - val startPattern = "LabelA{!id,aa}" - val endPattern = "LabelB{!idB,bb}" - val pattern = "(:$startPattern)<-[:REL_TYPE]-(:$endPattern)" - - // when - val result = RelationshipPatternConfiguration.parse(pattern, false, false) - - // then - val start = NodePatternConfiguration.parse(startPattern, false) - val end = NodePatternConfiguration.parse(endPattern, false) - val properties = emptyList() - val relType = "REL_TYPE" - val expected = - RelationshipPatternConfiguration( - start = end, - end = start, - relType = relType, - properties = properties, - type = PatternConfigurationType.ALL, - mergeProperties = false) - assertEquals(expected, result) - } - - @Test - fun `should extract all fixed params`() { - // given - val startPattern = "LabelA{!id}" - val endPattern = "LabelB{!idB}" - val pattern = "(:$startPattern)-[:REL_TYPE{foo, BAR}]->(:$endPattern)" - - // when - val result = RelationshipPatternConfiguration.parse(pattern, false, false) - - // then - val start = RelationshipPatternConfiguration.getNodeConf(startPattern, false) - val end = RelationshipPatternConfiguration.getNodeConf(endPattern, false) - val properties = listOf("foo", "BAR") - val relType = "REL_TYPE" - val expected = - RelationshipPatternConfiguration( - start = start, - end = end, - relType = relType, - properties = properties, - type = PatternConfigurationType.INCLUDE, - mergeProperties = false) - assertEquals(expected, result) - } - - @Test - fun `should extract complex params`() { - // given - val startPattern = "LabelA{!id}" - val endPattern = "LabelB{!idB}" - val pattern = "(:$startPattern)-[:REL_TYPE{foo.BAR, BAR.foo}]->(:$endPattern)" - - // when - val result = RelationshipPatternConfiguration.parse(pattern, false, false) - - // then - val start = RelationshipPatternConfiguration.getNodeConf(startPattern, false) - val end = RelationshipPatternConfiguration.getNodeConf(endPattern, false) - val properties = listOf("foo.BAR", "BAR.foo") - val relType = "REL_TYPE" - val expected = - RelationshipPatternConfiguration( - start = start, - end = end, - relType = relType, - properties = properties, - type = PatternConfigurationType.INCLUDE, - mergeProperties = false) - assertEquals(expected, result) - } - - @Test - fun `should extract all excluded params`() { - // given - val startPattern = "LabelA{!id}" - val endPattern = "LabelB{!idB}" - val pattern = "(:$startPattern)-[:REL_TYPE{-foo, -BAR}]->(:$endPattern)" - - // when - val result = RelationshipPatternConfiguration.parse(pattern, false, false) - - // then - val start = RelationshipPatternConfiguration.getNodeConf(startPattern, false) - val end = RelationshipPatternConfiguration.getNodeConf(endPattern, false) - val properties = listOf("foo", "BAR") - val relType = "REL_TYPE" - val expected = - RelationshipPatternConfiguration( - start = start, - end = end, - relType = relType, - properties = properties, - type = PatternConfigurationType.EXCLUDE, - mergeProperties = false) - assertEquals(expected, result) - } - - @Test - fun `should throw an exception because of mixed configuration`() { - // given - val pattern = "(:LabelA{!id})-[:REL_TYPE{foo, -BAR}]->(:LabelB{!idB})" - - val exception = - assertFailsWith(IllegalArgumentException::class) { - RelationshipPatternConfiguration.parse(pattern, false, false) - } - - assertEquals("The Relationship pattern $pattern is not homogeneous", exception.message) - } - - @Test - fun `should throw an exception because the pattern should contains nodes with only ids`() { - // given - val pattern = "(:LabelA{id})-[:REL_TYPE{foo,BAR}]->(:LabelB{!idB})" - - val exception = - assertFailsWith(IllegalArgumentException::class) { - RelationshipPatternConfiguration.parse(pattern, false, false) - } - - assertEquals("The Relationship pattern $pattern is invalid", exception.message) - } - - @Test - fun `should throw an exception because the pattern is invalid`() { - // given - val pattern = "(LabelA{!id})-[:REL_TYPE{foo,BAR}]->(:LabelB{!idB})" - - val exception = - assertFailsWith(IllegalArgumentException::class) { - RelationshipPatternConfiguration.parse(pattern, false, false) - } - - assertEquals("The Relationship pattern $pattern is invalid", exception.message) - } - - @Test - fun `should extract all params - simple`() { - // given - val startPattern = "LabelA{!id,aa}" - val endPattern = "LabelB{!idB,bb}" - val pattern = "$startPattern REL_TYPE $endPattern" - - // when - val result = RelationshipPatternConfiguration.parse(pattern, false, false) - - // then - val start = NodePatternConfiguration.parse(startPattern, false) - val end = NodePatternConfiguration.parse(endPattern, false) - val properties = emptyList() - val relType = "REL_TYPE" - val expected = - RelationshipPatternConfiguration( - start = start, - end = end, - relType = relType, - properties = properties, - type = PatternConfigurationType.ALL, - mergeProperties = false) - assertEquals(expected, result) - } - - @Test - fun `should extract all fixed params - simple`() { - // given - val startPattern = "LabelA{!id}" - val endPattern = "LabelB{!idB}" - val pattern = "$startPattern REL_TYPE{foo, BAR} $endPattern" - - // when - val result = RelationshipPatternConfiguration.parse(pattern, false, false) - - // then - val start = RelationshipPatternConfiguration.getNodeConf(startPattern, false) - val end = RelationshipPatternConfiguration.getNodeConf(endPattern, false) - val properties = listOf("foo", "BAR") - val relType = "REL_TYPE" - val expected = - RelationshipPatternConfiguration( - start = start, - end = end, - relType = relType, - properties = properties, - type = PatternConfigurationType.INCLUDE, - mergeProperties = false) - assertEquals(expected, result) - } - - @Test - fun `should extract complex params - simple`() { - // given - val startPattern = "LabelA{!id}" - val endPattern = "LabelB{!idB}" - val pattern = "$startPattern REL_TYPE{foo.BAR, BAR.foo} $endPattern" - - // when - val result = RelationshipPatternConfiguration.parse(pattern, false, false) - - // then - val start = RelationshipPatternConfiguration.getNodeConf(startPattern, false) - val end = RelationshipPatternConfiguration.getNodeConf(endPattern, false) - val properties = listOf("foo.BAR", "BAR.foo") - val relType = "REL_TYPE" - val expected = - RelationshipPatternConfiguration( - start = start, - end = end, - relType = relType, - properties = properties, - type = PatternConfigurationType.INCLUDE, - mergeProperties = false) - assertEquals(expected, result) - } - - @Test - fun `should extract all excluded params - simple`() { - // given - val startPattern = "LabelA{!id}" - val endPattern = "LabelB{!idB}" - val pattern = "$startPattern REL_TYPE{-foo, -BAR} $endPattern" - - // when - val result = RelationshipPatternConfiguration.parse(pattern, false, false) - - // then - val start = RelationshipPatternConfiguration.getNodeConf(startPattern, false) - val end = RelationshipPatternConfiguration.getNodeConf(endPattern, false) - val properties = listOf("foo", "BAR") - val relType = "REL_TYPE" - val expected = - RelationshipPatternConfiguration( - start = start, - end = end, - relType = relType, - properties = properties, - type = PatternConfigurationType.EXCLUDE, - mergeProperties = false) - assertEquals(expected, result) - } - - @Test - fun `should throw an exception because of mixed configuration - simple`() { - // given - val pattern = "LabelA{!id} REL_TYPE{foo, -BAR} LabelB{!idB}" - - val exception = - assertFailsWith(IllegalArgumentException::class) { - RelationshipPatternConfiguration.parse(pattern, false, false) - } - - assertEquals("The Relationship pattern $pattern is not homogeneous", exception.message) - } - - @Test - fun `should throw an exception because the pattern should contains nodes with only ids - simple`() { - // given - val pattern = "LabelA{id} REL_TYPE{foo,BAR} LabelB{!idB}" - - val exception = - assertFailsWith(IllegalArgumentException::class) { - RelationshipPatternConfiguration.parse(pattern, false, false) - } - - assertEquals("The Relationship pattern $pattern is invalid", exception.message) - } -} diff --git a/sink/src/main/kotlin/org/neo4j/connectors/kafka/sink/DeprecatedNeo4jSinkConfiguration.kt b/sink/src/main/kotlin/org/neo4j/connectors/kafka/sink/DeprecatedNeo4jSinkConfiguration.kt index 8e7417ac5..e31c5285f 100644 --- a/sink/src/main/kotlin/org/neo4j/connectors/kafka/sink/DeprecatedNeo4jSinkConfiguration.kt +++ b/sink/src/main/kotlin/org/neo4j/connectors/kafka/sink/DeprecatedNeo4jSinkConfiguration.kt @@ -20,7 +20,7 @@ import org.apache.kafka.common.config.ConfigDef import org.neo4j.connectors.kafka.configuration.ConfigGroup import org.neo4j.connectors.kafka.configuration.DeprecatedNeo4jConfiguration import org.neo4j.connectors.kafka.configuration.helpers.ConfigKeyBuilder -import org.neo4j.connectors.kafka.service.sink.strategy.SourceIdIngestionStrategyConfig +import org.neo4j.connectors.kafka.sink.strategy.legacy.SourceIdIngestionStrategyConfig import org.neo4j.connectors.kafka.utils.PropertiesUtil @Deprecated("use org.neo4j.connectors.kafka.sink.SinkConfiguration") diff --git a/sink/src/main/kotlin/org/neo4j/connectors/kafka/sink/DeprecatedNeo4jSinkService.kt b/sink/src/main/kotlin/org/neo4j/connectors/kafka/sink/DeprecatedNeo4jSinkService.kt index 1b02fafc5..6432531b6 100644 --- a/sink/src/main/kotlin/org/neo4j/connectors/kafka/sink/DeprecatedNeo4jSinkService.kt +++ b/sink/src/main/kotlin/org/neo4j/connectors/kafka/sink/DeprecatedNeo4jSinkService.kt @@ -25,8 +25,8 @@ import kotlinx.coroutines.awaitAll import kotlinx.coroutines.runBlocking import org.apache.kafka.connect.errors.ConnectException import org.neo4j.connectors.kafka.extensions.errors -import org.neo4j.connectors.kafka.service.StreamsSinkEntity -import org.neo4j.connectors.kafka.service.StreamsSinkService +import org.neo4j.connectors.kafka.sink.strategy.legacy.StreamsSinkEntity +import org.neo4j.connectors.kafka.sink.strategy.legacy.StreamsSinkService import org.neo4j.connectors.kafka.utils.retryForException import org.neo4j.driver.Bookmark import org.neo4j.driver.TransactionConfig diff --git a/sink/src/main/kotlin/org/neo4j/connectors/kafka/sink/DeprecatedNeo4jSinkTask.kt b/sink/src/main/kotlin/org/neo4j/connectors/kafka/sink/DeprecatedNeo4jSinkTask.kt index 901de7a64..ec585c7eb 100644 --- a/sink/src/main/kotlin/org/neo4j/connectors/kafka/sink/DeprecatedNeo4jSinkTask.kt +++ b/sink/src/main/kotlin/org/neo4j/connectors/kafka/sink/DeprecatedNeo4jSinkTask.kt @@ -21,10 +21,10 @@ import kotlinx.coroutines.ObsoleteCoroutinesApi import org.apache.kafka.connect.sink.SinkRecord import org.apache.kafka.connect.sink.SinkTask import org.neo4j.connectors.kafka.configuration.helpers.VersionUtil -import org.neo4j.connectors.kafka.extensions.asProperties import org.neo4j.connectors.kafka.service.errors.ErrorData import org.neo4j.connectors.kafka.service.errors.ErrorService import org.neo4j.connectors.kafka.service.errors.KafkaErrorService +import org.neo4j.connectors.kafka.sink.strategy.legacy.asProperties import org.neo4j.connectors.kafka.utils.StreamsUtils import org.slf4j.Logger import org.slf4j.LoggerFactory diff --git a/sink/src/main/kotlin/org/neo4j/connectors/kafka/sink/EventBuilder.kt b/sink/src/main/kotlin/org/neo4j/connectors/kafka/sink/EventBuilder.kt index 49bc2f8ef..62eacf792 100644 --- a/sink/src/main/kotlin/org/neo4j/connectors/kafka/sink/EventBuilder.kt +++ b/sink/src/main/kotlin/org/neo4j/connectors/kafka/sink/EventBuilder.kt @@ -17,7 +17,7 @@ package org.neo4j.connectors.kafka.sink import org.apache.kafka.connect.sink.SinkRecord -import org.neo4j.connectors.kafka.service.StreamsSinkEntity +import org.neo4j.connectors.kafka.sink.strategy.legacy.StreamsSinkEntity import org.neo4j.connectors.kafka.sink.utils.toStreamsSinkEntity class EventBuilder { diff --git a/sink/src/main/kotlin/org/neo4j/connectors/kafka/sink/Neo4jStrategyStorage.kt b/sink/src/main/kotlin/org/neo4j/connectors/kafka/sink/Neo4jStrategyStorage.kt index e459dcb1a..b537cb914 100644 --- a/sink/src/main/kotlin/org/neo4j/connectors/kafka/sink/Neo4jStrategyStorage.kt +++ b/sink/src/main/kotlin/org/neo4j/connectors/kafka/sink/Neo4jStrategyStorage.kt @@ -16,22 +16,22 @@ */ package org.neo4j.connectors.kafka.sink -import org.neo4j.connectors.kafka.service.StreamsStrategyStorage -import org.neo4j.connectors.kafka.service.TopicType -import org.neo4j.connectors.kafka.service.sink.strategy.CUDIngestionStrategy -import org.neo4j.connectors.kafka.service.sink.strategy.CypherTemplateStrategy -import org.neo4j.connectors.kafka.service.sink.strategy.IngestionStrategy -import org.neo4j.connectors.kafka.service.sink.strategy.NodePatternIngestionStrategy -import org.neo4j.connectors.kafka.service.sink.strategy.RelationshipPatternIngestionStrategy -import org.neo4j.connectors.kafka.service.sink.strategy.SchemaIngestionStrategy -import org.neo4j.connectors.kafka.service.sink.strategy.SourceIdIngestionStrategy +import org.neo4j.connectors.kafka.sink.strategy.legacy.CUDIngestionStrategy +import org.neo4j.connectors.kafka.sink.strategy.legacy.CypherTemplateStrategy +import org.neo4j.connectors.kafka.sink.strategy.legacy.IngestionStrategy +import org.neo4j.connectors.kafka.sink.strategy.legacy.NodePatternIngestionStrategy +import org.neo4j.connectors.kafka.sink.strategy.legacy.RelationshipPatternIngestionStrategy +import org.neo4j.connectors.kafka.sink.strategy.legacy.SchemaIngestionStrategy +import org.neo4j.connectors.kafka.sink.strategy.legacy.SourceIdIngestionStrategy +import org.neo4j.connectors.kafka.sink.strategy.legacy.StreamsStrategyStorage +import org.neo4j.connectors.kafka.sink.strategy.legacy.TopicType class Neo4jStrategyStorage(val config: SinkConfiguration) : StreamsStrategyStorage() { private val topicConfigMap = config.topics.asMap() @Suppress("UNCHECKED_CAST") override fun getTopicType(topic: String): TopicType? = - TopicType.values().firstOrNull { topicType -> + TopicType.entries.firstOrNull { topicType -> when (val topicConfig = topicConfigMap.getOrDefault(topicType, emptyList())) { is Collection<*> -> topicConfig.contains(topic) is Map<*, *> -> topicConfig.containsKey(topic) @@ -46,9 +46,13 @@ class Neo4jStrategyStorage(val config: SinkConfiguration) : StreamsStrategyStora TopicType.CDC_SCHEMA -> SchemaIngestionStrategy() TopicType.CUD -> CUDIngestionStrategy() TopicType.PATTERN_NODE -> - NodePatternIngestionStrategy(config.topics.nodePatternTopics.getValue(topic)) + NodePatternIngestionStrategy( + config.topics.nodePatternTopics.getValue(topic), config.topics.mergeNodeProperties) TopicType.PATTERN_RELATIONSHIP -> - RelationshipPatternIngestionStrategy(config.topics.relPatternTopics.getValue(topic)) + RelationshipPatternIngestionStrategy( + config.topics.relPatternTopics.getValue(topic), + config.topics.mergeNodeProperties, + config.topics.mergeRelationshipProperties) TopicType.CYPHER -> CypherTemplateStrategy(config.topics.cypherTopics.getValue(topic)) null -> throw RuntimeException("Topic Type not Found") } diff --git a/sink/src/main/kotlin/org/neo4j/connectors/kafka/sink/SinkConfiguration.kt b/sink/src/main/kotlin/org/neo4j/connectors/kafka/sink/SinkConfiguration.kt index 39deb0090..f5e6fcd93 100644 --- a/sink/src/main/kotlin/org/neo4j/connectors/kafka/sink/SinkConfiguration.kt +++ b/sink/src/main/kotlin/org/neo4j/connectors/kafka/sink/SinkConfiguration.kt @@ -33,8 +33,8 @@ import org.neo4j.connectors.kafka.configuration.helpers.SIMPLE_DURATION_PATTERN import org.neo4j.connectors.kafka.configuration.helpers.Validators import org.neo4j.connectors.kafka.configuration.helpers.parseSimpleString import org.neo4j.connectors.kafka.configuration.helpers.toSimpleString -import org.neo4j.connectors.kafka.service.TopicType -import org.neo4j.connectors.kafka.service.sink.strategy.SourceIdIngestionStrategyConfig +import org.neo4j.connectors.kafka.sink.strategy.legacy.SourceIdIngestionStrategyConfig +import org.neo4j.connectors.kafka.sink.strategy.legacy.TopicType import org.neo4j.connectors.kafka.sink.utils.TopicUtils import org.neo4j.connectors.kafka.sink.utils.Topics import org.neo4j.connectors.kafka.utils.PropertiesUtil @@ -122,8 +122,7 @@ class SinkConfiguration(originals: Map) : ?: emptyList() init { - // TODO: reinstate and fix issue - // validateAllTopics(originals) + validateAllTopics(originals) } override fun userAgentComment(): String = diff --git a/sink/src/main/kotlin/org/neo4j/connectors/kafka/sink/strategy/CdcHandler.kt b/sink/src/main/kotlin/org/neo4j/connectors/kafka/sink/strategy/CdcHandler.kt index 9b9c2c968..eb6f751db 100644 --- a/sink/src/main/kotlin/org/neo4j/connectors/kafka/sink/strategy/CdcHandler.kt +++ b/sink/src/main/kotlin/org/neo4j/connectors/kafka/sink/strategy/CdcHandler.kt @@ -26,8 +26,8 @@ import org.neo4j.connectors.kafka.data.StreamsTransactionEventExtensions.toChang import org.neo4j.connectors.kafka.sink.ChangeQuery import org.neo4j.connectors.kafka.sink.SinkMessage import org.neo4j.connectors.kafka.sink.SinkStrategyHandler +import org.neo4j.connectors.kafka.sink.strategy.legacy.SchemaUtils import org.neo4j.connectors.kafka.sink.utils.toStreamsSinkEntity -import org.neo4j.connectors.kafka.utils.SchemaUtils import org.neo4j.driver.Query import org.slf4j.Logger import org.slf4j.LoggerFactory diff --git a/sink/src/main/kotlin/org/neo4j/connectors/kafka/sink/strategy/CudHandler.kt b/sink/src/main/kotlin/org/neo4j/connectors/kafka/sink/strategy/CudHandler.kt index 8925ed054..4b0287e2e 100644 --- a/sink/src/main/kotlin/org/neo4j/connectors/kafka/sink/strategy/CudHandler.kt +++ b/sink/src/main/kotlin/org/neo4j/connectors/kafka/sink/strategy/CudHandler.kt @@ -16,8 +16,8 @@ */ package org.neo4j.connectors.kafka.sink.strategy -import org.neo4j.connectors.kafka.service.sink.strategy.CUDIngestionStrategy import org.neo4j.connectors.kafka.sink.SinkStrategy +import org.neo4j.connectors.kafka.sink.strategy.legacy.CUDIngestionStrategy class CudHandler(val topic: String, batchSize: Int) : RedirectingHandler(CUDIngestionStrategy(), batchSize) { diff --git a/sink/src/main/kotlin/org/neo4j/connectors/kafka/sink/strategy/NodePatternHandler.kt b/sink/src/main/kotlin/org/neo4j/connectors/kafka/sink/strategy/NodePatternHandler.kt index b736fe570..fd600d8c8 100644 --- a/sink/src/main/kotlin/org/neo4j/connectors/kafka/sink/strategy/NodePatternHandler.kt +++ b/sink/src/main/kotlin/org/neo4j/connectors/kafka/sink/strategy/NodePatternHandler.kt @@ -152,12 +152,12 @@ class NodePatternHandler( ) } else { it.set( - node.asExpression(), - Cypher.property(NAME_EVENT, PROPERTIES), - ) + node.asExpression(), + Cypher.property(NAME_EVENT, PROPERTIES), + ) + .mutate(node.asExpression(), Cypher.property(NAME_EVENT, KEYS)) } } - .mutate(node.asExpression(), Cypher.property(NAME_EVENT, KEYS)) .returning(Cypher.raw("count(${'$'}E)", node.requiredSymbolicName).`as`(NAME_CREATED)) .build() } diff --git a/sink/src/main/kotlin/org/neo4j/connectors/kafka/sink/strategy/PatternHandler.kt b/sink/src/main/kotlin/org/neo4j/connectors/kafka/sink/strategy/PatternHandler.kt index 450107217..795b138c4 100644 --- a/sink/src/main/kotlin/org/neo4j/connectors/kafka/sink/strategy/PatternHandler.kt +++ b/sink/src/main/kotlin/org/neo4j/connectors/kafka/sink/strategy/PatternHandler.kt @@ -19,10 +19,10 @@ package org.neo4j.connectors.kafka.sink.strategy import java.time.Instant import java.time.ZoneOffset import org.apache.kafka.connect.errors.ConnectException -import org.neo4j.connectors.kafka.extensions.flatten import org.neo4j.connectors.kafka.sink.SinkConfiguration import org.neo4j.connectors.kafka.sink.SinkMessage import org.neo4j.connectors.kafka.sink.SinkStrategyHandler +import org.neo4j.connectors.kafka.sink.strategy.legacy.flatten import org.neo4j.connectors.kafka.sink.strategy.pattern.Pattern import org.neo4j.cypherdsl.core.Cypher diff --git a/sink/src/main/kotlin/org/neo4j/connectors/kafka/sink/strategy/RedirectingHandler.kt b/sink/src/main/kotlin/org/neo4j/connectors/kafka/sink/strategy/RedirectingHandler.kt index c5940b7d5..c2ab5ac24 100644 --- a/sink/src/main/kotlin/org/neo4j/connectors/kafka/sink/strategy/RedirectingHandler.kt +++ b/sink/src/main/kotlin/org/neo4j/connectors/kafka/sink/strategy/RedirectingHandler.kt @@ -16,10 +16,10 @@ */ package org.neo4j.connectors.kafka.sink.strategy -import org.neo4j.connectors.kafka.service.sink.strategy.IngestionStrategy import org.neo4j.connectors.kafka.sink.ChangeQuery import org.neo4j.connectors.kafka.sink.SinkMessage import org.neo4j.connectors.kafka.sink.SinkStrategyHandler +import org.neo4j.connectors.kafka.sink.strategy.legacy.IngestionStrategy import org.neo4j.connectors.kafka.sink.utils.toStreamsSinkEntity import org.neo4j.driver.Query diff --git a/sink/src/main/kotlin/org/neo4j/connectors/kafka/sink/strategy/RelationshipPatternHandler.kt b/sink/src/main/kotlin/org/neo4j/connectors/kafka/sink/strategy/RelationshipPatternHandler.kt index 291e61fe4..a6e6da2bc 100644 --- a/sink/src/main/kotlin/org/neo4j/connectors/kafka/sink/strategy/RelationshipPatternHandler.kt +++ b/sink/src/main/kotlin/org/neo4j/connectors/kafka/sink/strategy/RelationshipPatternHandler.kt @@ -187,15 +187,15 @@ class RelationshipPatternHandler( ) } else { it.set( - startNode.asExpression(), - Cypher.property(NAME_EVENT, START, PROPERTIES), - ) + startNode.asExpression(), + Cypher.property(NAME_EVENT, START, PROPERTIES), + ) + .mutate( + startNode.asExpression(), + Cypher.property(NAME_EVENT, START, KEYS), + ) } } - .mutate( - startNode.asExpression(), - Cypher.property(NAME_EVENT, START, KEYS), - ) .merge(endNode) .let { if (mergeNodeProperties) { @@ -205,15 +205,15 @@ class RelationshipPatternHandler( ) } else { it.set( - endNode.asExpression(), - Cypher.property(NAME_EVENT, END, PROPERTIES), - ) + endNode.asExpression(), + Cypher.property(NAME_EVENT, END, PROPERTIES), + ) + .mutate( + endNode.asExpression(), + Cypher.property(NAME_EVENT, END, KEYS), + ) } } - .mutate( - endNode.asExpression(), - Cypher.property(NAME_EVENT, END, KEYS), - ) .merge(relationship) .let { if (mergeRelationshipProperties) { @@ -223,12 +223,12 @@ class RelationshipPatternHandler( ) } else { it.set( - relationship.asExpression(), - Cypher.property(NAME_EVENT, PROPERTIES), - ) + relationship.asExpression(), + Cypher.property(NAME_EVENT, PROPERTIES), + ) + .mutate(relationship.asExpression(), Cypher.property(NAME_EVENT, KEYS)) } } - .mutate(relationship.asExpression(), Cypher.property(NAME_EVENT, KEYS)) .returning( Cypher.raw("count(${'$'}E)", relationship.requiredSymbolicName).`as`(NAME_CREATED), ) diff --git a/common/src/main/kotlin/org/neo4j/connectors/kafka/service/sink/strategy/CUDIngestionStrategy.kt b/sink/src/main/kotlin/org/neo4j/connectors/kafka/sink/strategy/legacy/CUDIngestionStrategy.kt similarity index 96% rename from common/src/main/kotlin/org/neo4j/connectors/kafka/service/sink/strategy/CUDIngestionStrategy.kt rename to sink/src/main/kotlin/org/neo4j/connectors/kafka/sink/strategy/legacy/CUDIngestionStrategy.kt index fd333dcfb..0472a50a9 100644 --- a/common/src/main/kotlin/org/neo4j/connectors/kafka/service/sink/strategy/CUDIngestionStrategy.kt +++ b/sink/src/main/kotlin/org/neo4j/connectors/kafka/sink/strategy/legacy/CUDIngestionStrategy.kt @@ -14,16 +14,14 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.neo4j.connectors.kafka.service.sink.strategy +package org.neo4j.connectors.kafka.sink.strategy.legacy import java.util.Locale import org.neo4j.connectors.kafka.events.EntityType -import org.neo4j.connectors.kafka.extensions.quote -import org.neo4j.connectors.kafka.service.StreamsSinkEntity -import org.neo4j.connectors.kafka.service.sink.strategy.CUDIngestionStrategy.Companion.FROM_KEY -import org.neo4j.connectors.kafka.service.sink.strategy.CUDIngestionStrategy.Companion.TO_KEY -import org.neo4j.connectors.kafka.utils.IngestionUtils.getLabelsAsString -import org.neo4j.connectors.kafka.utils.IngestionUtils.getNodeKeysAsString +import org.neo4j.connectors.kafka.sink.strategy.legacy.CUDIngestionStrategy.Companion.FROM_KEY +import org.neo4j.connectors.kafka.sink.strategy.legacy.CUDIngestionStrategy.Companion.TO_KEY +import org.neo4j.connectors.kafka.sink.strategy.legacy.IngestionUtils.getLabelsAsString +import org.neo4j.connectors.kafka.sink.strategy.legacy.IngestionUtils.getNodeKeysAsString import org.neo4j.connectors.kafka.utils.JSONUtils import org.neo4j.connectors.kafka.utils.StreamsUtils diff --git a/common/src/main/kotlin/org/neo4j/connectors/kafka/extensions/CommonExtensions.kt b/sink/src/main/kotlin/org/neo4j/connectors/kafka/sink/strategy/legacy/CommonExtensions.kt similarity index 83% rename from common/src/main/kotlin/org/neo4j/connectors/kafka/extensions/CommonExtensions.kt rename to sink/src/main/kotlin/org/neo4j/connectors/kafka/sink/strategy/legacy/CommonExtensions.kt index 9b5418866..55b726f50 100644 --- a/common/src/main/kotlin/org/neo4j/connectors/kafka/extensions/CommonExtensions.kt +++ b/sink/src/main/kotlin/org/neo4j/connectors/kafka/sink/strategy/legacy/CommonExtensions.kt @@ -14,7 +14,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.neo4j.connectors.kafka.extensions +package org.neo4j.connectors.kafka.sink.strategy.legacy import java.nio.ByteBuffer import java.util.* @@ -22,10 +22,7 @@ import javax.lang.model.SourceVersion import org.apache.kafka.clients.consumer.ConsumerRecord import org.apache.kafka.clients.consumer.OffsetAndMetadata import org.apache.kafka.common.TopicPartition -import org.neo4j.connectors.kafka.service.StreamsSinkEntity import org.neo4j.connectors.kafka.utils.JSONUtils -import org.neo4j.driver.types.Node -import org.neo4j.driver.types.Relationship fun Map.getInt(name: String, defaultValue: Int) = this.get(name)?.toInt() ?: defaultValue @@ -37,22 +34,6 @@ fun Map<*, *>.asProperties() = properties } -fun Node.asStreamsMap(): Map { - val nodeMap = this.asMap().toMutableMap() - nodeMap[""] = this.id() - nodeMap[""] = this.labels() - return nodeMap -} - -fun Relationship.asStreamsMap(): Map { - val relMap = this.asMap().toMutableMap() - relMap[""] = this.id() - relMap[""] = this.type() - relMap[""] = this.startNodeId() - relMap[""] = this.endNodeId() - return relMap -} - fun String.toPointCase(): String { return this.split("(?<=[a-z])(?=[A-Z])".toRegex()) .joinToString(separator = ".") diff --git a/common/src/main/kotlin/org/neo4j/connectors/kafka/service/sink/strategy/CypherTemplateStrategy.kt b/sink/src/main/kotlin/org/neo4j/connectors/kafka/sink/strategy/legacy/CypherTemplateStrategy.kt similarity index 92% rename from common/src/main/kotlin/org/neo4j/connectors/kafka/service/sink/strategy/CypherTemplateStrategy.kt rename to sink/src/main/kotlin/org/neo4j/connectors/kafka/sink/strategy/legacy/CypherTemplateStrategy.kt index ce136fce2..3f4bf7a45 100644 --- a/common/src/main/kotlin/org/neo4j/connectors/kafka/service/sink/strategy/CypherTemplateStrategy.kt +++ b/sink/src/main/kotlin/org/neo4j/connectors/kafka/sink/strategy/legacy/CypherTemplateStrategy.kt @@ -14,9 +14,8 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.neo4j.connectors.kafka.service.sink.strategy +package org.neo4j.connectors.kafka.sink.strategy.legacy -import org.neo4j.connectors.kafka.service.StreamsSinkEntity import org.neo4j.connectors.kafka.utils.StreamsUtils class CypherTemplateStrategy(query: String) : IngestionStrategy { diff --git a/common/src/main/kotlin/org/neo4j/connectors/kafka/service/sink/strategy/IngestionStrategy.kt b/sink/src/main/kotlin/org/neo4j/connectors/kafka/sink/strategy/legacy/IngestionStrategy.kt similarity index 94% rename from common/src/main/kotlin/org/neo4j/connectors/kafka/service/sink/strategy/IngestionStrategy.kt rename to sink/src/main/kotlin/org/neo4j/connectors/kafka/sink/strategy/legacy/IngestionStrategy.kt index 8d9648e23..416c6f7e0 100644 --- a/common/src/main/kotlin/org/neo4j/connectors/kafka/service/sink/strategy/IngestionStrategy.kt +++ b/sink/src/main/kotlin/org/neo4j/connectors/kafka/sink/strategy/legacy/IngestionStrategy.kt @@ -14,11 +14,10 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.neo4j.connectors.kafka.service.sink.strategy +package org.neo4j.connectors.kafka.sink.strategy.legacy import org.neo4j.connectors.kafka.events.Constraint import org.neo4j.connectors.kafka.events.RelationshipPayload -import org.neo4j.connectors.kafka.service.StreamsSinkEntity data class QueryEvents(val query: String, val events: List>) diff --git a/common/src/main/kotlin/org/neo4j/connectors/kafka/utils/IngestionUtils.kt b/sink/src/main/kotlin/org/neo4j/connectors/kafka/sink/strategy/legacy/IngestionUtils.kt similarity index 91% rename from common/src/main/kotlin/org/neo4j/connectors/kafka/utils/IngestionUtils.kt rename to sink/src/main/kotlin/org/neo4j/connectors/kafka/sink/strategy/legacy/IngestionUtils.kt index 9b9a4ae64..e7a8f5031 100644 --- a/common/src/main/kotlin/org/neo4j/connectors/kafka/utils/IngestionUtils.kt +++ b/sink/src/main/kotlin/org/neo4j/connectors/kafka/sink/strategy/legacy/IngestionUtils.kt @@ -14,9 +14,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.neo4j.connectors.kafka.utils - -import org.neo4j.connectors.kafka.extensions.quote +package org.neo4j.connectors.kafka.sink.strategy.legacy object IngestionUtils { const val labelSeparator = ":" @@ -44,7 +42,7 @@ object IngestionUtils { } .joinToString(keySeparator) - fun containsProp(key: String, properties: List): Boolean = + fun containsProp(key: String, properties: Collection): Boolean = if (key.contains(".")) { properties.contains(key) || properties.any { key.startsWith("$it.") } } else { diff --git a/common/src/main/kotlin/org/neo4j/connectors/kafka/service/sink/strategy/NodePatternIngestionStrategy.kt b/sink/src/main/kotlin/org/neo4j/connectors/kafka/sink/strategy/legacy/NodePatternIngestionStrategy.kt similarity index 52% rename from common/src/main/kotlin/org/neo4j/connectors/kafka/service/sink/strategy/NodePatternIngestionStrategy.kt rename to sink/src/main/kotlin/org/neo4j/connectors/kafka/sink/strategy/legacy/NodePatternIngestionStrategy.kt index 9f3ecb26c..466b9cdcd 100644 --- a/common/src/main/kotlin/org/neo4j/connectors/kafka/service/sink/strategy/NodePatternIngestionStrategy.kt +++ b/sink/src/main/kotlin/org/neo4j/connectors/kafka/sink/strategy/legacy/NodePatternIngestionStrategy.kt @@ -14,26 +14,25 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.neo4j.connectors.kafka.service.sink.strategy +package org.neo4j.connectors.kafka.sink.strategy.legacy -import org.neo4j.connectors.kafka.extensions.flatten -import org.neo4j.connectors.kafka.service.StreamsSinkEntity -import org.neo4j.connectors.kafka.utils.IngestionUtils.containsProp -import org.neo4j.connectors.kafka.utils.IngestionUtils.getLabelsAsString -import org.neo4j.connectors.kafka.utils.IngestionUtils.getNodeMergeKeys +import org.neo4j.connectors.kafka.sink.strategy.legacy.IngestionUtils.containsProp +import org.neo4j.connectors.kafka.sink.strategy.legacy.IngestionUtils.getLabelsAsString +import org.neo4j.connectors.kafka.sink.strategy.legacy.IngestionUtils.getNodeMergeKeys +import org.neo4j.connectors.kafka.sink.strategy.pattern.NodePattern import org.neo4j.connectors.kafka.utils.JSONUtils import org.neo4j.connectors.kafka.utils.StreamsUtils -class NodePatternIngestionStrategy(private val nodePatternConfiguration: NodePatternConfiguration) : +class NodePatternIngestionStrategy(private val pattern: NodePattern, mergeProperties: Boolean) : IngestionStrategy { private val mergeNodeTemplate: String = """ |${StreamsUtils.UNWIND} - |MERGE (n${getLabelsAsString(nodePatternConfiguration.labels)}{${ - getNodeMergeKeys("keys", nodePatternConfiguration.keys) + |MERGE (n${getLabelsAsString(pattern.labels)}{${ + getNodeMergeKeys("keys", pattern.keyProperties.map { it.from }.toSet()) }}) - |SET n ${if (nodePatternConfiguration.mergeProperties) "+" else ""}= event.properties + |SET n ${if (mergeProperties) "+" else ""}= event.properties |SET n += event.keys """ .trimMargin() @@ -41,8 +40,8 @@ class NodePatternIngestionStrategy(private val nodePatternConfiguration: NodePat private val deleteNodeTemplate: String = """ |${StreamsUtils.UNWIND} - |MATCH (n${getLabelsAsString(nodePatternConfiguration.labels)}{${ - getNodeMergeKeys("keys", nodePatternConfiguration.keys) + |MATCH (n${getLabelsAsString(pattern.labels)}{${ + getNodeMergeKeys("keys", pattern.keyProperties.map { it.from }.toSet()) }}) |DETACH DELETE n """ @@ -52,7 +51,7 @@ class NodePatternIngestionStrategy(private val nodePatternConfiguration: NodePat val data = events .mapNotNull { if (it.value != null) JSONUtils.asMap(it.value) else null } - .mapNotNull { toData(nodePatternConfiguration, it) } + .mapNotNull { toData(pattern, it) } return if (data.isEmpty()) { emptyList() } else { @@ -65,7 +64,7 @@ class NodePatternIngestionStrategy(private val nodePatternConfiguration: NodePat events .filter { it.value == null && it.key != null } .mapNotNull { if (it.key != null) JSONUtils.asMap(it.key) else null } - .mapNotNull { toData(nodePatternConfiguration, it, false) } + .mapNotNull { toData(pattern, it, false) } return if (data.isEmpty()) { emptyList() } else { @@ -83,34 +82,47 @@ class NodePatternIngestionStrategy(private val nodePatternConfiguration: NodePat companion object { fun toData( - nodePatternConfiguration: NodePatternConfiguration, + pattern: NodePattern, props: Map, withProperties: Boolean = true ): Map>? { val properties = props.flatten() - val containsKeys = nodePatternConfiguration.keys.all { properties.containsKey(it) } + val keyProperties = pattern.keyProperties.map { it.from }.toSet() + val containsKeys = keyProperties.all { properties.containsKey(it) } return if (containsKeys) { val filteredProperties = - when (nodePatternConfiguration.type) { - PatternConfigurationType.ALL -> - properties.filterKeys { !nodePatternConfiguration.keys.contains(it) } - PatternConfigurationType.EXCLUDE -> - properties.filterKeys { key -> - val containsProp = containsProp(key, nodePatternConfiguration.properties) - !nodePatternConfiguration.keys.contains(key) && !containsProp - } - PatternConfigurationType.INCLUDE -> - properties.filterKeys { key -> - val containsProp = containsProp(key, nodePatternConfiguration.properties) - !nodePatternConfiguration.keys.contains(key) && containsProp - } + buildMap { + if (pattern.includeAllValueProperties) { + putAll(properties.filterKeys { key -> !keyProperties.contains(key) }) + } + + if (pattern.includeProperties.isNotEmpty()) { + putAll( + properties.filterKeys { key -> + val containsProp = + containsProp( + key, + pattern.includeProperties.map { include -> include.from }.toSet()) + !keyProperties.contains(key) && containsProp + }) + } + + if (pattern.excludeProperties.isNotEmpty()) { + keys + .filter { key -> + val excluded = containsProp(key, pattern.excludeProperties) + !keyProperties.contains(key) && excluded + } + .forEach { remove(it) } + } } + if (withProperties) { mapOf( - "keys" to properties.filterKeys { nodePatternConfiguration.keys.contains(it) }, + "keys" to properties.filterKeys { keyProperties.contains(it) }, "properties" to filteredProperties) } else { - mapOf("keys" to properties.filterKeys { nodePatternConfiguration.keys.contains(it) }) + mapOf("keys" to properties.filterKeys { keyProperties.contains(it) }) } } else { null diff --git a/sink/src/main/kotlin/org/neo4j/connectors/kafka/sink/strategy/legacy/RelationshipPatternIngestionStrategy.kt b/sink/src/main/kotlin/org/neo4j/connectors/kafka/sink/strategy/legacy/RelationshipPatternIngestionStrategy.kt new file mode 100644 index 000000000..86ac54807 --- /dev/null +++ b/sink/src/main/kotlin/org/neo4j/connectors/kafka/sink/strategy/legacy/RelationshipPatternIngestionStrategy.kt @@ -0,0 +1,169 @@ +/* + * Copyright (c) "Neo4j" + * Neo4j Sweden AB [https://neo4j.com] + * + * 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 org.neo4j.connectors.kafka.sink.strategy.legacy + +import org.neo4j.connectors.kafka.sink.strategy.legacy.IngestionUtils.containsProp +import org.neo4j.connectors.kafka.sink.strategy.legacy.IngestionUtils.getLabelsAsString +import org.neo4j.connectors.kafka.sink.strategy.legacy.IngestionUtils.getNodeMergeKeys +import org.neo4j.connectors.kafka.sink.strategy.pattern.RelationshipPattern +import org.neo4j.connectors.kafka.utils.JSONUtils +import org.neo4j.connectors.kafka.utils.StreamsUtils + +class RelationshipPatternIngestionStrategy( + private val pattern: RelationshipPattern, + mergeNodeProperties: Boolean, + mergeRelationshipProperties: Boolean, +) : IngestionStrategy { + + private val mergeRelationshipTemplate: String = + """ + |${StreamsUtils.UNWIND} + |MERGE (start${getLabelsAsString(pattern.start.labels)}{${ + getNodeMergeKeys("start.keys", pattern.start.keyProperties.map { it.from }.toSet()) + }}) + |SET start ${if (mergeNodeProperties) "+" else ""}= event.start.properties + |SET start += event.start.keys + |MERGE (end${getLabelsAsString(pattern.end.labels)}{${ + getNodeMergeKeys("end.keys", pattern.end.keyProperties.map { it.from }.toSet()) + }}) + |SET end ${if (mergeNodeProperties) "+" else ""}= event.end.properties + |SET end += event.end.keys + |MERGE (start)-[r:${pattern.type}]->(end) + |SET r ${if (mergeRelationshipProperties) "+" else ""}= event.properties + """ + .trimMargin() + + private val deleteRelationshipTemplate: String = + """ + |${StreamsUtils.UNWIND} + |MATCH (start${getLabelsAsString(pattern.start.labels)}{${ + getNodeMergeKeys("start.keys", pattern.start.keyProperties.map { it.from }.toSet()) + }}) + |MATCH (end${getLabelsAsString(pattern.end.labels)}{${ + getNodeMergeKeys("end.keys", pattern.end.keyProperties.map { it.from }.toSet()) + }}) + |MATCH (start)-[r:${pattern.type}]->(end) + |DELETE r + """ + .trimMargin() + + override fun mergeNodeEvents(events: Collection): List { + return emptyList() + } + + override fun deleteNodeEvents(events: Collection): List { + return emptyList() + } + + override fun mergeRelationshipEvents(events: Collection): List { + val data = + events + .mapNotNull { if (it.value != null) JSONUtils.asMap(it.value) else null } + .mapNotNull { props -> + val properties = props.flatten() + val startKeys = pattern.start.keyProperties.map { it.from }.toSet() + val endKeys = pattern.end.keyProperties.map { it.from }.toSet() + val containsKeys = + startKeys.all { properties.containsKey(it) } && + endKeys.all { properties.containsKey(it) } + if (containsKeys) { + + val filteredProperties = + buildMap { + if (pattern.includeAllValueProperties) { + putAll(properties.filterKeys { key -> isRelationshipProperty(key) }) + } + + if (pattern.includeProperties.isNotEmpty()) { + putAll( + properties.filterKeys { key -> + val containsProp = + containsProp( + key, + pattern.includeProperties + .map { include -> include.from } + .toSet()) + isRelationshipProperty(key) && containsProp + }) + } + + if (pattern.excludeProperties.isNotEmpty()) { + keys + .filter { key -> + val excluded = containsProp(key, pattern.excludeProperties) + isRelationshipProperty(key) && excluded + } + .forEach { remove(it) } + } + } + + val startConf = pattern.start + val endConf = pattern.end + + val start = NodePatternIngestionStrategy.toData(startConf, props) + val end = NodePatternIngestionStrategy.toData(endConf, props) + + mapOf("start" to start, "end" to end, "properties" to filteredProperties) + } else { + null + } + } + return if (data.isEmpty()) { + emptyList() + } else { + listOf(QueryEvents(mergeRelationshipTemplate, data)) + } + } + + private fun isRelationshipProperty(propertyName: String): Boolean { + return (!pattern.start.keyProperties.map { it.from }.contains(propertyName) && + !pattern.start.includeProperties.map { it.from }.contains(propertyName) && + !pattern.end.keyProperties.map { it.from }.contains(propertyName) && + !pattern.end.includeProperties.map { it.from }.contains(propertyName)) + } + + override fun deleteRelationshipEvents(events: Collection): List { + val data = + events + .filter { it.value == null && it.key != null } + .mapNotNull { if (it.key != null) JSONUtils.asMap(it.key) else null } + .mapNotNull { props -> + val properties = props.flatten() + val startKeys = pattern.start.keyProperties.map { it.from }.toSet() + val endKeys = pattern.end.keyProperties.map { it.from }.toSet() + val containsKeys = + startKeys.all { properties.containsKey(it) } && + endKeys.all { properties.containsKey(it) } + if (containsKeys) { + val startConf = pattern.start + val endConf = pattern.end + + val start = NodePatternIngestionStrategy.toData(startConf, props) + val end = NodePatternIngestionStrategy.toData(endConf, props) + + mapOf("start" to start, "end" to end) + } else { + null + } + } + return if (data.isEmpty()) { + emptyList() + } else { + listOf(QueryEvents(deleteRelationshipTemplate, data)) + } + } +} diff --git a/common/src/main/kotlin/org/neo4j/connectors/kafka/service/sink/strategy/SchemaIngestionStrategy.kt b/sink/src/main/kotlin/org/neo4j/connectors/kafka/sink/strategy/legacy/SchemaIngestionStrategy.kt similarity index 95% rename from common/src/main/kotlin/org/neo4j/connectors/kafka/service/sink/strategy/SchemaIngestionStrategy.kt rename to sink/src/main/kotlin/org/neo4j/connectors/kafka/sink/strategy/legacy/SchemaIngestionStrategy.kt index b06806aa1..fed8c7ddf 100644 --- a/common/src/main/kotlin/org/neo4j/connectors/kafka/service/sink/strategy/SchemaIngestionStrategy.kt +++ b/sink/src/main/kotlin/org/neo4j/connectors/kafka/sink/strategy/legacy/SchemaIngestionStrategy.kt @@ -14,7 +14,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.neo4j.connectors.kafka.service.sink.strategy +package org.neo4j.connectors.kafka.sink.strategy.legacy import org.neo4j.connectors.kafka.events.Constraint import org.neo4j.connectors.kafka.events.EntityType @@ -23,12 +23,10 @@ import org.neo4j.connectors.kafka.events.OperationType import org.neo4j.connectors.kafka.events.RelationshipPayload import org.neo4j.connectors.kafka.events.StreamsConstraintType import org.neo4j.connectors.kafka.events.StreamsTransactionEvent -import org.neo4j.connectors.kafka.extensions.quote -import org.neo4j.connectors.kafka.service.StreamsSinkEntity -import org.neo4j.connectors.kafka.utils.IngestionUtils.getLabelsAsString -import org.neo4j.connectors.kafka.utils.IngestionUtils.getNodeKeysAsString -import org.neo4j.connectors.kafka.utils.SchemaUtils.getNodeKeys -import org.neo4j.connectors.kafka.utils.SchemaUtils.toStreamsTransactionEvent +import org.neo4j.connectors.kafka.sink.strategy.legacy.IngestionUtils.getLabelsAsString +import org.neo4j.connectors.kafka.sink.strategy.legacy.IngestionUtils.getNodeKeysAsString +import org.neo4j.connectors.kafka.sink.strategy.legacy.SchemaUtils.getNodeKeys +import org.neo4j.connectors.kafka.sink.strategy.legacy.SchemaUtils.toStreamsTransactionEvent import org.neo4j.connectors.kafka.utils.StreamsUtils class SchemaIngestionStrategy : IngestionStrategy { diff --git a/common/src/main/kotlin/org/neo4j/connectors/kafka/utils/SchemaUtils.kt b/sink/src/main/kotlin/org/neo4j/connectors/kafka/sink/strategy/legacy/SchemaUtils.kt similarity index 96% rename from common/src/main/kotlin/org/neo4j/connectors/kafka/utils/SchemaUtils.kt rename to sink/src/main/kotlin/org/neo4j/connectors/kafka/sink/strategy/legacy/SchemaUtils.kt index 98aad1d17..a9dbe67a6 100644 --- a/common/src/main/kotlin/org/neo4j/connectors/kafka/utils/SchemaUtils.kt +++ b/sink/src/main/kotlin/org/neo4j/connectors/kafka/sink/strategy/legacy/SchemaUtils.kt @@ -14,13 +14,13 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.neo4j.connectors.kafka.utils +package org.neo4j.connectors.kafka.sink.strategy.legacy import org.neo4j.connectors.kafka.events.Constraint import org.neo4j.connectors.kafka.events.RelKeyStrategy import org.neo4j.connectors.kafka.events.StreamsConstraintType import org.neo4j.connectors.kafka.events.StreamsTransactionEvent -import org.neo4j.connectors.kafka.service.StreamsSinkEntity +import org.neo4j.connectors.kafka.utils.JSONUtils object SchemaUtils { fun getNodeKeys( diff --git a/common/src/main/kotlin/org/neo4j/connectors/kafka/service/sink/strategy/SourceIdIngestionStrategy.kt b/sink/src/main/kotlin/org/neo4j/connectors/kafka/sink/strategy/legacy/SourceIdIngestionStrategy.kt similarity index 95% rename from common/src/main/kotlin/org/neo4j/connectors/kafka/service/sink/strategy/SourceIdIngestionStrategy.kt rename to sink/src/main/kotlin/org/neo4j/connectors/kafka/sink/strategy/legacy/SourceIdIngestionStrategy.kt index 5f04d6e4d..7bec21ff1 100644 --- a/common/src/main/kotlin/org/neo4j/connectors/kafka/service/sink/strategy/SourceIdIngestionStrategy.kt +++ b/sink/src/main/kotlin/org/neo4j/connectors/kafka/sink/strategy/legacy/SourceIdIngestionStrategy.kt @@ -14,17 +14,14 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.neo4j.connectors.kafka.service.sink.strategy +package org.neo4j.connectors.kafka.sink.strategy.legacy import org.neo4j.connectors.kafka.events.EntityType import org.neo4j.connectors.kafka.events.NodeChange import org.neo4j.connectors.kafka.events.OperationType import org.neo4j.connectors.kafka.events.RelationshipChange import org.neo4j.connectors.kafka.events.RelationshipPayload -import org.neo4j.connectors.kafka.extensions.quote -import org.neo4j.connectors.kafka.service.StreamsSinkEntity -import org.neo4j.connectors.kafka.utils.IngestionUtils.getLabelsAsString -import org.neo4j.connectors.kafka.utils.SchemaUtils +import org.neo4j.connectors.kafka.sink.strategy.legacy.IngestionUtils.getLabelsAsString import org.neo4j.connectors.kafka.utils.StreamsUtils data class SourceIdIngestionStrategyConfig( diff --git a/common/src/main/kotlin/org/neo4j/connectors/kafka/service/StreamsSinkService.kt b/sink/src/main/kotlin/org/neo4j/connectors/kafka/sink/strategy/legacy/StreamsSinkService.kt similarity index 94% rename from common/src/main/kotlin/org/neo4j/connectors/kafka/service/StreamsSinkService.kt rename to sink/src/main/kotlin/org/neo4j/connectors/kafka/sink/strategy/legacy/StreamsSinkService.kt index fbb50d01d..ea04aeba5 100644 --- a/common/src/main/kotlin/org/neo4j/connectors/kafka/service/StreamsSinkService.kt +++ b/sink/src/main/kotlin/org/neo4j/connectors/kafka/sink/strategy/legacy/StreamsSinkService.kt @@ -14,9 +14,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.neo4j.connectors.kafka.service - -import org.neo4j.connectors.kafka.service.sink.strategy.IngestionStrategy +package org.neo4j.connectors.kafka.sink.strategy.legacy enum class TopicTypeGroup { CYPHER, diff --git a/sink/src/main/kotlin/org/neo4j/connectors/kafka/sink/utils/ConnectExtensionFunctions.kt b/sink/src/main/kotlin/org/neo4j/connectors/kafka/sink/utils/ConnectExtensionFunctions.kt index edda3dee0..731407a8a 100644 --- a/sink/src/main/kotlin/org/neo4j/connectors/kafka/sink/utils/ConnectExtensionFunctions.kt +++ b/sink/src/main/kotlin/org/neo4j/connectors/kafka/sink/utils/ConnectExtensionFunctions.kt @@ -21,10 +21,10 @@ import org.apache.kafka.connect.data.Schema import org.apache.kafka.connect.data.SchemaBuilder import org.apache.kafka.connect.data.Struct import org.apache.kafka.connect.sink.SinkRecord -import org.neo4j.connectors.kafka.extensions.asStreamsMap -import org.neo4j.connectors.kafka.service.StreamsSinkEntity import org.neo4j.connectors.kafka.sink.converters.Neo4jValueConverter +import org.neo4j.connectors.kafka.sink.strategy.legacy.StreamsSinkEntity import org.neo4j.connectors.kafka.utils.JSONUtils +import org.neo4j.connectors.kafka.utils.asStreamsMap import org.neo4j.driver.Record import org.neo4j.driver.types.Node import org.neo4j.driver.types.Point diff --git a/sink/src/main/kotlin/org/neo4j/connectors/kafka/sink/utils/Topics.kt b/sink/src/main/kotlin/org/neo4j/connectors/kafka/sink/utils/Topics.kt index bebc5548a..7db302755 100644 --- a/sink/src/main/kotlin/org/neo4j/connectors/kafka/sink/utils/Topics.kt +++ b/sink/src/main/kotlin/org/neo4j/connectors/kafka/sink/utils/Topics.kt @@ -17,17 +17,18 @@ package org.neo4j.connectors.kafka.sink.utils import kotlin.reflect.jvm.javaType -import org.neo4j.connectors.kafka.service.TopicType -import org.neo4j.connectors.kafka.service.TopicTypeGroup -import org.neo4j.connectors.kafka.service.sink.strategy.CUDIngestionStrategy -import org.neo4j.connectors.kafka.service.sink.strategy.NodePatternConfiguration -import org.neo4j.connectors.kafka.service.sink.strategy.NodePatternIngestionStrategy -import org.neo4j.connectors.kafka.service.sink.strategy.RelationshipPatternConfiguration -import org.neo4j.connectors.kafka.service.sink.strategy.RelationshipPatternIngestionStrategy -import org.neo4j.connectors.kafka.service.sink.strategy.SchemaIngestionStrategy -import org.neo4j.connectors.kafka.service.sink.strategy.SourceIdIngestionStrategy -import org.neo4j.connectors.kafka.service.sink.strategy.SourceIdIngestionStrategyConfig import org.neo4j.connectors.kafka.sink.SinkConfiguration +import org.neo4j.connectors.kafka.sink.strategy.legacy.CUDIngestionStrategy +import org.neo4j.connectors.kafka.sink.strategy.legacy.NodePatternIngestionStrategy +import org.neo4j.connectors.kafka.sink.strategy.legacy.RelationshipPatternIngestionStrategy +import org.neo4j.connectors.kafka.sink.strategy.legacy.SchemaIngestionStrategy +import org.neo4j.connectors.kafka.sink.strategy.legacy.SourceIdIngestionStrategy +import org.neo4j.connectors.kafka.sink.strategy.legacy.SourceIdIngestionStrategyConfig +import org.neo4j.connectors.kafka.sink.strategy.legacy.TopicType +import org.neo4j.connectors.kafka.sink.strategy.legacy.TopicTypeGroup +import org.neo4j.connectors.kafka.sink.strategy.pattern.NodePattern +import org.neo4j.connectors.kafka.sink.strategy.pattern.Pattern +import org.neo4j.connectors.kafka.sink.strategy.pattern.RelationshipPattern @Suppress("UNCHECKED_CAST") data class Topics( @@ -36,8 +37,10 @@ data class Topics( (emptySet() to SourceIdIngestionStrategyConfig()), val cdcSchemaTopics: Set = emptySet(), val cudTopics: Set = emptySet(), - val nodePatternTopics: Map = emptyMap(), - val relPatternTopics: Map = emptyMap(), + val nodePatternTopics: Map = emptyMap(), + val relPatternTopics: Map = emptyMap(), + val mergeNodeProperties: Boolean = false, + val mergeRelationshipProperties: Boolean = false, val invalid: List = emptyList() ) { @@ -76,13 +79,24 @@ data class Topics( val nodePatternTopics = TopicUtils.filterByPrefix( config, SinkConfiguration.PATTERN_NODE_TOPIC_PREFIX, invalidTopics) - .mapValues { NodePatternConfiguration.parse(it.value, mergeNodeProperties) } + .mapValues { + val pattern = Pattern.parse(it.value) + if (pattern !is NodePattern) { + throw IllegalArgumentException( + "Unsupported pattern type for node strategy: $pattern") + } + pattern + } val relPatternTopics = TopicUtils.filterByPrefix( config, SinkConfiguration.PATTERN_RELATIONSHIP_TOPIC_PREFIX, invalidTopics) .mapValues { - RelationshipPatternConfiguration.parse( - it.value, mergeNodeProperties, mergeRelProperties) + val pattern = Pattern.parse(it.value) + if (pattern !is RelationshipPattern) { + throw IllegalArgumentException( + "Unsupported pattern type for relationship strategy: $pattern") + } + pattern } val cdcSourceIdTopics = TopicUtils.splitTopics( @@ -110,7 +124,9 @@ data class Topics( cdcSchemaTopics, cudTopics, nodePatternTopics, - relPatternTopics) + relPatternTopics, + mergeNodeProperties, + mergeRelProperties) } } } @@ -170,12 +186,15 @@ object TopicUtils { TopicType.CDC_SCHEMA -> SchemaIngestionStrategy() TopicType.CUD -> CUDIngestionStrategy() TopicType.PATTERN_NODE -> { - val map = config as Map - map.mapValues { NodePatternIngestionStrategy(it.value) } + val map = config as Map + map.mapValues { NodePatternIngestionStrategy(it.value, topics.mergeNodeProperties) } } TopicType.PATTERN_RELATIONSHIP -> { - val map = config as Map - map.mapValues { RelationshipPatternIngestionStrategy(it.value) } + val map = config as Map + map.mapValues { + RelationshipPatternIngestionStrategy( + it.value, topics.mergeNodeProperties, topics.mergeRelationshipProperties) + } } else -> throw RuntimeException("Unsupported topic type $type") } diff --git a/sink/src/test/kotlin/org/neo4j/connectors/kafka/sink/DeprecatedNeo4jSinkTaskAuraTest.kt b/sink/src/test/kotlin/org/neo4j/connectors/kafka/sink/DeprecatedNeo4jSinkTaskAuraTest.kt index 642bdbbba..39080798d 100644 --- a/sink/src/test/kotlin/org/neo4j/connectors/kafka/sink/DeprecatedNeo4jSinkTaskAuraTest.kt +++ b/sink/src/test/kotlin/org/neo4j/connectors/kafka/sink/DeprecatedNeo4jSinkTaskAuraTest.kt @@ -41,8 +41,8 @@ import org.neo4j.connectors.kafka.events.RelationshipChange import org.neo4j.connectors.kafka.events.RelationshipNodeChange import org.neo4j.connectors.kafka.events.RelationshipPayload import org.neo4j.connectors.kafka.events.StreamsTransactionEvent -import org.neo4j.connectors.kafka.service.sink.strategy.CUDNode -import org.neo4j.connectors.kafka.service.sink.strategy.CUDOperations +import org.neo4j.connectors.kafka.sink.strategy.legacy.CUDNode +import org.neo4j.connectors.kafka.sink.strategy.legacy.CUDOperations import org.neo4j.connectors.kafka.utils.JSONUtils import org.neo4j.driver.AuthTokens import org.neo4j.driver.Driver diff --git a/sink/src/test/kotlin/org/neo4j/connectors/kafka/sink/DeprecatedNeo4jSinkTaskTest.kt b/sink/src/test/kotlin/org/neo4j/connectors/kafka/sink/DeprecatedNeo4jSinkTaskTest.kt index 414381c29..5b354307a 100644 --- a/sink/src/test/kotlin/org/neo4j/connectors/kafka/sink/DeprecatedNeo4jSinkTaskTest.kt +++ b/sink/src/test/kotlin/org/neo4j/connectors/kafka/sink/DeprecatedNeo4jSinkTaskTest.kt @@ -50,12 +50,11 @@ import org.neo4j.connectors.kafka.events.StreamsConstraintType import org.neo4j.connectors.kafka.events.StreamsTransactionEvent import org.neo4j.connectors.kafka.service.errors.ErrorService import org.neo4j.connectors.kafka.service.errors.ProcessingError -import org.neo4j.connectors.kafka.service.sink.strategy.CUDNode -import org.neo4j.connectors.kafka.service.sink.strategy.CUDNodeRel -import org.neo4j.connectors.kafka.service.sink.strategy.CUDOperations -import org.neo4j.connectors.kafka.service.sink.strategy.CUDRelationship -import org.neo4j.connectors.kafka.sink.DeprecatedNeo4jSinkTaskTest.Companion.session import org.neo4j.connectors.kafka.sink.converters.Neo4jValueConverterTest +import org.neo4j.connectors.kafka.sink.strategy.legacy.CUDNode +import org.neo4j.connectors.kafka.sink.strategy.legacy.CUDNodeRel +import org.neo4j.connectors.kafka.sink.strategy.legacy.CUDOperations +import org.neo4j.connectors.kafka.sink.strategy.legacy.CUDRelationship import org.neo4j.connectors.kafka.sink.utils.allLabels import org.neo4j.connectors.kafka.sink.utils.allNodes import org.neo4j.connectors.kafka.sink.utils.allRelationships diff --git a/sink/src/test/kotlin/org/neo4j/connectors/kafka/sink/SinkConfigurationTest.kt b/sink/src/test/kotlin/org/neo4j/connectors/kafka/sink/SinkConfigurationTest.kt index a0b5dd58e..14085dda0 100644 --- a/sink/src/test/kotlin/org/neo4j/connectors/kafka/sink/SinkConfigurationTest.kt +++ b/sink/src/test/kotlin/org/neo4j/connectors/kafka/sink/SinkConfigurationTest.kt @@ -28,7 +28,7 @@ import org.junit.jupiter.params.ParameterizedTest import org.junit.jupiter.params.provider.EnumSource import org.neo4j.connectors.kafka.configuration.DeprecatedNeo4jConfiguration import org.neo4j.connectors.kafka.configuration.Neo4jConfiguration -import org.neo4j.connectors.kafka.service.sink.strategy.SourceIdIngestionStrategyConfig +import org.neo4j.connectors.kafka.sink.strategy.legacy.SourceIdIngestionStrategyConfig import org.neo4j.driver.TransactionConfig class SinkConfigurationTest { diff --git a/sink/src/test/kotlin/org/neo4j/connectors/kafka/sink/strategy/NodePatternHandlerTest.kt b/sink/src/test/kotlin/org/neo4j/connectors/kafka/sink/strategy/NodePatternHandlerTest.kt index 45809b510..775279c8e 100644 --- a/sink/src/test/kotlin/org/neo4j/connectors/kafka/sink/strategy/NodePatternHandlerTest.kt +++ b/sink/src/test/kotlin/org/neo4j/connectors/kafka/sink/strategy/NodePatternHandlerTest.kt @@ -218,7 +218,7 @@ class NodePatternHandlerTest : HandlerTest() { handler.query shouldBe CypherParser.parse( """ - UNWIND ${'$'}messages AS event + UNWIND ${'$'}events AS event CALL { WITH event WITH event WHERE event[0] = 'C' WITH event[1] AS event diff --git a/sink/src/test/kotlin/org/neo4j/connectors/kafka/sink/strategy/RelationshipPatternHandlerTest.kt b/sink/src/test/kotlin/org/neo4j/connectors/kafka/sink/strategy/RelationshipPatternHandlerTest.kt index c5daa41d0..97bbe53af 100644 --- a/sink/src/test/kotlin/org/neo4j/connectors/kafka/sink/strategy/RelationshipPatternHandlerTest.kt +++ b/sink/src/test/kotlin/org/neo4j/connectors/kafka/sink/strategy/RelationshipPatternHandlerTest.kt @@ -362,7 +362,7 @@ class RelationshipPatternHandlerTest : HandlerTest() { handler.query shouldBe CypherParser.parse( """ - UNWIND ${'$'}messages AS event + UNWIND ${'$'}events AS event CALL { WITH event WITH event WHERE event[0] = 'C' WITH event[1] AS event diff --git a/common/src/test/kotlin/org/neo4j/connectors/kafka/service/sink/strategy/CUDIngestionStrategyTest.kt b/sink/src/test/kotlin/org/neo4j/connectors/kafka/sink/strategy/legacy/CUDIngestionStrategyTest.kt similarity index 99% rename from common/src/test/kotlin/org/neo4j/connectors/kafka/service/sink/strategy/CUDIngestionStrategyTest.kt rename to sink/src/test/kotlin/org/neo4j/connectors/kafka/sink/strategy/legacy/CUDIngestionStrategyTest.kt index fe6332f0d..b5aed481e 100644 --- a/common/src/test/kotlin/org/neo4j/connectors/kafka/service/sink/strategy/CUDIngestionStrategyTest.kt +++ b/sink/src/test/kotlin/org/neo4j/connectors/kafka/sink/strategy/legacy/CUDIngestionStrategyTest.kt @@ -14,13 +14,11 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.neo4j.connectors.kafka.service.sink.strategy +package org.neo4j.connectors.kafka.sink.strategy.legacy import kotlin.test.assertEquals import kotlin.test.assertTrue import org.junit.jupiter.api.Test -import org.neo4j.connectors.kafka.extensions.quote -import org.neo4j.connectors.kafka.service.StreamsSinkEntity import org.neo4j.connectors.kafka.utils.StreamsUtils @Suppress("UNCHECKED_CAST") diff --git a/common/src/test/kotlin/org/neo4j/connectors/kafka/service/sink/strategy/NodePatternIngestionStrategyTest.kt b/sink/src/test/kotlin/org/neo4j/connectors/kafka/sink/strategy/legacy/NodePatternIngestionStrategyTest.kt similarity index 86% rename from common/src/test/kotlin/org/neo4j/connectors/kafka/service/sink/strategy/NodePatternIngestionStrategyTest.kt rename to sink/src/test/kotlin/org/neo4j/connectors/kafka/sink/strategy/legacy/NodePatternIngestionStrategyTest.kt index 533386045..1529c4767 100644 --- a/common/src/test/kotlin/org/neo4j/connectors/kafka/service/sink/strategy/NodePatternIngestionStrategyTest.kt +++ b/sink/src/test/kotlin/org/neo4j/connectors/kafka/sink/strategy/legacy/NodePatternIngestionStrategyTest.kt @@ -14,11 +14,12 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.neo4j.connectors.kafka.service.sink.strategy +package org.neo4j.connectors.kafka.sink.strategy.legacy import kotlin.test.assertEquals import org.junit.jupiter.api.Test -import org.neo4j.connectors.kafka.service.StreamsSinkEntity +import org.neo4j.connectors.kafka.sink.strategy.pattern.NodePattern +import org.neo4j.connectors.kafka.sink.strategy.pattern.Pattern import org.neo4j.connectors.kafka.utils.StreamsUtils class NodePatternIngestionStrategyTest { @@ -26,8 +27,8 @@ class NodePatternIngestionStrategyTest { @Test fun `should get all properties`() { // given - val config = NodePatternConfiguration.parse("(:LabelA:LabelB{!id})", true) - val strategy = NodePatternIngestionStrategy(config) + val config = Pattern.parse("(:LabelA:LabelB{!id})") as NodePattern + val strategy = NodePatternIngestionStrategy(config, true) val data = mapOf("id" to 1, "foo" to "foo", "bar" to "bar", "foobar" to "foobar") // when @@ -58,8 +59,8 @@ class NodePatternIngestionStrategyTest { @Test fun `should get nested properties`() { // given - val config = NodePatternConfiguration.parse("(:LabelA:LabelB{!id, foo.bar})", false) - val strategy = NodePatternIngestionStrategy(config) + val config = Pattern.parse("(:LabelA:LabelB{!id, foo.bar})") as NodePattern + val strategy = NodePatternIngestionStrategy(config, false) val data = mapOf("id" to 1, "foo" to mapOf("bar" to "bar", "foobar" to "foobar")) // when @@ -88,8 +89,8 @@ class NodePatternIngestionStrategyTest { @Test fun `should exclude nested properties`() { // given - val config = NodePatternConfiguration.parse("(:LabelA:LabelB{!id, -foo})", false) - val strategy = NodePatternIngestionStrategy(config) + val config = Pattern.parse("(:LabelA:LabelB{!id, -foo})") as NodePattern + val strategy = NodePatternIngestionStrategy(config, false) val map = mapOf("id" to 1, "foo" to mapOf("bar" to "bar", "foobar" to "foobar"), "prop" to 100) // when @@ -118,8 +119,8 @@ class NodePatternIngestionStrategyTest { @Test fun `should include nested properties`() { // given - val config = NodePatternConfiguration.parse("(:LabelA:LabelB{!id, foo})", true) - val strategy = NodePatternIngestionStrategy(config) + val config = Pattern.parse("(:LabelA:LabelB{!id, foo})") as NodePattern + val strategy = NodePatternIngestionStrategy(config, true) val data = mapOf("id" to 1, "foo" to mapOf("bar" to "bar", "foobar" to "foobar"), "prop" to 100) // when @@ -151,8 +152,8 @@ class NodePatternIngestionStrategyTest { @Test fun `should exclude the properties`() { // given - val config = NodePatternConfiguration.parse("(:LabelA:LabelB{!id,-foo,-bar})", false) - val strategy = NodePatternIngestionStrategy(config) + val config = Pattern.parse("(:LabelA:LabelB{!id,-foo,-bar})") as NodePattern + val strategy = NodePatternIngestionStrategy(config, false) val data = mapOf("id" to 1, "foo" to "foo", "bar" to "bar", "foobar" to "foobar") // when @@ -181,8 +182,8 @@ class NodePatternIngestionStrategyTest { @Test fun `should include the properties`() { // given - val config = NodePatternConfiguration.parse("(:LabelA:LabelB{!id,foo,bar})", false) - val strategy = NodePatternIngestionStrategy(config) + val config = Pattern.parse("(:LabelA:LabelB{!id,foo,bar})") as NodePattern + val strategy = NodePatternIngestionStrategy(config, false) val data = mapOf("id" to 1, "foo" to "foo", "bar" to "bar", "foobar" to "foobar") // when @@ -212,8 +213,8 @@ class NodePatternIngestionStrategyTest { @Test fun `should delete the node`() { // given - val config = NodePatternConfiguration.parse("(:LabelA:LabelB{!id})", true) - val strategy = NodePatternIngestionStrategy(config) + val config = Pattern.parse("(:LabelA:LabelB{!id})") as NodePattern + val strategy = NodePatternIngestionStrategy(config, true) val data = mapOf("id" to 1, "foo" to "foo", "bar" to "bar", "foobar" to "foobar") // when diff --git a/common/src/test/kotlin/org/neo4j/connectors/kafka/service/sink/strategy/RelationshipPatternIngestionStrategyTest.kt b/sink/src/test/kotlin/org/neo4j/connectors/kafka/sink/strategy/legacy/RelationshipPatternIngestionStrategyTest.kt similarity index 86% rename from common/src/test/kotlin/org/neo4j/connectors/kafka/service/sink/strategy/RelationshipPatternIngestionStrategyTest.kt rename to sink/src/test/kotlin/org/neo4j/connectors/kafka/sink/strategy/legacy/RelationshipPatternIngestionStrategyTest.kt index 2b0d44f8e..d72f7bf05 100644 --- a/common/src/test/kotlin/org/neo4j/connectors/kafka/service/sink/strategy/RelationshipPatternIngestionStrategyTest.kt +++ b/sink/src/test/kotlin/org/neo4j/connectors/kafka/sink/strategy/legacy/RelationshipPatternIngestionStrategyTest.kt @@ -14,11 +14,12 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.neo4j.connectors.kafka.service.sink.strategy +package org.neo4j.connectors.kafka.sink.strategy.legacy import kotlin.test.assertEquals import org.junit.jupiter.api.Test -import org.neo4j.connectors.kafka.service.StreamsSinkEntity +import org.neo4j.connectors.kafka.sink.strategy.pattern.Pattern +import org.neo4j.connectors.kafka.sink.strategy.pattern.RelationshipPattern import org.neo4j.connectors.kafka.utils.StreamsUtils class RelationshipPatternIngestionStrategyTest { @@ -29,10 +30,8 @@ class RelationshipPatternIngestionStrategyTest { val startPattern = "LabelA{!idStart}" val endPattern = "LabelB{!idEnd}" val pattern = "(:$startPattern)-[:REL_TYPE]->(:$endPattern)" - val config = - RelationshipPatternConfiguration.parse( - pattern, mergeNodeProps = false, mergeRelProps = true) - val strategy = RelationshipPatternIngestionStrategy(config) + val config = Pattern.parse(pattern) as RelationshipPattern + val strategy = RelationshipPatternIngestionStrategy(config, false, true) val data = mapOf("idStart" to 1, "idEnd" to 2, "foo" to "foo", "bar" to "bar") // when @@ -45,10 +44,10 @@ class RelationshipPatternIngestionStrategyTest { """ |${StreamsUtils.UNWIND} |MERGE (start:LabelA{idStart: event.start.keys.idStart}) - |SET start += event.start.properties + |SET start = event.start.properties |SET start += event.start.keys |MERGE (end:LabelB{idEnd: event.end.keys.idEnd}) - |SET end += event.end.properties + |SET end = event.end.properties |SET end += event.end.keys |MERGE (start)-[r:REL_TYPE]->(end) |SET r += event.properties @@ -73,10 +72,8 @@ class RelationshipPatternIngestionStrategyTest { val startPattern = "LabelA{!idStart}" val endPattern = "LabelB{!idEnd}" val pattern = "$startPattern REL_TYPE $endPattern" - val config = - RelationshipPatternConfiguration.parse( - pattern, mergeNodeProps = false, mergeRelProps = false) - val strategy = RelationshipPatternIngestionStrategy(config) + val config = Pattern.parse(pattern) as RelationshipPattern + val strategy = RelationshipPatternIngestionStrategy(config, false, false) val data = mapOf("idStart" to 1, "idEnd" to 2, "foo" to "foo", "bar" to "bar") // when @@ -117,10 +114,8 @@ class RelationshipPatternIngestionStrategyTest { val startPattern = "LabelA{!idStart}" val endPattern = "LabelB{!idEnd}" val pattern = "(:$endPattern)<-[:REL_TYPE]-(:$startPattern)" - val config = - RelationshipPatternConfiguration.parse( - pattern, mergeNodeProps = false, mergeRelProps = false) - val strategy = RelationshipPatternIngestionStrategy(config) + val config = Pattern.parse(pattern) as RelationshipPattern + val strategy = RelationshipPatternIngestionStrategy(config, false, false) val data = mapOf("idStart" to 1, "idEnd" to 2, "foo" to "foo", "bar" to "bar") // when @@ -161,10 +156,8 @@ class RelationshipPatternIngestionStrategyTest { val startPattern = "LabelA{!idStart, foo.mapFoo}" val endPattern = "LabelB{!idEnd, bar.mapBar}" val pattern = "(:$startPattern)-[:REL_TYPE]->(:$endPattern)" - val config = - RelationshipPatternConfiguration.parse( - pattern, mergeNodeProps = false, mergeRelProps = true) - val strategy = RelationshipPatternIngestionStrategy(config) + val config = Pattern.parse(pattern) as RelationshipPattern + val strategy = RelationshipPatternIngestionStrategy(config, false, true) val data = mapOf( "idStart" to 1, @@ -184,10 +177,10 @@ class RelationshipPatternIngestionStrategyTest { """ |${StreamsUtils.UNWIND} |MERGE (start:LabelA{idStart: event.start.keys.idStart}) - |SET start += event.start.properties + |SET start = event.start.properties |SET start += event.start.keys |MERGE (end:LabelB{idEnd: event.end.keys.idEnd}) - |SET end += event.end.properties + |SET end = event.end.properties |SET end += event.end.keys |MERGE (start)-[r:REL_TYPE]->(end) |SET r += event.properties @@ -218,10 +211,8 @@ class RelationshipPatternIngestionStrategyTest { val startPattern = "LabelA{!idStart}" val endPattern = "LabelB{!idEnd}" val pattern = "(:$startPattern)-[:REL_TYPE]->(:$endPattern)" - val config = - RelationshipPatternConfiguration.parse( - pattern, mergeNodeProps = false, mergeRelProps = false) - val strategy = RelationshipPatternIngestionStrategy(config) + val config = Pattern.parse(pattern) as RelationshipPattern + val strategy = RelationshipPatternIngestionStrategy(config, false, false) val data = mapOf("idStart" to 1, "idEnd" to 2, "foo" to "foo", "bar" to "bar") // when diff --git a/common/src/test/kotlin/org/neo4j/connectors/kafka/service/sink/strategy/SchemaIngestionStrategyTest.kt b/sink/src/test/kotlin/org/neo4j/connectors/kafka/sink/strategy/legacy/SchemaIngestionStrategyTest.kt similarity index 99% rename from common/src/test/kotlin/org/neo4j/connectors/kafka/service/sink/strategy/SchemaIngestionStrategyTest.kt rename to sink/src/test/kotlin/org/neo4j/connectors/kafka/sink/strategy/legacy/SchemaIngestionStrategyTest.kt index 8240c6918..63b8854c7 100644 --- a/common/src/test/kotlin/org/neo4j/connectors/kafka/service/sink/strategy/SchemaIngestionStrategyTest.kt +++ b/sink/src/test/kotlin/org/neo4j/connectors/kafka/sink/strategy/legacy/SchemaIngestionStrategyTest.kt @@ -14,7 +14,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.neo4j.connectors.kafka.service.sink.strategy +package org.neo4j.connectors.kafka.sink.strategy.legacy import kotlin.test.assertEquals import kotlin.test.assertTrue @@ -30,7 +30,6 @@ import org.neo4j.connectors.kafka.events.RelationshipPayload import org.neo4j.connectors.kafka.events.Schema import org.neo4j.connectors.kafka.events.StreamsConstraintType import org.neo4j.connectors.kafka.events.StreamsTransactionEvent -import org.neo4j.connectors.kafka.service.StreamsSinkEntity import org.neo4j.connectors.kafka.utils.StreamsUtils class SchemaIngestionStrategyTest { diff --git a/common/src/test/kotlin/org/neo4j/connectors/kafka/utils/SchemaUtilsTest.kt b/sink/src/test/kotlin/org/neo4j/connectors/kafka/sink/strategy/legacy/SchemaUtilsTest.kt similarity index 97% rename from common/src/test/kotlin/org/neo4j/connectors/kafka/utils/SchemaUtilsTest.kt rename to sink/src/test/kotlin/org/neo4j/connectors/kafka/sink/strategy/legacy/SchemaUtilsTest.kt index adc08d4b6..af7547859 100644 --- a/common/src/test/kotlin/org/neo4j/connectors/kafka/utils/SchemaUtilsTest.kt +++ b/sink/src/test/kotlin/org/neo4j/connectors/kafka/sink/strategy/legacy/SchemaUtilsTest.kt @@ -14,14 +14,14 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.neo4j.connectors.kafka.utils +package org.neo4j.connectors.kafka.sink.strategy.legacy import kotlin.test.assertEquals import org.junit.jupiter.api.Test import org.neo4j.connectors.kafka.events.Constraint import org.neo4j.connectors.kafka.events.RelKeyStrategy import org.neo4j.connectors.kafka.events.StreamsConstraintType -import org.neo4j.connectors.kafka.utils.SchemaUtils.getNodeKeys +import org.neo4j.connectors.kafka.sink.strategy.legacy.SchemaUtils.getNodeKeys class SchemaUtilsTest { diff --git a/common/src/test/kotlin/org/neo4j/connectors/kafka/service/sink/strategy/SourceIdIngestionStrategyTest.kt b/sink/src/test/kotlin/org/neo4j/connectors/kafka/sink/strategy/legacy/SourceIdIngestionStrategyTest.kt similarity index 99% rename from common/src/test/kotlin/org/neo4j/connectors/kafka/service/sink/strategy/SourceIdIngestionStrategyTest.kt rename to sink/src/test/kotlin/org/neo4j/connectors/kafka/sink/strategy/legacy/SourceIdIngestionStrategyTest.kt index c434c3ab8..edd5e0aee 100644 --- a/common/src/test/kotlin/org/neo4j/connectors/kafka/service/sink/strategy/SourceIdIngestionStrategyTest.kt +++ b/sink/src/test/kotlin/org/neo4j/connectors/kafka/sink/strategy/legacy/SourceIdIngestionStrategyTest.kt @@ -14,7 +14,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.neo4j.connectors.kafka.service.sink.strategy +package org.neo4j.connectors.kafka.sink.strategy.legacy import kotlin.test.assertEquals import org.junit.jupiter.api.Test @@ -27,7 +27,6 @@ import org.neo4j.connectors.kafka.events.RelationshipNodeChange import org.neo4j.connectors.kafka.events.RelationshipPayload import org.neo4j.connectors.kafka.events.Schema import org.neo4j.connectors.kafka.events.StreamsTransactionEvent -import org.neo4j.connectors.kafka.service.StreamsSinkEntity import org.neo4j.connectors.kafka.utils.StreamsUtils class SourceIdIngestionStrategyTest { diff --git a/source/src/main/kotlin/org/neo4j/connectors/kafka/source/utils/ConnectExtensionFunctions.kt b/source/src/main/kotlin/org/neo4j/connectors/kafka/source/utils/ConnectExtensionFunctions.kt index a62dbdf50..303e687e7 100644 --- a/source/src/main/kotlin/org/neo4j/connectors/kafka/source/utils/ConnectExtensionFunctions.kt +++ b/source/src/main/kotlin/org/neo4j/connectors/kafka/source/utils/ConnectExtensionFunctions.kt @@ -20,8 +20,8 @@ import java.time.temporal.TemporalAccessor import org.apache.kafka.connect.data.Schema import org.apache.kafka.connect.data.SchemaBuilder import org.apache.kafka.connect.data.Struct -import org.neo4j.connectors.kafka.extensions.asStreamsMap import org.neo4j.connectors.kafka.utils.JSONUtils +import org.neo4j.connectors.kafka.utils.asStreamsMap import org.neo4j.driver.Record import org.neo4j.driver.types.Node import org.neo4j.driver.types.Point From f5265c994ae502effdaa3e3f2e16233f46da723d Mon Sep 17 00:00:00 2001 From: emrehizal Date: Wed, 22 May 2024 17:25:06 +0200 Subject: [PATCH 06/14] test: publish multiple kafka messages in a transaction --- .../kafka/sink/Neo4jRelationshipPatternIT.kt | 4 + .../testing/kafka/ConvertingKafkaProducer.kt | 95 ++++++++++++------- .../kafka/testing/sink/Neo4jSinkExtension.kt | 5 +- 3 files changed, 71 insertions(+), 33 deletions(-) create mode 100644 sink-connector/src/test/kotlin/org/neo4j/connectors/kafka/sink/Neo4jRelationshipPatternIT.kt diff --git a/sink-connector/src/test/kotlin/org/neo4j/connectors/kafka/sink/Neo4jRelationshipPatternIT.kt b/sink-connector/src/test/kotlin/org/neo4j/connectors/kafka/sink/Neo4jRelationshipPatternIT.kt new file mode 100644 index 000000000..81c231b4f --- /dev/null +++ b/sink-connector/src/test/kotlin/org/neo4j/connectors/kafka/sink/Neo4jRelationshipPatternIT.kt @@ -0,0 +1,4 @@ +package org.neo4j.connectors.kafka.sink + +class Neo4jRelationshipPatternIT { +} diff --git a/testing/src/main/kotlin/org/neo4j/connectors/kafka/testing/kafka/ConvertingKafkaProducer.kt b/testing/src/main/kotlin/org/neo4j/connectors/kafka/testing/kafka/ConvertingKafkaProducer.kt index 677a8b14c..f6cde9675 100644 --- a/testing/src/main/kotlin/org/neo4j/connectors/kafka/testing/kafka/ConvertingKafkaProducer.kt +++ b/testing/src/main/kotlin/org/neo4j/connectors/kafka/testing/kafka/ConvertingKafkaProducer.kt @@ -33,6 +33,15 @@ import org.neo4j.connectors.kafka.testing.format.KafkaConverter import org.neo4j.connectors.kafka.testing.sink.SchemaCompatibilityMode import org.neo4j.connectors.kafka.utils.JSONUtils +data class KafkaMessage( + val keySchema: Schema? = null, + val key: Any? = null, + val valueSchema: Schema? = null, + val value: Any? = null, + val timestamp: Instant? = null, + val headers: Map = emptyMap() +) + data class ConvertingKafkaProducer( val schemaRegistryURI: URI, val keyCompatibilityMode: SchemaCompatibilityMode, @@ -43,6 +52,59 @@ data class ConvertingKafkaProducer( val topic: String ) { + init { + ensureSchemaCompatibility(topic) + } + + fun publish(vararg kafkaMessages: KafkaMessage) { + kafkaProducer.beginTransaction() + try { + kafkaMessages.forEach { + val serializedKey = + when (it.key) { + null -> null + else -> + keyConverter.testShimSerializer.serialize( + it.key, it.keySchema ?: throw IllegalArgumentException("null key schema")) + } + val serializedValue = + when (it.value) { + null -> null + else -> + valueConverter.testShimSerializer.serialize( + it.value, + it.valueSchema ?: throw IllegalArgumentException("null value schema")) + } + val converter = SimpleHeaderConverter() + val recordHeaders = + it.headers.map { e -> + object : Header { + override fun key(): String = e.key + + override fun value(): ByteArray { + return converter.fromConnectHeader( + "", e.key, Values.inferSchema(e.value), e.value) + } + } + } + + val record: ProducerRecord = + ProducerRecord( + topic, + null, + it.timestamp?.toEpochMilli(), + serializedKey, + serializedValue, + recordHeaders) + kafkaProducer.send(record).get() + } + kafkaProducer.commitTransaction() + } catch (e: Exception) { + kafkaProducer.abortTransaction() + throw e + } + } + fun publish( keySchema: Schema? = null, key: Any? = null, @@ -51,38 +113,7 @@ data class ConvertingKafkaProducer( timestamp: Instant? = null, headers: Map = emptyMap() ) { - val serializedKey = - when (key) { - null -> null - else -> - keyConverter.testShimSerializer.serialize( - key, keySchema ?: throw IllegalArgumentException("null key schema")) - } - val serializedValue = - when (value) { - null -> null - else -> - valueConverter.testShimSerializer.serialize( - value, valueSchema ?: throw IllegalArgumentException("null value schema")) - } - val converter = SimpleHeaderConverter() - val recordHeaders = - headers.map { e -> - object : Header { - override fun key(): String = e.key - - override fun value(): ByteArray { - return converter.fromConnectHeader("", e.key, Values.inferSchema(e.value), e.value) - } - } - } - - ensureSchemaCompatibility(topic) - - val record: ProducerRecord = - ProducerRecord( - topic, null, timestamp?.toEpochMilli(), serializedKey, serializedValue, recordHeaders) - kafkaProducer.send(record).get() + publish(KafkaMessage(keySchema, key, valueSchema, value, timestamp, headers)) } fun publish(event: ChangeEvent) { diff --git a/testing/src/main/kotlin/org/neo4j/connectors/kafka/testing/sink/Neo4jSinkExtension.kt b/testing/src/main/kotlin/org/neo4j/connectors/kafka/testing/sink/Neo4jSinkExtension.kt index a029ed4bc..300467704 100644 --- a/testing/src/main/kotlin/org/neo4j/connectors/kafka/testing/sink/Neo4jSinkExtension.kt +++ b/testing/src/main/kotlin/org/neo4j/connectors/kafka/testing/sink/Neo4jSinkExtension.kt @@ -319,7 +319,10 @@ internal class Neo4jSinkExtension( ProducerConfig.VALUE_SERIALIZER_CLASS_CONFIG, valueConverter.serializerClass.name, ) - return KafkaProducer(properties) + properties.setProperty(ProducerConfig.TRANSACTIONAL_ID_CONFIG, UUID.randomUUID().toString()) + val producer = KafkaProducer(properties) + producer.initTransactions() + return producer } private fun resolveGenericProducer( From 8eb565fe066a905e3b0433626618e3b31ead6fa5 Mon Sep 17 00:00:00 2001 From: emrehizal Date: Thu, 23 May 2024 14:27:09 +0200 Subject: [PATCH 07/14] test: more integration tests node pattern --- .../kafka/sink/Neo4jNodePatternIT.kt | 248 ++++++++++++++---- 1 file changed, 194 insertions(+), 54 deletions(-) diff --git a/sink-connector/src/test/kotlin/org/neo4j/connectors/kafka/sink/Neo4jNodePatternIT.kt b/sink-connector/src/test/kotlin/org/neo4j/connectors/kafka/sink/Neo4jNodePatternIT.kt index 65e4d507b..54da0e442 100644 --- a/sink-connector/src/test/kotlin/org/neo4j/connectors/kafka/sink/Neo4jNodePatternIT.kt +++ b/sink-connector/src/test/kotlin/org/neo4j/connectors/kafka/sink/Neo4jNodePatternIT.kt @@ -17,10 +17,12 @@ package org.neo4j.connectors.kafka.sink import com.fasterxml.jackson.databind.ObjectMapper +import com.squareup.wire.Instant import io.kotest.assertions.nondeterministic.eventually import io.kotest.matchers.should import io.kotest.matchers.shouldBe -import io.kotest.matchers.shouldNotBe +import java.time.ZoneOffset +import java.time.temporal.ChronoUnit import kotlin.time.Duration.Companion.seconds import org.apache.kafka.connect.data.Schema import org.apache.kafka.connect.data.SchemaBuilder @@ -28,6 +30,7 @@ import org.apache.kafka.connect.data.Struct import org.junit.jupiter.api.Test import org.neo4j.connectors.kafka.testing.TestSupport.runTest import org.neo4j.connectors.kafka.testing.kafka.ConvertingKafkaProducer +import org.neo4j.connectors.kafka.testing.kafka.KafkaMessage import org.neo4j.connectors.kafka.testing.sink.Neo4jSink import org.neo4j.connectors.kafka.testing.sink.NodePatternStrategy import org.neo4j.connectors.kafka.testing.sink.TopicProducer @@ -46,6 +49,8 @@ class Neo4jNodePatternIT { @TopicProducer(TOPIC) producer: ConvertingKafkaProducer, session: Session ) = runTest { + session.run("CREATE CONSTRAINT FOR (n:User) REQUIRE n.id IS KEY").consume() + producer.publish( valueSchema = Schema.STRING_SCHEMA, value = """{"id": 1, "name": "john", "surname": "doe"}""") @@ -65,6 +70,8 @@ class Neo4jNodePatternIT { @TopicProducer(TOPIC) producer: ConvertingKafkaProducer, session: Session ) = runTest { + session.run("CREATE CONSTRAINT FOR (n:User) REQUIRE n.id IS KEY").consume() + SchemaBuilder.struct() .field("id", Schema.INT64_SCHEMA) .field("name", Schema.STRING_SCHEMA) @@ -91,6 +98,8 @@ class Neo4jNodePatternIT { @TopicProducer(TOPIC) producer: ConvertingKafkaProducer, session: Session ) = runTest { + session.run("CREATE CONSTRAINT FOR (n:User) REQUIRE n.id IS KEY").consume() + producer.publish( valueSchema = Schema.BYTES_SCHEMA, value = @@ -116,6 +125,8 @@ class Neo4jNodePatternIT { @TopicProducer(TOPIC) producer: ConvertingKafkaProducer, session: Session ) = runTest { + session.run("CREATE CONSTRAINT FOR (n:User) REQUIRE n.id IS KEY").consume() + producer.publish( valueSchema = Schema.STRING_SCHEMA, value = """{"old_id": 1, "first_name": "john", "last_name": "doe"}""") @@ -141,11 +152,14 @@ class Neo4jNodePatternIT { @TopicProducer(TOPIC) producer: ConvertingKafkaProducer, session: Session ) = runTest { + session.run("CREATE CONSTRAINT FOR (n:User) REQUIRE n.id IS KEY").consume() + producer.publish( keySchema = Schema.STRING_SCHEMA, key = """{"first_name": "john"}""", valueSchema = Schema.STRING_SCHEMA, value = """{"old_id": 1, "last_name": "doe"}""") + eventually(30.seconds) { session.run("MATCH (n:User) RETURN n", emptyMap()).single() } .get("n") .asNode() should @@ -158,6 +172,7 @@ class Neo4jNodePatternIT { keySchema = Schema.STRING_SCHEMA, key = """{"old_id": 1}""", ) + eventually(30.seconds) { session .run("MATCH (n:User) RETURN count(n) AS count", emptyMap()) @@ -167,28 +182,56 @@ class Neo4jNodePatternIT { } } - @Neo4jSink(nodePattern = [NodePatternStrategy(TOPIC, "(:User{!id,name,surname})", false)]) + @Neo4jSink(nodePattern = [NodePatternStrategy(TOPIC, "(:User{!id})", false)]) @Test - fun `should create, delete and recreate node`( + fun `should create and delete node in the same kafka transaction`( @TopicProducer(TOPIC) producer: ConvertingKafkaProducer, session: Session ) = runTest { + session.run("CREATE CONSTRAINT FOR (n:User) REQUIRE n.id IS KEY").consume() + producer.publish( - keySchema = Schema.STRING_SCHEMA, - key = """{"id": 1}""", - valueSchema = Schema.STRING_SCHEMA, - value = """{"name": "john", "surname": "doe"}""") - producer.publish( - keySchema = Schema.STRING_SCHEMA, - key = """{"id": 1}""", - ) - producer.publish( - keySchema = Schema.STRING_SCHEMA, - key = """{"id": 1}""", - valueSchema = Schema.STRING_SCHEMA, - value = """{"name": "john-new", "surname": "doe-new"}""") + KafkaMessage( + keySchema = Schema.STRING_SCHEMA, + key = """{"id": 1}""", + valueSchema = Schema.STRING_SCHEMA, + value = """{"name": "john", "surname": "doe"}"""), + KafkaMessage( + keySchema = Schema.STRING_SCHEMA, + key = """{"id": 1}""", + )) eventually(30.seconds) { + session + .run("MATCH (n:User) RETURN count(n) AS count", emptyMap()) + .single() + .get("count") + .asLong() shouldBe 0 + } + } + + @Neo4jSink(nodePattern = [NodePatternStrategy(TOPIC, "(:User{!id,name,surname})", false)]) + @Test + fun `should create, delete and recreate node`( + @TopicProducer(TOPIC) producer: ConvertingKafkaProducer, + session: Session + ) = runTest { + session.run("CREATE CONSTRAINT FOR (n:User) REQUIRE n.id IS KEY").consume() + + producer.publish( + KafkaMessage( + keySchema = Schema.STRING_SCHEMA, + key = """{"id": 1}""", + valueSchema = Schema.STRING_SCHEMA, + value = """{"name": "john", "surname": "doe"}"""), + KafkaMessage(keySchema = Schema.STRING_SCHEMA, key = """{"id": 1}"""), + KafkaMessage( + keySchema = Schema.STRING_SCHEMA, + key = """{"id": 1}""", + valueSchema = Schema.STRING_SCHEMA, + value = """{"name": "john-new", "surname": "doe-new"}""")) + + eventually(60.seconds) { session.run("MATCH (n:User) RETURN n", emptyMap()).single().get("n").asNode() should { it.labels() shouldBe listOf("User") @@ -203,16 +246,19 @@ class Neo4jNodePatternIT { @TopicProducer(TOPIC) producer: ConvertingKafkaProducer, session: Session ) = runTest { + session.run("CREATE CONSTRAINT FOR (n:User) REQUIRE n.id IS KEY").consume() + producer.publish( - keySchema = Schema.STRING_SCHEMA, - key = """{"id": 1}""", - valueSchema = Schema.STRING_SCHEMA, - value = """{"name": "john", "surname": "doe"}""") - producer.publish( - keySchema = Schema.STRING_SCHEMA, - key = """{"id": 2}""", - valueSchema = Schema.STRING_SCHEMA, - value = """{"name": "mary", "surname": "doe"}""") + KafkaMessage( + keySchema = Schema.STRING_SCHEMA, + key = """{"id": 1}""", + valueSchema = Schema.STRING_SCHEMA, + value = """{"name": "john", "surname": "doe"}"""), + KafkaMessage( + keySchema = Schema.STRING_SCHEMA, + key = """{"id": 2}""", + valueSchema = Schema.STRING_SCHEMA, + value = """{"name": "mary", "surname": "doe"}""")) eventually(30.seconds) { session.run("MATCH (n:User) RETURN n", emptyMap()).list { r -> @@ -230,15 +276,18 @@ class Neo4jNodePatternIT { @Neo4jSink(nodePattern = [NodePatternStrategy(TOPIC, "(:User{!id,!name,surname})", false)]) @Test - fun `should create node with compositeKey`( + fun `should create node with composite key`( @TopicProducer(TOPIC) producer: ConvertingKafkaProducer, session: Session ) = runTest { + session.run("CREATE CONSTRAINT FOR (n:User) REQUIRE n.id IS KEY").consume() + producer.publish( keySchema = Schema.STRING_SCHEMA, key = """{"id": 1, "name": "john"}""", valueSchema = Schema.STRING_SCHEMA, value = """{"surname": "doe"}""") + eventually(30.seconds) { session.run("MATCH (n:User) RETURN n", emptyMap()).single() } .get("n") .asNode() should @@ -254,10 +303,13 @@ class Neo4jNodePatternIT { @TopicProducer(TOPIC) producer: ConvertingKafkaProducer, session: Session ) = runTest { + session.run("CREATE CONSTRAINT FOR (n:User) REQUIRE n.id IS KEY").consume() + producer.publish( valueSchema = Schema.STRING_SCHEMA, value = """{"id": 1, "name": "john", "surname": "doe", "address": {"city": "london", "country": "uk"}}""") + eventually(30.seconds) { session.run("MATCH (n:User) RETURN n", emptyMap()).single() } .get("n") .asNode() should @@ -279,10 +331,13 @@ class Neo4jNodePatternIT { @TopicProducer(TOPIC) producer: ConvertingKafkaProducer, session: Session ) = runTest { + session.run("CREATE CONSTRAINT FOR (n:User) REQUIRE n.id IS KEY").consume() + producer.publish( valueSchema = Schema.STRING_SCHEMA, value = """{"id": 1, "name": "john", "surname": "doe", "dob": "2000-01-01", "address": {"city": "london", "country": "uk"}}""") + eventually(30.seconds) { session.run("MATCH (n:User) RETURN n", emptyMap()).single() } .get("n") .asNode() should @@ -300,19 +355,25 @@ class Neo4jNodePatternIT { @Neo4jSink( nodePattern = [NodePatternStrategy(TOPIC, "(:User{!id, created_at: __timestamp})", false)]) @Test - fun `should create node with createdAt`( + fun `should create node with timestamp`( @TopicProducer(TOPIC) producer: ConvertingKafkaProducer, session: Session ) = runTest { + val timestamp = Instant.now().truncatedTo(ChronoUnit.MILLIS) + session.run("CREATE CONSTRAINT FOR (n:User) REQUIRE n.id IS KEY").consume() + producer.publish( valueSchema = Schema.STRING_SCHEMA, - value = """{"id": 1, "name": "john", "surname": "doe"}""") + value = """{"id": 1, "name": "john", "surname": "doe"}""", + timestamp = timestamp) + eventually(30.seconds) { session.run("MATCH (n:User) RETURN n", emptyMap()).single() } .get("n") .asNode() should { + println(it.asMap()) it.labels() shouldBe listOf("User") - it.asMap()["created_at"] shouldNotBe null + it.asMap() shouldBe mapOf("id" to 1L, "created_at" to timestamp.atZone(ZoneOffset.UTC)) } } @@ -324,15 +385,18 @@ class Neo4jNodePatternIT { "(:User{!id: __key.old_id, name: __key.first_name, surname: __key.last_name})", false)]) @Test - fun `should create and delete node with all keys pattern`( + fun `should create and delete node with explicit properties from message key`( @TopicProducer(TOPIC) producer: ConvertingKafkaProducer, session: Session ) = runTest { + session.run("CREATE CONSTRAINT FOR (n:User) REQUIRE n.id IS KEY").consume() + producer.publish( keySchema = Schema.STRING_SCHEMA, key = """{"old_id": 1, "first_name": "john", "last_name": "doe"}""", valueSchema = Schema.STRING_SCHEMA, value = """{}""") + eventually(30.seconds) { session.run("MATCH (n:User) RETURN n", emptyMap()).single() } .get("n") .asNode() should @@ -345,6 +409,7 @@ class Neo4jNodePatternIT { keySchema = Schema.STRING_SCHEMA, key = """{"old_id": 1}""", ) + eventually(30.seconds) { session .run("MATCH (n:User) RETURN count(n) AS count", emptyMap()) @@ -360,19 +425,13 @@ class Neo4jNodePatternIT { @TopicProducer(TOPIC) producer: ConvertingKafkaProducer, session: Session ) = runTest { - producer.publish( - keySchema = Schema.STRING_SCHEMA, - key = """{"id": 1}""", - valueSchema = Schema.STRING_SCHEMA, - value = """{"name": "john", "surname": "doe"}""") + session.run("CREATE CONSTRAINT FOR (n:User) REQUIRE n.id IS KEY").consume() - eventually(30.seconds) { session.run("MATCH (n:User) RETURN n", emptyMap()).single() } - .get("n") - .asNode() should - { - it.labels() shouldBe listOf("User") - it.asMap() shouldBe mapOf("id" to 1L, "name" to "john", "surname" to "doe") - } + session + .run( + "CREATE (n:User) SET n = ${'$'}props", + mapOf("props" to mapOf("id" to 1L, "name" to "john", "surname" to "doe"))) + .consume() producer.publish( keySchema = Schema.STRING_SCHEMA, @@ -400,19 +459,11 @@ class Neo4jNodePatternIT { @TopicProducer(TOPIC) producer: ConvertingKafkaProducer, session: Session ) = runTest { - producer.publish( - keySchema = Schema.STRING_SCHEMA, - key = """{"old_id": 1}""", - valueSchema = Schema.STRING_SCHEMA, - value = """{"first_name": "john", "last_name": "doe"}""") + session.run("CREATE CONSTRAINT FOR (n:User) REQUIRE n.id IS KEY").consume() - eventually(30.seconds) { session.run("MATCH (n:User) RETURN n", emptyMap()).single() } - .get("n") - .asNode() should - { - it.labels() shouldBe listOf("User") - it.asMap() shouldBe mapOf("id" to 1L, "name" to "john", "surname" to "doe") - } + session.run( + "CREATE (n:User) SET n = ${'$'}props", + mapOf("props" to mapOf("id" to 1L, "name" to "john", "surname" to "doe"))) producer.publish( keySchema = Schema.STRING_SCHEMA, @@ -441,6 +492,9 @@ class Neo4jNodePatternIT { @TopicProducer(TOPIC_2) producer2: ConvertingKafkaProducer, session: Session ) = runTest { + session.run("CREATE CONSTRAINT FOR (n:User) REQUIRE n.id IS KEY").consume() + session.run("CREATE CONSTRAINT FOR (n:Account) REQUIRE n.id IS KEY").consume() + producer1.publish( keySchema = Schema.STRING_SCHEMA, key = """{"id": 1}""", @@ -469,4 +523,90 @@ class Neo4jNodePatternIT { it.asMap() shouldBe mapOf("id" to 1L, "email" to "john@doe.com") } } + + @Neo4jSink(nodePattern = [NodePatternStrategy(TOPIC, "(:User{!id})", false)]) + @Test + fun `should create 1000 nodes`( + @TopicProducer(TOPIC) producer: ConvertingKafkaProducer, + session: Session + ) = runTest { + session.run("CREATE CONSTRAINT FOR (n:User) REQUIRE n.id IS KEY").consume() + + val kafkaMessages = mutableListOf() + for (i in 1..1000) { + kafkaMessages.add( + KafkaMessage( + keySchema = Schema.STRING_SCHEMA, + key = """{"id": $i}""", + valueSchema = Schema.STRING_SCHEMA, + value = """{"name": "john-$i", "surname": "doe-$i"}""")) + } + + producer.publish(*kafkaMessages.toTypedArray()) + + eventually(30.seconds) { + session + .run("MATCH (n:User) RETURN count(n) AS count", emptyMap()) + .single() + .get("count") + .asLong() shouldBe 1000 + } + } + + @Neo4jSink(nodePattern = [NodePatternStrategy(TOPIC, "(:User{!id})", true)]) + @Test + fun `should append new properties when merge node properties true`( + @TopicProducer(TOPIC) producer: ConvertingKafkaProducer, + session: Session + ) = runTest { + session.run("CREATE CONSTRAINT FOR (n:User) REQUIRE n.id IS KEY").consume() + session + .run( + "CREATE (n:User) SET n = ${'$'}props", + mapOf("props" to mapOf("id" to 1, "name" to "john", "surname" to "doe"))) + .consume() + + producer.publish( + keySchema = Schema.STRING_SCHEMA, + key = """{"id": 1}""", + valueSchema = Schema.STRING_SCHEMA, + value = """{"born": 1970}""") + + eventually(30.seconds) { + session.run("MATCH (n:User) RETURN n", emptyMap()).single().get("n").asNode() should + { + it.labels() shouldBe listOf("User") + it.asMap() shouldBe + mapOf("id" to 1L, "name" to "john", "surname" to "doe", "born" to 1970) + } + } + } + + @Neo4jSink(nodePattern = [NodePatternStrategy(TOPIC, "(:User{!id})", false)]) + @Test + fun `should remove existing properties and add new ones when merge node properties false`( + @TopicProducer(TOPIC) producer: ConvertingKafkaProducer, + session: Session + ) = runTest { + session.run("CREATE CONSTRAINT FOR (n:User) REQUIRE n.id IS KEY").consume() + session + .run( + "CREATE (n:User) SET n = ${'$'}props", + mapOf("props" to mapOf("id" to 1, "name" to "john", "surname" to "doe"))) + .consume() + + producer.publish( + keySchema = Schema.STRING_SCHEMA, + key = """{"id": 1}""", + valueSchema = Schema.STRING_SCHEMA, + value = """{"born": 1970}""") + + eventually(30.seconds) { + session.run("MATCH (n:User) RETURN n", emptyMap()).single().get("n").asNode() should + { + it.labels() shouldBe listOf("User") + it.asMap() shouldBe mapOf("id" to 1L, "born" to 1970) + } + } + } } From 6cce736183b976ee1b5d277ec11f0fb9e3b7bd50 Mon Sep 17 00:00:00 2001 From: emrehizal Date: Tue, 28 May 2024 10:44:12 +0200 Subject: [PATCH 08/14] test: add multiple label pattern test and change some method names for node pattern --- .../kafka/sink/Neo4jNodePatternIT.kt | 35 ++++++++++++++++--- 1 file changed, 30 insertions(+), 5 deletions(-) diff --git a/sink-connector/src/test/kotlin/org/neo4j/connectors/kafka/sink/Neo4jNodePatternIT.kt b/sink-connector/src/test/kotlin/org/neo4j/connectors/kafka/sink/Neo4jNodePatternIT.kt index 54da0e442..1ee9fafe1 100644 --- a/sink-connector/src/test/kotlin/org/neo4j/connectors/kafka/sink/Neo4jNodePatternIT.kt +++ b/sink-connector/src/test/kotlin/org/neo4j/connectors/kafka/sink/Neo4jNodePatternIT.kt @@ -17,7 +17,6 @@ package org.neo4j.connectors.kafka.sink import com.fasterxml.jackson.databind.ObjectMapper -import com.squareup.wire.Instant import io.kotest.assertions.nondeterministic.eventually import io.kotest.matchers.should import io.kotest.matchers.shouldBe @@ -35,6 +34,7 @@ import org.neo4j.connectors.kafka.testing.sink.Neo4jSink import org.neo4j.connectors.kafka.testing.sink.NodePatternStrategy import org.neo4j.connectors.kafka.testing.sink.TopicProducer import org.neo4j.driver.Session +import java.time.Instant class Neo4jNodePatternIT { companion object { @@ -115,6 +115,31 @@ class Neo4jNodePatternIT { } } + @Neo4jSink(nodePattern = [NodePatternStrategy(TOPIC, "(:User:Person{!id,name,surname})", false)]) + @Test + fun `should create node with multiple labels pattern`( + @TopicProducer(TOPIC) producer: ConvertingKafkaProducer, + session: Session + ) = runTest { + session.run("CREATE CONSTRAINT FOR (n:User) REQUIRE n.id IS KEY").consume() + session.run("CREATE CONSTRAINT FOR (n:Person) REQUIRE n.id IS KEY").consume() + + producer.publish( + valueSchema = Schema.BYTES_SCHEMA, + value = + ObjectMapper() + .writeValueAsBytes(mapOf("id" to 1L, "name" to "john", "surname" to "doe"))) + + eventually(30.seconds) { session.run("MATCH (n:User) RETURN n", emptyMap()).single() } + .get("n") + .asNode() should + { + it.labels() shouldBe listOf("User", "Person") + it.asMap() shouldBe mapOf("id" to 1L, "name" to "john", "surname" to "doe") + } + } + + @Neo4jSink( nodePattern = [ @@ -421,7 +446,7 @@ class Neo4jNodePatternIT { @Neo4jSink(nodePattern = [NodePatternStrategy(TOPIC, "(:User{!id})", false)]) @Test - fun `should create and update node`( + fun `should update node`( @TopicProducer(TOPIC) producer: ConvertingKafkaProducer, session: Session ) = runTest { @@ -455,7 +480,7 @@ class Neo4jNodePatternIT { NodePatternStrategy( TOPIC, "(:User{!id: old_id, name: first_name, surname: last_name})", false)]) @Test - fun `should create and update node with aliases`( + fun `should update node with aliases`( @TopicProducer(TOPIC) producer: ConvertingKafkaProducer, session: Session ) = runTest { @@ -555,7 +580,7 @@ class Neo4jNodePatternIT { @Neo4jSink(nodePattern = [NodePatternStrategy(TOPIC, "(:User{!id})", true)]) @Test - fun `should append new properties when merge node properties true`( + fun `should merge node properties`( @TopicProducer(TOPIC) producer: ConvertingKafkaProducer, session: Session ) = runTest { @@ -584,7 +609,7 @@ class Neo4jNodePatternIT { @Neo4jSink(nodePattern = [NodePatternStrategy(TOPIC, "(:User{!id})", false)]) @Test - fun `should remove existing properties and add new ones when merge node properties false`( + fun `should not merge node properties`( @TopicProducer(TOPIC) producer: ConvertingKafkaProducer, session: Session ) = runTest { From 6706f65334c04b833aabd0c0d23ad8ad35748e1d Mon Sep 17 00:00:00 2001 From: emrehizal Date: Tue, 28 May 2024 10:58:12 +0200 Subject: [PATCH 09/14] test: add integration tests for relationship pattern --- .../kafka/sink/Neo4jRelationshipPatternIT.kt | 873 ++++++++++++++++++ 1 file changed, 873 insertions(+) diff --git a/sink-connector/src/test/kotlin/org/neo4j/connectors/kafka/sink/Neo4jRelationshipPatternIT.kt b/sink-connector/src/test/kotlin/org/neo4j/connectors/kafka/sink/Neo4jRelationshipPatternIT.kt index 81c231b4f..0ab19ee85 100644 --- a/sink-connector/src/test/kotlin/org/neo4j/connectors/kafka/sink/Neo4jRelationshipPatternIT.kt +++ b/sink-connector/src/test/kotlin/org/neo4j/connectors/kafka/sink/Neo4jRelationshipPatternIT.kt @@ -1,4 +1,877 @@ +/* + * Copyright (c) "Neo4j" + * Neo4j Sweden AB [https://neo4j.com] + * + * 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 org.neo4j.connectors.kafka.sink +import com.fasterxml.jackson.databind.ObjectMapper +import io.kotest.assertions.nondeterministic.eventually +import io.kotest.matchers.collections.shouldContainAll +import io.kotest.matchers.collections.shouldHaveSize +import io.kotest.matchers.should +import io.kotest.matchers.shouldBe +import kotlin.time.Duration.Companion.seconds +import org.apache.kafka.connect.data.Schema +import org.apache.kafka.connect.data.SchemaBuilder +import org.apache.kafka.connect.data.Struct +import org.junit.jupiter.api.Test +import org.neo4j.connectors.kafka.testing.TestSupport.runTest +import org.neo4j.connectors.kafka.testing.kafka.ConvertingKafkaProducer +import org.neo4j.connectors.kafka.testing.kafka.KafkaMessage +import org.neo4j.connectors.kafka.testing.sink.Neo4jSink +import org.neo4j.connectors.kafka.testing.sink.RelationshipPatternStrategy +import org.neo4j.connectors.kafka.testing.sink.TopicProducer +import org.neo4j.driver.Session + class Neo4jRelationshipPatternIT { + + companion object { + const val TOPIC = "test" + const val TOPIC_1 = "test-1" + const val TOPIC_2 = "test-2" + } + + @Neo4jSink( + relationshipPattern = + [ + RelationshipPatternStrategy( + TOPIC, + "(:User{!userId})-[:BOUGHT]->(:Product{!productId})", + mergeNodeProperties = false, + mergeRelationshipProperties = false)]) + @Test + fun `should create nodes and relationship from struct`( + @TopicProducer(TOPIC) producer: ConvertingKafkaProducer, + session: Session + ) = runTest { + session.run("CREATE CONSTRAINT FOR (n:User) REQUIRE n.userId IS KEY").consume() + session.run("CREATE CONSTRAINT FOR (n:Product) REQUIRE n.productId IS KEY").consume() + + SchemaBuilder.struct() + .field("userId", Schema.INT64_SCHEMA) + .field("productId", Schema.INT64_SCHEMA) + .build() + .let { schema -> + producer.publish( + valueSchema = schema, value = Struct(schema).put("userId", 1L).put("productId", 2L)) + } + + eventually(30.seconds) { + val result = + session.run("MATCH (u:User)-[r:BOUGHT]->(p:Product) RETURN u, r, p", emptyMap()).single() + + result.get("u").asNode() should + { + it.labels() shouldBe listOf("User") + it.asMap() shouldBe mapOf("userId" to 1L) + } + + result.get("r").asRelationship() should { it.type() shouldBe "BOUGHT" } + + result.get("p").asNode() should + { + it.labels() shouldBe listOf("Product") + it.asMap() shouldBe mapOf("productId" to 2L) + } + } + } + + @Neo4jSink( + relationshipPattern = + [ + RelationshipPatternStrategy( + TOPIC, + "(:User{!userId})-[:BOUGHT]->(:Product{!productId})", + mergeNodeProperties = false, + mergeRelationshipProperties = false)]) + @Test + fun `should create nodes and relationship from json string`( + @TopicProducer(TOPIC) producer: ConvertingKafkaProducer, + session: Session + ) = runTest { + session.run("CREATE CONSTRAINT FOR (n:User) REQUIRE n.userId IS KEY").consume() + session.run("CREATE CONSTRAINT FOR (n:Product) REQUIRE n.productId IS KEY").consume() + + producer.publish( + valueSchema = Schema.STRING_SCHEMA, value = """{"userId": 1, "productId": 2}""") + + eventually(30.seconds) { + val result = + session.run("MATCH (u:User)-[r:BOUGHT]->(p:Product) RETURN u, r, p", emptyMap()).single() + + result.get("u").asNode() should + { + it.labels() shouldBe listOf("User") + it.asMap() shouldBe mapOf("userId" to 1L) + } + + result.get("r").asRelationship() should { it.type() shouldBe "BOUGHT" } + + result.get("p").asNode() should + { + it.labels() shouldBe listOf("Product") + it.asMap() shouldBe mapOf("productId" to 2L) + } + } + } + + @Neo4jSink( + relationshipPattern = + [ + RelationshipPatternStrategy( + TOPIC, + "(:User{!userId})-[:BOUGHT]->(:Product{!productId})", + mergeNodeProperties = false, + mergeRelationshipProperties = false)]) + @Test + fun `should create nodes and relationship from byte array`( + @TopicProducer(TOPIC) producer: ConvertingKafkaProducer, + session: Session + ) = runTest { + session.run("CREATE CONSTRAINT FOR (n:User) REQUIRE n.userId IS KEY").consume() + session.run("CREATE CONSTRAINT FOR (n:Product) REQUIRE n.productId IS KEY").consume() + + producer.publish( + valueSchema = Schema.BYTES_SCHEMA, + value = ObjectMapper().writeValueAsBytes(mapOf("userId" to 1L, "productId" to 2L))) + + eventually(30.seconds) { + val result = + session.run("MATCH (u:User)-[r:BOUGHT]->(p:Product) RETURN u, r, p", emptyMap()).single() + + result.get("u").asNode() should + { + it.labels() shouldBe listOf("User") + it.asMap() shouldBe mapOf("userId" to 1L) + } + + result.get("r").asRelationship() should { it.type() shouldBe "BOUGHT" } + + result.get("p").asNode() should + { + it.labels() shouldBe listOf("Product") + it.asMap() shouldBe mapOf("productId" to 2L) + } + } + } + + @Neo4jSink( + relationshipPattern = + [ + RelationshipPatternStrategy( + TOPIC, + "(:User{!userId})-[:BOUGHT{-currency}]->(:Product{!productId})", + mergeNodeProperties = false, + mergeRelationshipProperties = false)]) + @Test + fun `should create nodes and relationship with excluded properties`( + @TopicProducer(TOPIC) producer: ConvertingKafkaProducer, + session: Session + ) = runTest { + session.run("CREATE CONSTRAINT FOR (n:User) REQUIRE n.userId IS KEY").consume() + session.run("CREATE CONSTRAINT FOR (n:Product) REQUIRE n.productId IS KEY").consume() + + producer.publish( + valueSchema = Schema.BYTES_SCHEMA, + value = + ObjectMapper() + .writeValueAsBytes( + mapOf( + "userId" to 1L, + "productId" to 2L, + "amount" to 5, + "currency" to "EUR", + ))) + + eventually(30.seconds) { + val result = + session.run("MATCH (u:User)-[r:BOUGHT]->(p:Product) RETURN u, r, p", emptyMap()).single() + + result.get("u").asNode() should + { + it.labels() shouldBe listOf("User") + it.asMap() shouldBe + mapOf( + "userId" to 1L, + ) + } + + result.get("r").asRelationship() should + { + it.type() shouldBe "BOUGHT" + it.asMap() shouldBe mapOf("amount" to 5) + } + + result.get("p").asNode() should + { + it.labels() shouldBe listOf("Product") + it.asMap() shouldBe + mapOf( + "productId" to 2L, + ) + } + } + } + + @Neo4jSink( + relationshipPattern = + [ + RelationshipPatternStrategy( + TOPIC, + "User{!userId} BOUGHT Product{!productId}", + mergeNodeProperties = false, + mergeRelationshipProperties = false)]) + @Test + fun `should create nodes and relationship with simpler pattern`( + @TopicProducer(TOPIC) producer: ConvertingKafkaProducer, + session: Session + ) = runTest { + session.run("CREATE CONSTRAINT FOR (n:User) REQUIRE n.userId IS KEY").consume() + session.run("CREATE CONSTRAINT FOR (n:Product) REQUIRE n.productId IS KEY").consume() + + producer.publish( + valueSchema = Schema.BYTES_SCHEMA, + value = ObjectMapper().writeValueAsBytes(mapOf("userId" to 1L, "productId" to 2L))) + + eventually(30.seconds) { + val result = + session.run("MATCH (u:User)-[r:BOUGHT]->(p:Product) RETURN u, r, p", emptyMap()).single() + + result.get("u").asNode() should + { + it.labels() shouldBe listOf("User") + it.asMap() shouldBe mapOf("userId" to 1L) + } + + result.get("r").asRelationship() should { it.type() shouldBe "BOUGHT" } + + result.get("p").asNode() should + { + it.labels() shouldBe listOf("Product") + it.asMap() shouldBe mapOf("productId" to 2L) + } + } + } + + @Neo4jSink( + relationshipPattern = + [ + RelationshipPatternStrategy( + TOPIC, + "(:User{!id: userId})-[:BOUGHT {!id: transactionId}]->(:Product{!id: productId})", + mergeNodeProperties = false, + mergeRelationshipProperties = false)]) + @Test + fun `should create nodes and relationship with aliased properties`( + @TopicProducer(TOPIC) producer: ConvertingKafkaProducer, + session: Session + ) = runTest { + session.run("CREATE CONSTRAINT FOR (n:User) REQUIRE n.id IS KEY").consume() + session.run("CREATE CONSTRAINT FOR (n:Product) REQUIRE n.id IS KEY").consume() + + producer.publish( + valueSchema = Schema.STRING_SCHEMA, + value = """{"userId": 1, "productId": 2, "transactionId": 3}""") + + eventually(30.seconds) { + val result = + session.run("MATCH (u:User)-[r:BOUGHT]->(p:Product) RETURN u, r, p", emptyMap()).single() + + result.get("u").asNode() should + { + it.labels() shouldBe listOf("User") + it.asMap() shouldBe mapOf("id" to 1L) + } + + result.get("r").asRelationship() should { it.type() shouldBe "BOUGHT" } + + result.get("p").asNode() should + { + it.labels() shouldBe listOf("Product") + it.asMap() shouldBe mapOf("id" to 2L) + } + } + } + + @Neo4jSink( + relationshipPattern = + [ + RelationshipPatternStrategy( + TOPIC, + "(:User{!id: userId})-[:BOUGHT{!id: transactionId, price, currency}]->(:Product{!id: productId})", + mergeNodeProperties = false, + mergeRelationshipProperties = false)]) + @Test + fun `should create nodes and relationship with relationship key and properties`( + @TopicProducer(TOPIC) producer: ConvertingKafkaProducer, + session: Session + ) = runTest { + session.run("CREATE CONSTRAINT FOR (n:User) REQUIRE n.id IS KEY").consume() + session.run("CREATE CONSTRAINT FOR (n:Product) REQUIRE n.id IS KEY").consume() + + producer.publish( + keySchema = Schema.STRING_SCHEMA, + key = """{"userId": 1, "productId": 2, "transactionId": 3}""}""", + valueSchema = Schema.STRING_SCHEMA, + value = """{"price": 10.5, "currency": "EUR"}""") + + eventually(30.seconds) { + val result = + session.run("MATCH (u:User)-[r:BOUGHT]->(p:Product) RETURN u, r, p", emptyMap()).single() + + result.get("u").asNode() should + { + it.labels() shouldBe listOf("User") + it.asMap() shouldBe mapOf("id" to 1L) + } + + result.get("r").asRelationship() should + { + it.type() shouldBe "BOUGHT" + it.asMap() shouldBe mapOf("id" to 3L, "price" to 10.5, "currency" to "EUR") + } + + result.get("p").asNode() should + { + it.labels() shouldBe listOf("Product") + it.asMap() shouldBe mapOf("id" to 2L) + } + } + } + + @Neo4jSink( + relationshipPattern = + [ + RelationshipPatternStrategy( + TOPIC, + """(:User{!id: __key.user_id, name: __value.first_name})-[:BOUGHT{!id: __key.transaction_id, price: __value.paid_price, currency}]->(:Product{!id: __key.product_id, name: __value.product_name})""", + mergeNodeProperties = false, + mergeRelationshipProperties = false)]) + @Test + fun `should create nodes and relationship with explicit properties`( + @TopicProducer(TOPIC) producer: ConvertingKafkaProducer, + session: Session + ) = runTest { + session.run("CREATE CONSTRAINT FOR (n:User) REQUIRE n.id IS KEY").consume() + session.run("CREATE CONSTRAINT FOR (n:Product) REQUIRE n.id IS KEY").consume() + + producer.publish( + keySchema = Schema.STRING_SCHEMA, + key = """{"user_id": 1, "product_id": 2, "transaction_id": 3}""}""", + valueSchema = Schema.STRING_SCHEMA, + value = + """{"first_name": "John", "paid_price": 10.5, "currency": "EUR", "product_name": "computer"}""") + + eventually(30.seconds) { + val result = + session.run("MATCH (u:User)-[r:BOUGHT]->(p:Product) RETURN u, r, p", emptyMap()).single() + + result.get("u").asNode() should + { + it.labels() shouldBe listOf("User") + it.asMap() shouldBe mapOf("id" to 1L, "name" to "John") + } + + result.get("r").asRelationship() should + { + it.type() shouldBe "BOUGHT" + it.asMap() shouldBe + mapOf( + "id" to 3L, + "price" to 10.5, + "currency" to "EUR", + ) + } + + result.get("p").asNode() should + { + it.labels() shouldBe listOf("Product") + it.asMap() shouldBe mapOf("id" to 2L, "name" to "computer") + } + } + } + + @Neo4jSink( + relationshipPattern = + [ + RelationshipPatternStrategy( + TOPIC, + "(:User:Person{!id: userId})-[:BOUGHT]->(:Product:Item{!id: productId})", + mergeNodeProperties = false, + mergeRelationshipProperties = false)]) + @Test + fun `should create nodes and relationship with multiple labels pattern`( + @TopicProducer(TOPIC) producer: ConvertingKafkaProducer, + session: Session + ) = runTest { + session.run("CREATE CONSTRAINT FOR (n:User) REQUIRE n.id IS KEY").consume() + session.run("CREATE CONSTRAINT FOR (n:Person) REQUIRE n.id IS KEY").consume() + session.run("CREATE CONSTRAINT FOR (n:Product) REQUIRE n.id IS KEY").consume() + session.run("CREATE CONSTRAINT FOR (n:Item) REQUIRE n.id IS KEY").consume() + + producer.publish( + valueSchema = Schema.STRING_SCHEMA, value = """{"userId": 1, "productId": 2}""") + + eventually(30.seconds) { + val result = + session.run("MATCH (u:User)-[r:BOUGHT]->(p:Product) RETURN u, r, p", emptyMap()).single() + + result.get("u").asNode() should + { + it.labels() shouldBe listOf("User", "Person") + it.asMap() shouldBe mapOf("id" to 1L) + } + + result.get("r").asRelationship() should { it.type() shouldBe "BOUGHT" } + + result.get("p").asNode() should + { + it.labels() shouldBe listOf("Product", "Item") + it.asMap() shouldBe mapOf("id" to 2L) + } + } + } + + @Neo4jSink( + relationshipPattern = + [ + RelationshipPatternStrategy( + TOPIC, + "(:User{!id: userId})-[:BOUGHT]->(:Product{!id: productId})", + mergeNodeProperties = false, + mergeRelationshipProperties = false)]) + @Test + fun `should add non id values to relationship`( + @TopicProducer(TOPIC) producer: ConvertingKafkaProducer, + session: Session + ) = runTest { + session.run("CREATE CONSTRAINT FOR (n:User) REQUIRE n.id IS KEY").consume() + session.run("CREATE CONSTRAINT FOR (n:Product) REQUIRE n.id IS KEY").consume() + + producer.publish( + valueSchema = Schema.STRING_SCHEMA, + value = """{"userId": 1, "productId": 2, "price": 10, "currency": "EUR"}""") + + eventually(30.seconds) { + val result = + session.run("MATCH (u:User)-[r:BOUGHT]->(p:Product) RETURN u, r, p", emptyMap()).single() + + result.get("u").asNode() should + { + it.labels() shouldBe listOf("User") + it.asMap() shouldBe mapOf("id" to 1L) + } + + result.get("r").asRelationship() should + { + it.type() shouldBe "BOUGHT" + it.asMap() shouldBe mapOf("price" to 10, "currency" to "EUR") + } + + result.get("p").asNode() should + { + it.labels() shouldBe listOf("Product") + it.asMap() shouldBe mapOf("id" to 2L) + } + } + } + + @Neo4jSink( + relationshipPattern = + [ + RelationshipPatternStrategy( + TOPIC, + "(:User{!id: userId})-[:BOUGHT]->(:Product{!id: productId})", + mergeNodeProperties = false, + mergeRelationshipProperties = false)]) + @Test + fun `should delete relationship`( + @TopicProducer(TOPIC) producer: ConvertingKafkaProducer, + session: Session + ) = runTest { + session.run("CREATE CONSTRAINT FOR (n:User) REQUIRE n.id IS KEY").consume() + session.run("CREATE CONSTRAINT FOR (n:Product) REQUIRE n.id IS KEY").consume() + + session + .run( + """CREATE (u:User) SET u.id = 1 + CREATE (p:Product) SET p.id = 2 + MERGE (u)-[:BOUGHT]->(p)""") + .consume() + + producer.publish(keySchema = Schema.STRING_SCHEMA, key = """{"userId": 1, "productId": 2}""") + + eventually(30.seconds) { + session + .run( + "MATCH (:User {id: 1})-[r:BOUGHT]->(:Product {id: 2}) RETURN count(r) as count", + emptyMap()) + .single() + .get("count") + .asLong() shouldBe 0 + } + } + + @Neo4jSink( + relationshipPattern = + [ + RelationshipPatternStrategy( + TOPIC, + "(:User{!id: userId})-[:BOUGHT{price, currency}]->(:Product{!id: productId})", + mergeNodeProperties = false, + mergeRelationshipProperties = false)]) + @Test + fun `should add only relationship`( + @TopicProducer(TOPIC) producer: ConvertingKafkaProducer, + session: Session + ) = runTest { + session.run("CREATE CONSTRAINT FOR (n:User) REQUIRE n.id IS KEY").consume() + session.run("CREATE CONSTRAINT FOR (n:Product) REQUIRE n.id IS KEY").consume() + + session + .run( + """CREATE (u:User) SET u.id = 1 + CREATE (p:Product) SET p.id = 2""") + .consume() + + producer.publish( + valueSchema = Schema.STRING_SCHEMA, + value = """{"userId": 1, "productId": 2, "price": 10.5, "currency": "EUR"}""") + + eventually(30.seconds) { + session + .run("MATCH (:User {id: 1})-[r:BOUGHT]->(:Product {id: 2}) RETURN r ", emptyMap()) + .single() + } + .get("r") + .asRelationship() should + { + it.type() shouldBe "BOUGHT" + it.asMap() shouldBe mapOf("price" to 10.5, "currency" to "EUR") + } + } + + @Neo4jSink( + relationshipPattern = + [ + RelationshipPatternStrategy( + TOPIC, + "(:User{!id: user_id, name: user_name})-[:BOUGHT {amount}]->(:Product{!id: product_id, name: product_name})", + mergeNodeProperties = false, + mergeRelationshipProperties = false)]) + @Test + fun `should create, delete and recreate relationship`( + @TopicProducer(TOPIC) producer: ConvertingKafkaProducer, + session: Session + ) = runTest { + session.run("CREATE CONSTRAINT FOR (n:User) REQUIRE n.id IS KEY").consume() + session.run("CREATE CONSTRAINT FOR (n:Product) REQUIRE n.id IS KEY").consume() + + producer.publish( + KafkaMessage( + keySchema = Schema.STRING_SCHEMA, + key = """{"user_id": 1, "product_id": 2}""", + valueSchema = Schema.STRING_SCHEMA, + value = """{"user_name": "John", "amount": 1, "product_name": "computer"}"""), + KafkaMessage(keySchema = Schema.STRING_SCHEMA, key = """{"user_id": 1, "product_id": 2}"""), + KafkaMessage( + keySchema = Schema.STRING_SCHEMA, + key = """{"user_id": 1, "product_id": 2}""", + valueSchema = Schema.STRING_SCHEMA, + value = """{"user_name": "John-new", "amount": 5, "product_name": "computer-new"}""")) + + eventually(30.seconds) { + val result = + session + .run( + "MATCH (u:User {id: 1})-[r:BOUGHT]->(p:Product {id: 2}) RETURN u, r, p", + emptyMap(), + ) + .single() + + result.get("u").asNode() should + { + it.labels() shouldBe listOf("User") + it.asMap() shouldBe mapOf("id" to 1L, "name" to "John-new") + } + + result.get("r").asRelationship() should + { + it.type() shouldBe "BOUGHT" + it.asMap() shouldBe mapOf("amount" to 5) + } + + result.get("p").asNode() should + { + it.labels() shouldBe listOf("Product") + it.asMap() shouldBe mapOf("id" to 2, "name" to "computer-new") + } + } + } + + @Neo4jSink( + relationshipPattern = + [ + RelationshipPatternStrategy( + TOPIC_1, + "(:User{!id: userId})-[:BOUGHT]->(:Product{!id: productId})", + mergeNodeProperties = false, + mergeRelationshipProperties = false), + RelationshipPatternStrategy( + TOPIC_2, + "(:User{!id: userId})-[:SOLD]->(:Product{!id: productId})", + mergeNodeProperties = false, + mergeRelationshipProperties = false)]) + @Test + fun `should create multiple relationships from different topics`( + @TopicProducer(TOPIC_1) producer1: ConvertingKafkaProducer, + @TopicProducer(TOPIC_2) producer2: ConvertingKafkaProducer, + session: Session + ) = runTest { + session.run("CREATE CONSTRAINT FOR (n:User) REQUIRE n.id IS KEY").consume() + session.run("CREATE CONSTRAINT FOR (n:Product) REQUIRE n.id IS KEY").consume() + + producer1.publish( + valueSchema = Schema.STRING_SCHEMA, value = """{"userId": 1, "productId": 2}""") + + producer2.publish( + valueSchema = Schema.STRING_SCHEMA, value = """{"userId": 1, "productId": 2}""") + + eventually(30.seconds) { + session + .run( + "MATCH (u:User {id: 1})-[r]->(p:Product {id: 2}) RETURN r", + emptyMap(), + ) + .list() + .map { it.get("r").asRelationship().type() } shouldContainAll listOf("BOUGHT", "SOLD") + } + } + + @Neo4jSink( + relationshipPattern = + [ + RelationshipPatternStrategy( + TOPIC, + "(:User{!id: userId})-[:BOUGHT {!id: transactionId}]->(:Product{!id: productId})", + mergeNodeProperties = false, + mergeRelationshipProperties = false)]) + @Test + fun `should create 1000 relationships`( + @TopicProducer(TOPIC) producer: ConvertingKafkaProducer, + session: Session + ) = runTest { + session.run("CREATE CONSTRAINT FOR (n:User) REQUIRE n.id IS KEY").consume() + session.run("CREATE CONSTRAINT FOR (n:Product) REQUIRE n.id IS KEY").consume() + + val kafkaMessages = mutableListOf() + for (i in 1..1000) { + kafkaMessages.add( + KafkaMessage( + valueSchema = Schema.STRING_SCHEMA, + value = """{"userId": 1, "productId": 2, "transactionId": $i}""")) + } + + producer.publish(*kafkaMessages.toTypedArray()) + + eventually(30.seconds) { + session + .run( + "MATCH (u:User {id: 1})-[r]->(p:Product {id: 2}) RETURN r", + emptyMap(), + ) + .list() shouldHaveSize 1000 + } + } + + @Neo4jSink( + relationshipPattern = + [ + RelationshipPatternStrategy( + TOPIC, + "(:User{!id: userId, born})-[:BOUGHT]->(:Product{!id: productId, price})", + mergeNodeProperties = true, + mergeRelationshipProperties = false)]) + @Test + fun `should merge node properties`( + @TopicProducer(TOPIC) producer: ConvertingKafkaProducer, + session: Session + ) = runTest { + session.run("CREATE CONSTRAINT FOR (n:User) REQUIRE n.id IS KEY").consume() + session.run("CREATE CONSTRAINT FOR (n:Product) REQUIRE n.id IS KEY").consume() + + session + .run( + """CREATE (:User {id: 1, name: "Joe", surname: "Doe"})-[:BOUGHT]->(:Product {id: 2, name: "computer"})""") + .consume() + + producer.publish( + KafkaMessage( + valueSchema = Schema.STRING_SCHEMA, + value = """{"userId": 1, "productId": 2, "born": 1970, "price": 10.5}""")) + + eventually(30.seconds) { + val result = + session + .run( + "MATCH (u:User {id: 1})-[:BOUGHT]->(p:Product {id: 2}) RETURN u, p", + emptyMap(), + ) + .single() + + result.get("u").asNode() should + { + it.labels() shouldBe listOf("User") + it.asMap() shouldBe + mapOf("id" to 1, "name" to "Joe", "surname" to "Doe", "born" to 1970) + } + + result.get("p").asNode() should + { + it.labels() shouldBe listOf("Product") + it.asMap() shouldBe mapOf("id" to 2, "name" to "computer", "price" to 10.5) + } + } + } + + @Neo4jSink( + relationshipPattern = + [ + RelationshipPatternStrategy( + TOPIC, + "(:User{!id: userId, born})-[:BOUGHT]->(:Product{!id: productId, price})", + mergeNodeProperties = false, + mergeRelationshipProperties = false)]) + @Test + fun `should not merge node properties`( + @TopicProducer(TOPIC) producer: ConvertingKafkaProducer, + session: Session + ) = runTest { + session.run("CREATE CONSTRAINT FOR (n:User) REQUIRE n.id IS KEY").consume() + session.run("CREATE CONSTRAINT FOR (n:Product) REQUIRE n.id IS KEY").consume() + + session + .run( + """CREATE (:User {id: 1, name: "Joe", surname: "Doe"})-[:BOUGHT]->(:Product {id: 2, name: "computer"})""") + .consume() + + producer.publish( + KafkaMessage( + valueSchema = Schema.STRING_SCHEMA, + value = """{"userId": 1, "productId": 2, "born": 1970, "price": 10.5}""")) + + eventually(30.seconds) { + val result = + session + .run( + "MATCH (u:User {id: 1})-[:BOUGHT]->(p:Product {id: 2}) RETURN u, p", + emptyMap(), + ) + .single() + + result.get("u").asNode() should + { + it.labels() shouldBe listOf("User") + it.asMap() shouldBe mapOf("id" to 1, "born" to 1970) + } + + result.get("p").asNode() should + { + it.labels() shouldBe listOf("Product") + it.asMap() shouldBe mapOf("id" to 2, "price" to 10.5) + } + } + } + + @Neo4jSink( + relationshipPattern = + [ + RelationshipPatternStrategy( + TOPIC, + "(:User{!id: userId})-[:BOUGHT]->(:Product{!id: productId})", + mergeNodeProperties = false, + mergeRelationshipProperties = true)]) + @Test + fun `should merge relationship properties`( + @TopicProducer(TOPIC) producer: ConvertingKafkaProducer, + session: Session + ) = runTest { + session.run("CREATE CONSTRAINT FOR (n:User) REQUIRE n.id IS KEY").consume() + session.run("CREATE CONSTRAINT FOR (n:Product) REQUIRE n.id IS KEY").consume() + + session.run("""CREATE (:User {id: 1})-[:BOUGHT {amount: 10}]->(:Product {id: 2})""").consume() + + producer.publish( + KafkaMessage( + valueSchema = Schema.STRING_SCHEMA, + value = """{"userId": 1, "productId": 2, "date": "2024-05-27"}""")) + + eventually(30.seconds) { + val result = + session + .run( + "MATCH (:User {id: 1})-[r:BOUGHT]->(:Product {id: 2}) RETURN r", + emptyMap(), + ) + .single() + + result.get("r").asRelationship() should + { + it.type() shouldBe "BOUGHT" + it.asMap() shouldBe mapOf("amount" to 10, "date" to "2024-05-27") + } + } + } + + @Neo4jSink( + relationshipPattern = + [ + RelationshipPatternStrategy( + TOPIC, + "(:User{!id: userId})-[:BOUGHT]->(:Product{!id: productId})", + mergeNodeProperties = false, + mergeRelationshipProperties = false)]) + @Test + fun `should not merge relationship properties`( + @TopicProducer(TOPIC) producer: ConvertingKafkaProducer, + session: Session + ) = runTest { + session.run("CREATE CONSTRAINT FOR (n:User) REQUIRE n.id IS KEY").consume() + session.run("CREATE CONSTRAINT FOR (n:Product) REQUIRE n.id IS KEY").consume() + + session.run("""CREATE (:User {id: 1})-[:BOUGHT {amount: 10}]->(:Product {id: 2})""").consume() + + producer.publish( + KafkaMessage( + valueSchema = Schema.STRING_SCHEMA, + value = """{"userId": 1, "productId": 2, "date": "2024-05-27"}""")) + + eventually(30.seconds) { + val result = + session + .run( + "MATCH (:User {id: 1})-[r:BOUGHT]->(:Product {id: 2}) RETURN r", + emptyMap(), + ) + .single() + + result.get("r").asRelationship() should + { + it.type() shouldBe "BOUGHT" + it.asMap() shouldBe mapOf("date" to "2024-05-27") + } + } + } } From a95198854335390da146f1d92ed09fb3010bfaaa Mon Sep 17 00:00:00 2001 From: emrehizal Date: Tue, 28 May 2024 11:07:54 +0200 Subject: [PATCH 10/14] style: formatting --- .../neo4j/connectors/kafka/sink/Neo4jNodePatternIT.kt | 11 +++++------ .../kafka/sink/Neo4jRelationshipPatternIT.kt | 3 +-- 2 files changed, 6 insertions(+), 8 deletions(-) diff --git a/sink-connector/src/test/kotlin/org/neo4j/connectors/kafka/sink/Neo4jNodePatternIT.kt b/sink-connector/src/test/kotlin/org/neo4j/connectors/kafka/sink/Neo4jNodePatternIT.kt index 1ee9fafe1..d682a8307 100644 --- a/sink-connector/src/test/kotlin/org/neo4j/connectors/kafka/sink/Neo4jNodePatternIT.kt +++ b/sink-connector/src/test/kotlin/org/neo4j/connectors/kafka/sink/Neo4jNodePatternIT.kt @@ -20,6 +20,7 @@ import com.fasterxml.jackson.databind.ObjectMapper import io.kotest.assertions.nondeterministic.eventually import io.kotest.matchers.should import io.kotest.matchers.shouldBe +import java.time.Instant import java.time.ZoneOffset import java.time.temporal.ChronoUnit import kotlin.time.Duration.Companion.seconds @@ -34,7 +35,6 @@ import org.neo4j.connectors.kafka.testing.sink.Neo4jSink import org.neo4j.connectors.kafka.testing.sink.NodePatternStrategy import org.neo4j.connectors.kafka.testing.sink.TopicProducer import org.neo4j.driver.Session -import java.time.Instant class Neo4jNodePatternIT { companion object { @@ -118,8 +118,8 @@ class Neo4jNodePatternIT { @Neo4jSink(nodePattern = [NodePatternStrategy(TOPIC, "(:User:Person{!id,name,surname})", false)]) @Test fun `should create node with multiple labels pattern`( - @TopicProducer(TOPIC) producer: ConvertingKafkaProducer, - session: Session + @TopicProducer(TOPIC) producer: ConvertingKafkaProducer, + session: Session ) = runTest { session.run("CREATE CONSTRAINT FOR (n:User) REQUIRE n.id IS KEY").consume() session.run("CREATE CONSTRAINT FOR (n:Person) REQUIRE n.id IS KEY").consume() @@ -127,8 +127,8 @@ class Neo4jNodePatternIT { producer.publish( valueSchema = Schema.BYTES_SCHEMA, value = - ObjectMapper() - .writeValueAsBytes(mapOf("id" to 1L, "name" to "john", "surname" to "doe"))) + ObjectMapper() + .writeValueAsBytes(mapOf("id" to 1L, "name" to "john", "surname" to "doe"))) eventually(30.seconds) { session.run("MATCH (n:User) RETURN n", emptyMap()).single() } .get("n") @@ -139,7 +139,6 @@ class Neo4jNodePatternIT { } } - @Neo4jSink( nodePattern = [ diff --git a/sink-connector/src/test/kotlin/org/neo4j/connectors/kafka/sink/Neo4jRelationshipPatternIT.kt b/sink-connector/src/test/kotlin/org/neo4j/connectors/kafka/sink/Neo4jRelationshipPatternIT.kt index 0ab19ee85..73764ad34 100644 --- a/sink-connector/src/test/kotlin/org/neo4j/connectors/kafka/sink/Neo4jRelationshipPatternIT.kt +++ b/sink-connector/src/test/kotlin/org/neo4j/connectors/kafka/sink/Neo4jRelationshipPatternIT.kt @@ -541,8 +541,7 @@ class Neo4jRelationshipPatternIT { session.run("CREATE CONSTRAINT FOR (n:Product) REQUIRE n.id IS KEY").consume() session - .run( - """CREATE (u:User) SET u.id = 1 + .run("""CREATE (u:User) SET u.id = 1 CREATE (p:Product) SET p.id = 2""") .consume() From c933959cdfcccd771ffcf2997e3c918df140ddff Mon Sep 17 00:00:00 2001 From: Ali Ince Date: Fri, 31 May 2024 14:43:50 +0100 Subject: [PATCH 11/14] test: add support other converters in pattern tests Signed-off-by: Emre Hizal --- .../kafka/sink/Neo4jNodePatternIT.kt | 44 +++++++++++++++++-- .../kafka/sink/Neo4jRelationshipPatternIT.kt | 42 ++++++++++++++++-- 2 files changed, 79 insertions(+), 7 deletions(-) diff --git a/sink-connector/src/test/kotlin/org/neo4j/connectors/kafka/sink/Neo4jNodePatternIT.kt b/sink-connector/src/test/kotlin/org/neo4j/connectors/kafka/sink/Neo4jNodePatternIT.kt index d682a8307..5514da0c6 100644 --- a/sink-connector/src/test/kotlin/org/neo4j/connectors/kafka/sink/Neo4jNodePatternIT.kt +++ b/sink-connector/src/test/kotlin/org/neo4j/connectors/kafka/sink/Neo4jNodePatternIT.kt @@ -21,6 +21,7 @@ import io.kotest.assertions.nondeterministic.eventually import io.kotest.matchers.should import io.kotest.matchers.shouldBe import java.time.Instant +import java.time.LocalDate import java.time.ZoneOffset import java.time.temporal.ChronoUnit import kotlin.time.Duration.Companion.seconds @@ -28,15 +29,20 @@ import org.apache.kafka.connect.data.Schema import org.apache.kafka.connect.data.SchemaBuilder import org.apache.kafka.connect.data.Struct import org.junit.jupiter.api.Test +import org.neo4j.connectors.kafka.data.DynamicTypes +import org.neo4j.connectors.kafka.data.SimpleTypes import org.neo4j.connectors.kafka.testing.TestSupport.runTest +import org.neo4j.connectors.kafka.testing.format.KafkaConverter +import org.neo4j.connectors.kafka.testing.format.KeyValueConverter import org.neo4j.connectors.kafka.testing.kafka.ConvertingKafkaProducer import org.neo4j.connectors.kafka.testing.kafka.KafkaMessage import org.neo4j.connectors.kafka.testing.sink.Neo4jSink import org.neo4j.connectors.kafka.testing.sink.NodePatternStrategy import org.neo4j.connectors.kafka.testing.sink.TopicProducer import org.neo4j.driver.Session +import org.neo4j.driver.Values -class Neo4jNodePatternIT { +abstract class Neo4jNodePatternIT { companion object { const val TOPIC = "test" const val TOPIC_1 = "test-1" @@ -64,7 +70,8 @@ class Neo4jNodePatternIT { } } - @Neo4jSink(nodePattern = [NodePatternStrategy(TOPIC, "(:User{!id,name,surname})", false)]) + @Neo4jSink( + nodePattern = [NodePatternStrategy(TOPIC, "(:User{!id,name,surname,dob,place})", false)]) @Test fun `should create node from struct`( @TopicProducer(TOPIC) producer: ConvertingKafkaProducer, @@ -76,11 +83,25 @@ class Neo4jNodePatternIT { .field("id", Schema.INT64_SCHEMA) .field("name", Schema.STRING_SCHEMA) .field("surname", Schema.STRING_SCHEMA) + .field("dob", SimpleTypes.LOCALDATE_STRUCT.schema) + .field("place", SimpleTypes.POINT.schema) .build() .let { schema -> producer.publish( valueSchema = schema, - value = Struct(schema).put("id", 1L).put("name", "john").put("surname", "doe")) + value = + Struct(schema) + .put("id", 1L) + .put("name", "john") + .put("surname", "doe") + .put( + "dob", + DynamicTypes.toConnectValue( + SimpleTypes.LOCALDATE_STRUCT.schema, LocalDate.of(1995, 1, 1))) + .put( + "place", + DynamicTypes.toConnectValue( + SimpleTypes.POINT.schema, Values.point(7203, 1.0, 2.5).asPoint()))) } eventually(30.seconds) { session.run("MATCH (n:User) RETURN n", emptyMap()).single() } @@ -88,7 +109,13 @@ class Neo4jNodePatternIT { .asNode() should { it.labels() shouldBe listOf("User") - it.asMap() shouldBe mapOf("id" to 1L, "name" to "john", "surname" to "doe") + it.asMap() shouldBe + mapOf( + "id" to 1L, + "name" to "john", + "surname" to "doe", + "dob" to LocalDate.of(1995, 1, 1), + "place" to Values.point(7203, 1.0, 2.5).asPoint()) } } @@ -633,4 +660,13 @@ class Neo4jNodePatternIT { } } } + + @KeyValueConverter(key = KafkaConverter.AVRO, value = KafkaConverter.AVRO) + class Neo4jNodePatternAvroIT : Neo4jNodePatternIT() + + @KeyValueConverter(key = KafkaConverter.JSON_SCHEMA, value = KafkaConverter.JSON_SCHEMA) + class Neo4jNodePatternJsonIT : Neo4jNodePatternIT() + + @KeyValueConverter(key = KafkaConverter.PROTOBUF, value = KafkaConverter.PROTOBUF) + class Neo4jNodePatternProtobufIT : Neo4jNodePatternIT() } diff --git a/sink-connector/src/test/kotlin/org/neo4j/connectors/kafka/sink/Neo4jRelationshipPatternIT.kt b/sink-connector/src/test/kotlin/org/neo4j/connectors/kafka/sink/Neo4jRelationshipPatternIT.kt index 73764ad34..8028a4477 100644 --- a/sink-connector/src/test/kotlin/org/neo4j/connectors/kafka/sink/Neo4jRelationshipPatternIT.kt +++ b/sink-connector/src/test/kotlin/org/neo4j/connectors/kafka/sink/Neo4jRelationshipPatternIT.kt @@ -22,20 +22,26 @@ import io.kotest.matchers.collections.shouldContainAll import io.kotest.matchers.collections.shouldHaveSize import io.kotest.matchers.should import io.kotest.matchers.shouldBe +import java.time.LocalDate import kotlin.time.Duration.Companion.seconds import org.apache.kafka.connect.data.Schema import org.apache.kafka.connect.data.SchemaBuilder import org.apache.kafka.connect.data.Struct import org.junit.jupiter.api.Test +import org.neo4j.connectors.kafka.data.DynamicTypes +import org.neo4j.connectors.kafka.data.SimpleTypes import org.neo4j.connectors.kafka.testing.TestSupport.runTest +import org.neo4j.connectors.kafka.testing.format.KafkaConverter +import org.neo4j.connectors.kafka.testing.format.KeyValueConverter import org.neo4j.connectors.kafka.testing.kafka.ConvertingKafkaProducer import org.neo4j.connectors.kafka.testing.kafka.KafkaMessage import org.neo4j.connectors.kafka.testing.sink.Neo4jSink import org.neo4j.connectors.kafka.testing.sink.RelationshipPatternStrategy import org.neo4j.connectors.kafka.testing.sink.TopicProducer import org.neo4j.driver.Session +import org.neo4j.driver.Values -class Neo4jRelationshipPatternIT { +abstract class Neo4jRelationshipPatternIT { companion object { const val TOPIC = "test" @@ -62,10 +68,24 @@ class Neo4jRelationshipPatternIT { SchemaBuilder.struct() .field("userId", Schema.INT64_SCHEMA) .field("productId", Schema.INT64_SCHEMA) + .field("at", SimpleTypes.LOCALDATE_STRUCT.schema) + .field("place", SimpleTypes.POINT.schema) .build() .let { schema -> producer.publish( - valueSchema = schema, value = Struct(schema).put("userId", 1L).put("productId", 2L)) + valueSchema = schema, + value = + Struct(schema) + .put("userId", 1L) + .put("productId", 2L) + .put( + "at", + DynamicTypes.toConnectValue( + SimpleTypes.LOCALDATE_STRUCT.schema, LocalDate.of(1995, 1, 1))) + .put( + "place", + DynamicTypes.toConnectValue( + SimpleTypes.POINT.schema, Values.point(7203, 1.0, 2.5).asPoint()))) } eventually(30.seconds) { @@ -78,7 +98,14 @@ class Neo4jRelationshipPatternIT { it.asMap() shouldBe mapOf("userId" to 1L) } - result.get("r").asRelationship() should { it.type() shouldBe "BOUGHT" } + result.get("r").asRelationship() should + { + it.type() shouldBe "BOUGHT" + it.asMap() shouldBe + mapOf( + "at" to LocalDate.of(1995, 1, 1), + "place" to Values.point(7203, 1.0, 2.5).asPoint()) + } result.get("p").asNode() should { @@ -873,4 +900,13 @@ class Neo4jRelationshipPatternIT { } } } + + @KeyValueConverter(key = KafkaConverter.AVRO, value = KafkaConverter.AVRO) + class Neo4jRelationshipPatternAvroIT : Neo4jRelationshipPatternIT() + + @KeyValueConverter(key = KafkaConverter.JSON_SCHEMA, value = KafkaConverter.JSON_SCHEMA) + class Neo4jRelationshipPatternJsonIT : Neo4jRelationshipPatternIT() + + @KeyValueConverter(key = KafkaConverter.PROTOBUF, value = KafkaConverter.PROTOBUF) + class Neo4jRelationshipPatternProtobufIT : Neo4jRelationshipPatternIT() } From 67f0a8916929418768cf56a87a3bab473e4901b6 Mon Sep 17 00:00:00 2001 From: Ali Ince Date: Fri, 31 May 2024 16:33:42 +0100 Subject: [PATCH 12/14] fix: add another level of subquery for proper ordering --- .../kafka/sink/strategy/NodePatternHandler.kt | 8 +- .../strategy/RelationshipPatternHandler.kt | 8 +- .../sink/strategy/NodePatternHandlerTest.kt | 225 +++++----- .../RelationshipPatternHandlerTest.kt | 408 ++++++++++-------- 4 files changed, 354 insertions(+), 295 deletions(-) diff --git a/sink/src/main/kotlin/org/neo4j/connectors/kafka/sink/strategy/NodePatternHandler.kt b/sink/src/main/kotlin/org/neo4j/connectors/kafka/sink/strategy/NodePatternHandler.kt index fd600d8c8..d270a873a 100644 --- a/sink/src/main/kotlin/org/neo4j/connectors/kafka/sink/strategy/NodePatternHandler.kt +++ b/sink/src/main/kotlin/org/neo4j/connectors/kafka/sink/strategy/NodePatternHandler.kt @@ -111,8 +111,12 @@ class NodePatternHandler( return renderer.render( Cypher.unwind(Cypher.parameter(EVENTS)) .`as`(NAME_EVENT) - .call(buildCreateStatement(NAME_EVENT, createOperation, node)) - .call(buildDeleteStatement(NAME_EVENT, deleteOperation, node)) + .call( + Cypher.call(buildCreateStatement(NAME_EVENT, createOperation, node)) + .call(buildDeleteStatement(NAME_EVENT, deleteOperation, node)) + .returning(NAME_CREATED, NAME_DELETED) + .build(), + NAME_EVENT) .returning( Cypher.raw("sum(${'$'}E)", NAME_CREATED).`as`(NAME_CREATED), Cypher.raw("sum(${'$'}E)", NAME_DELETED).`as`(NAME_DELETED)) diff --git a/sink/src/main/kotlin/org/neo4j/connectors/kafka/sink/strategy/RelationshipPatternHandler.kt b/sink/src/main/kotlin/org/neo4j/connectors/kafka/sink/strategy/RelationshipPatternHandler.kt index a6e6da2bc..ba63ab8f1 100644 --- a/sink/src/main/kotlin/org/neo4j/connectors/kafka/sink/strategy/RelationshipPatternHandler.kt +++ b/sink/src/main/kotlin/org/neo4j/connectors/kafka/sink/strategy/RelationshipPatternHandler.kt @@ -144,8 +144,12 @@ class RelationshipPatternHandler( return renderer.render( Cypher.unwind(Cypher.parameter(EVENTS)) .`as`(NAME_EVENT) - .call(buildCreateStatement(startNode, endNode, relationship, createOperation)) - .call(buildDeleteStatement(relationship, deleteOperation)) + .call( + Cypher.call(buildCreateStatement(startNode, endNode, relationship, createOperation)) + .call(buildDeleteStatement(relationship, deleteOperation)) + .returning(NAME_CREATED, NAME_DELETED) + .build(), + NAME_EVENT) .returning( Cypher.raw("sum(${'$'}E)", NAME_CREATED).`as`(NAME_CREATED), Cypher.raw("sum(${'$'}E)", NAME_DELETED).`as`(NAME_DELETED)) diff --git a/sink/src/test/kotlin/org/neo4j/connectors/kafka/sink/strategy/NodePatternHandlerTest.kt b/sink/src/test/kotlin/org/neo4j/connectors/kafka/sink/strategy/NodePatternHandlerTest.kt index 775279c8e..4d644955c 100644 --- a/sink/src/test/kotlin/org/neo4j/connectors/kafka/sink/strategy/NodePatternHandlerTest.kt +++ b/sink/src/test/kotlin/org/neo4j/connectors/kafka/sink/strategy/NodePatternHandlerTest.kt @@ -47,20 +47,23 @@ class NodePatternHandlerTest : HandlerTest() { CypherParser.parse( """ UNWIND ${'$'}events AS event - CALL { WITH event - WITH event WHERE event[0] = 'C' - WITH event[1] AS event - MERGE (n:`ALabel` {id: event.keys.id}) - SET n += event.properties - RETURN count(n) AS created - } - CALL { WITH event - WITH event WHERE event[0] = 'D' - WITH event[1] AS event - MATCH (n:`ALabel` {id: event.keys.id}) - DETACH DELETE n - RETURN count(n) AS deleted - } + CALL { WITH event + CALL { WITH event + WITH event WHERE event[0] = 'C' + WITH event[1] AS event + MERGE (n:`ALabel` {id: event.keys.id}) + SET n += event.properties + RETURN count(n) AS created + } + CALL { WITH event + WITH event WHERE event[0] = 'D' + WITH event[1] AS event + MATCH (n:`ALabel` {id: event.keys.id}) + DETACH DELETE n + RETURN count(n) AS deleted + } + RETURN created, deleted + } RETURN sum(created) AS created, sum(deleted) AS deleted """ .trimIndent()) @@ -81,20 +84,23 @@ class NodePatternHandlerTest : HandlerTest() { CypherParser.parse( """ UNWIND ${'$'}events AS event - CALL { WITH event - WITH event WHERE event[0] = 'C' - WITH event[1] AS event - MERGE (n:`ALabel` {id: event.keys.id}) - SET n += event.properties - RETURN count(n) AS created - } - CALL { WITH event - WITH event WHERE event[0] = 'D' - WITH event[1] AS event - MATCH (n:`ALabel` {id: event.keys.id}) - DETACH DELETE n - RETURN count(n) AS deleted - } + CALL { WITH event + CALL { WITH event + WITH event WHERE event[0] = 'C' + WITH event[1] AS event + MERGE (n:`ALabel` {id: event.keys.id}) + SET n += event.properties + RETURN count(n) AS created + } + CALL { WITH event + WITH event WHERE event[0] = 'D' + WITH event[1] AS event + MATCH (n:`ALabel` {id: event.keys.id}) + DETACH DELETE n + RETURN count(n) AS deleted + } + RETURN created, deleted + } RETURN sum(created) AS created, sum(deleted) AS deleted """ .trimIndent()) @@ -115,20 +121,23 @@ class NodePatternHandlerTest : HandlerTest() { CypherParser.parse( """ UNWIND ${'$'}events AS event - CALL { WITH event - WITH event WHERE event[0] = 'C' - WITH event[1] AS event - MERGE (n:`ALabel` {idA: event.keys.idA, idB: event.keys.idB}) - SET n += event.properties - RETURN count(n) AS created - } - CALL { WITH event - WITH event WHERE event[0] = 'D' - WITH event[1] AS event - MATCH (n:`ALabel` {idA: event.keys.idA, idB: event.keys.idB}) - DETACH DELETE n - RETURN count(n) AS deleted - } + CALL { WITH event + CALL { WITH event + WITH event WHERE event[0] = 'C' + WITH event[1] AS event + MERGE (n:`ALabel` {idA: event.keys.idA, idB: event.keys.idB}) + SET n += event.properties + RETURN count(n) AS created + } + CALL { WITH event + WITH event WHERE event[0] = 'D' + WITH event[1] AS event + MATCH (n:`ALabel` {idA: event.keys.idA, idB: event.keys.idB}) + DETACH DELETE n + RETURN count(n) AS deleted + } + RETURN created, deleted + } RETURN sum(created) AS created, sum(deleted) AS deleted """ .trimIndent()) @@ -149,21 +158,24 @@ class NodePatternHandlerTest : HandlerTest() { CypherParser.parse( """ UNWIND ${'$'}events AS event - CALL { WITH event - WITH event WHERE event[0] = 'C' - WITH event[1] AS event - MERGE (n:`ALabel` {id: event.keys.id}) - SET n = event.properties - SET n += event.keys - RETURN count(n) AS created - } - CALL { WITH event - WITH event WHERE event[0] = 'D' - WITH event[1] AS event - MATCH (n:`ALabel` {id: event.keys.id}) - DETACH DELETE n - RETURN count(n) AS deleted - } + CALL { WITH event + CALL { WITH event + WITH event WHERE event[0] = 'C' + WITH event[1] AS event + MERGE (n:`ALabel` {id: event.keys.id}) + SET n = event.properties + SET n += event.keys + RETURN count(n) AS created + } + CALL { WITH event + WITH event WHERE event[0] = 'D' + WITH event[1] AS event + MATCH (n:`ALabel` {id: event.keys.id}) + DETACH DELETE n + RETURN count(n) AS deleted + } + RETURN created, deleted + } RETURN sum(created) AS created, sum(deleted) AS deleted """ .trimIndent()) @@ -184,21 +196,24 @@ class NodePatternHandlerTest : HandlerTest() { CypherParser.parse( """ UNWIND ${'$'}events AS event - CALL { WITH event - WITH event WHERE event[0] = 'C' - WITH event[1] AS event - MERGE (n:`ALabel`:`BLabel` {id: event.keys.id}) - SET n = event.properties - SET n += event.keys - RETURN count(n) AS created - } - CALL { WITH event - WITH event WHERE event[0] = 'D' - WITH event[1] AS event - MATCH (n:`ALabel`:`BLabel` {id: event.keys.id}) - DETACH DELETE n - RETURN count(n) AS deleted - } + CALL { WITH event + CALL { WITH event + WITH event WHERE event[0] = 'C' + WITH event[1] AS event + MERGE (n:`ALabel`:`BLabel` {id: event.keys.id}) + SET n = event.properties + SET n += event.keys + RETURN count(n) AS created + } + CALL { WITH event + WITH event WHERE event[0] = 'D' + WITH event[1] AS event + MATCH (n:`ALabel`:`BLabel` {id: event.keys.id}) + DETACH DELETE n + RETURN count(n) AS deleted + } + RETURN created, deleted + } RETURN sum(created) AS created, sum(deleted) AS deleted """ .trimIndent()) @@ -219,21 +234,24 @@ class NodePatternHandlerTest : HandlerTest() { CypherParser.parse( """ UNWIND ${'$'}events AS event - CALL { WITH event - WITH event WHERE event[0] = 'C' - WITH event[1] AS event - MERGE (n:`ALabel With Space`:`BLabel` {id: event.keys.id, `another id`: event.keys.`another id`}) - SET n = event.properties - SET n += event.keys - RETURN count(n) AS created - } - CALL { WITH event - WITH event WHERE event[0] = 'D' - WITH event[1] AS event - MATCH (n:`ALabel With Space`:`BLabel` {id: event.keys.id, `another id`: event.keys.`another id`}) - DETACH DELETE n - RETURN count(n) AS deleted - } + CALL { WITH event + CALL { WITH event + WITH event WHERE event[0] = 'C' + WITH event[1] AS event + MERGE (n:`ALabel With Space`:`BLabel` {id: event.keys.id, `another id`: event.keys.`another id`}) + SET n = event.properties + SET n += event.keys + RETURN count(n) AS created + } + CALL { WITH event + WITH event WHERE event[0] = 'D' + WITH event[1] AS event + MATCH (n:`ALabel With Space`:`BLabel` {id: event.keys.id, `another id`: event.keys.`another id`}) + DETACH DELETE n + RETURN count(n) AS deleted + } + RETURN created, deleted + } RETURN sum(created) AS created, sum(deleted) AS deleted """ .trimIndent()) @@ -638,21 +656,24 @@ class NodePatternHandlerTest : HandlerTest() { CypherParser.parse( """ UNWIND ${'$'}events AS event - CALL { WITH event - WITH event WHERE event[0] = 'C' - WITH event[1] AS event - MERGE (n:`ALabel` {id: event.keys.id}) - SET n = event.properties - SET n += event.keys - RETURN count(n) AS created - } - CALL { WITH event - WITH event WHERE event[0] = 'D' - WITH event[1] AS event - MATCH (n:`ALabel` {id: event.keys.id}) - DETACH DELETE n - RETURN count(n) AS deleted - } + CALL { WITH event + CALL { WITH event + WITH event WHERE event[0] = 'C' + WITH event[1] AS event + MERGE (n:`ALabel` {id: event.keys.id}) + SET n = event.properties + SET n += event.keys + RETURN count(n) AS created + } + CALL { WITH event + WITH event WHERE event[0] = 'D' + WITH event[1] AS event + MATCH (n:`ALabel` {id: event.keys.id}) + DETACH DELETE n + RETURN count(n) AS deleted + } + RETURN created, deleted + } RETURN sum(created) AS created, sum(deleted) AS deleted """ .trimIndent(), diff --git a/sink/src/test/kotlin/org/neo4j/connectors/kafka/sink/strategy/RelationshipPatternHandlerTest.kt b/sink/src/test/kotlin/org/neo4j/connectors/kafka/sink/strategy/RelationshipPatternHandlerTest.kt index 97bbe53af..ffd043593 100644 --- a/sink/src/test/kotlin/org/neo4j/connectors/kafka/sink/strategy/RelationshipPatternHandlerTest.kt +++ b/sink/src/test/kotlin/org/neo4j/connectors/kafka/sink/strategy/RelationshipPatternHandlerTest.kt @@ -45,24 +45,27 @@ class RelationshipPatternHandlerTest : HandlerTest() { CypherParser.parse( """ UNWIND ${'$'}events AS event - CALL { WITH event - WITH event WHERE event[0] = 'C' - WITH event[1] AS event - MERGE (start:`LabelA` {idStart: event.start.keys.idStart}) - SET start += event.start.properties - MERGE (end:`LabelB` {idEnd: event.end.keys.idEnd}) - SET end += event.end.properties - MERGE (start)-[relationship:`REL_TYPE` {}]->(end) - SET relationship += event.properties - RETURN count(relationship) AS created - } - CALL { WITH event - WITH event WHERE event[0] = 'D' - WITH event[1] AS event - MATCH (start:`LabelA` {idStart: event.start.keys.idStart})-[relationship:`REL_TYPE` {}]->(end:`LabelB` {idEnd: event.end.keys.idEnd}) - DELETE relationship - RETURN count(relationship) AS deleted - } + CALL { WITH event + CALL { WITH event + WITH event WHERE event[0] = 'C' + WITH event[1] AS event + MERGE (start:`LabelA` {idStart: event.start.keys.idStart}) + SET start += event.start.properties + MERGE (end:`LabelB` {idEnd: event.end.keys.idEnd}) + SET end += event.end.properties + MERGE (start)-[relationship:`REL_TYPE` {}]->(end) + SET relationship += event.properties + RETURN count(relationship) AS created + } + CALL { WITH event + WITH event WHERE event[0] = 'D' + WITH event[1] AS event + MATCH (start:`LabelA` {idStart: event.start.keys.idStart})-[relationship:`REL_TYPE` {}]->(end:`LabelB` {idEnd: event.end.keys.idEnd}) + DELETE relationship + RETURN count(relationship) AS deleted + } + RETURN created, deleted + } RETURN sum(created) AS created, sum(deleted) AS deleted """ .trimIndent()) @@ -84,24 +87,27 @@ class RelationshipPatternHandlerTest : HandlerTest() { CypherParser.parse( """ UNWIND ${'$'}events AS event - CALL { WITH event - WITH event WHERE event[0] = 'C' - WITH event[1] AS event - MERGE (start:`LabelA` {id: event.start.keys.id}) - SET start += event.start.properties - MERGE (end:`LabelB` {id: event.end.keys.id}) - SET end += event.end.properties - MERGE (start)-[relationship:`REL_TYPE` {}]->(end) - SET relationship += event.properties - RETURN count(relationship) AS created - } - CALL { WITH event - WITH event WHERE event[0] = 'D' - WITH event[1] AS event - MATCH (start:`LabelA` {id: event.start.keys.id})-[relationship:`REL_TYPE` {}]->(end:`LabelB` {id: event.end.keys.id}) - DELETE relationship - RETURN count(relationship) AS deleted - } + CALL { WITH event + CALL { WITH event + WITH event WHERE event[0] = 'C' + WITH event[1] AS event + MERGE (start:`LabelA` {id: event.start.keys.id}) + SET start += event.start.properties + MERGE (end:`LabelB` {id: event.end.keys.id}) + SET end += event.end.properties + MERGE (start)-[relationship:`REL_TYPE` {}]->(end) + SET relationship += event.properties + RETURN count(relationship) AS created + } + CALL { WITH event + WITH event WHERE event[0] = 'D' + WITH event[1] AS event + MATCH (start:`LabelA` {id: event.start.keys.id})-[relationship:`REL_TYPE` {}]->(end:`LabelB` {id: event.end.keys.id}) + DELETE relationship + RETURN count(relationship) AS deleted + } + RETURN created, deleted + } RETURN sum(created) AS created, sum(deleted) AS deleted """ .trimIndent()) @@ -123,24 +129,27 @@ class RelationshipPatternHandlerTest : HandlerTest() { CypherParser.parse( """ UNWIND ${'$'}events AS event - CALL { WITH event - WITH event WHERE event[0] = 'C' - WITH event[1] AS event - MERGE (start:`LabelA` {id: event.start.keys.id}) - SET start += event.start.properties - MERGE (end:`LabelB` {id: event.end.keys.id}) - SET end += event.end.properties - MERGE (start)-[relationship:`REL_TYPE` {rel_id: event.keys.rel_id}]->(end) - SET relationship += event.properties - RETURN count(relationship) AS created - } - CALL { WITH event - WITH event WHERE event[0] = 'D' - WITH event[1] AS event - MATCH (start:`LabelA` {id: event.start.keys.id})-[relationship:`REL_TYPE` {rel_id: event.keys.rel_id}]->(end:`LabelB` {id: event.end.keys.id}) - DELETE relationship - RETURN count(relationship) AS deleted - } + CALL { WITH event + CALL { WITH event + WITH event WHERE event[0] = 'C' + WITH event[1] AS event + MERGE (start:`LabelA` {id: event.start.keys.id}) + SET start += event.start.properties + MERGE (end:`LabelB` {id: event.end.keys.id}) + SET end += event.end.properties + MERGE (start)-[relationship:`REL_TYPE` {rel_id: event.keys.rel_id}]->(end) + SET relationship += event.properties + RETURN count(relationship) AS created + } + CALL { WITH event + WITH event WHERE event[0] = 'D' + WITH event[1] AS event + MATCH (start:`LabelA` {id: event.start.keys.id})-[relationship:`REL_TYPE` {rel_id: event.keys.rel_id}]->(end:`LabelB` {id: event.end.keys.id}) + DELETE relationship + RETURN count(relationship) AS deleted + } + RETURN created, deleted + } RETURN sum(created) AS created, sum(deleted) AS deleted """ .trimIndent()) @@ -162,26 +171,29 @@ class RelationshipPatternHandlerTest : HandlerTest() { CypherParser.parse( """ UNWIND ${'$'}events AS event - CALL { WITH event - WITH event WHERE event[0] = 'C' - WITH event[1] AS event - MERGE (start:`LabelA` {id: event.start.keys.id}) - SET start = event.start.properties - SET start += event.start.keys - MERGE (end:`LabelB` {id: event.end.keys.id}) - SET end = event.end.properties - SET end += event.end.keys - MERGE (start)-[relationship:`REL_TYPE` {rel_id: event.keys.rel_id}]->(end) - SET relationship += event.properties - RETURN count(relationship) AS created - } - CALL { WITH event - WITH event WHERE event[0] = 'D' - WITH event[1] AS event - MATCH (start:`LabelA` {id: event.start.keys.id})-[relationship:`REL_TYPE` {rel_id: event.keys.rel_id}]->(end:`LabelB` {id: event.end.keys.id}) - DELETE relationship - RETURN count(relationship) AS deleted - } + CALL { WITH event + CALL { WITH event + WITH event WHERE event[0] = 'C' + WITH event[1] AS event + MERGE (start:`LabelA` {id: event.start.keys.id}) + SET start = event.start.properties + SET start += event.start.keys + MERGE (end:`LabelB` {id: event.end.keys.id}) + SET end = event.end.properties + SET end += event.end.keys + MERGE (start)-[relationship:`REL_TYPE` {rel_id: event.keys.rel_id}]->(end) + SET relationship += event.properties + RETURN count(relationship) AS created + } + CALL { WITH event + WITH event WHERE event[0] = 'D' + WITH event[1] AS event + MATCH (start:`LabelA` {id: event.start.keys.id})-[relationship:`REL_TYPE` {rel_id: event.keys.rel_id}]->(end:`LabelB` {id: event.end.keys.id}) + DELETE relationship + RETURN count(relationship) AS deleted + } + RETURN created, deleted + } RETURN sum(created) AS created, sum(deleted) AS deleted """ .trimIndent()) @@ -203,25 +215,28 @@ class RelationshipPatternHandlerTest : HandlerTest() { CypherParser.parse( """ UNWIND ${'$'}events AS event - CALL { WITH event - WITH event WHERE event[0] = 'C' - WITH event[1] AS event - MERGE (start:`LabelA` {id: event.start.keys.id}) - SET start += event.start.properties - MERGE (end:`LabelB` {id: event.end.keys.id}) - SET end += event.end.properties - MERGE (start)-[relationship:`REL_TYPE` {rel_id: event.keys.rel_id}]->(end) - SET relationship = event.properties - SET relationship += event.keys - RETURN count(relationship) AS created - } - CALL { WITH event - WITH event WHERE event[0] = 'D' - WITH event[1] AS event - MATCH (start:`LabelA` {id: event.start.keys.id})-[relationship:`REL_TYPE` {rel_id: event.keys.rel_id}]->(end:`LabelB` {id: event.end.keys.id}) - DELETE relationship - RETURN count(relationship) AS deleted - } + CALL { WITH event + CALL { WITH event + WITH event WHERE event[0] = 'C' + WITH event[1] AS event + MERGE (start:`LabelA` {id: event.start.keys.id}) + SET start += event.start.properties + MERGE (end:`LabelB` {id: event.end.keys.id}) + SET end += event.end.properties + MERGE (start)-[relationship:`REL_TYPE` {rel_id: event.keys.rel_id}]->(end) + SET relationship = event.properties + SET relationship += event.keys + RETURN count(relationship) AS created + } + CALL { WITH event + WITH event WHERE event[0] = 'D' + WITH event[1] AS event + MATCH (start:`LabelA` {id: event.start.keys.id})-[relationship:`REL_TYPE` {rel_id: event.keys.rel_id}]->(end:`LabelB` {id: event.end.keys.id}) + DELETE relationship + RETURN count(relationship) AS deleted + } + RETURN created, deleted + } RETURN sum(created) AS created, sum(deleted) AS deleted """ .trimIndent()) @@ -243,27 +258,30 @@ class RelationshipPatternHandlerTest : HandlerTest() { CypherParser.parse( """ UNWIND ${'$'}events AS event - CALL { WITH event - WITH event WHERE event[0] = 'C' - WITH event[1] AS event - MERGE (start:`LabelA` {id: event.start.keys.id}) - SET start = event.start.properties - SET start += event.start.keys - MERGE (end:`LabelB` {id: event.end.keys.id}) - SET end = event.end.properties - SET end += event.end.keys - MERGE (start)-[relationship:`REL_TYPE` {rel_id: event.keys.rel_id}]->(end) - SET relationship = event.properties - SET relationship += event.keys - RETURN count(relationship) AS created - } - CALL { WITH event - WITH event WHERE event[0] = 'D' - WITH event[1] AS event - MATCH (start:`LabelA` {id: event.start.keys.id})-[relationship:`REL_TYPE` {rel_id: event.keys.rel_id}]->(end:`LabelB` {id: event.end.keys.id}) - DELETE relationship - RETURN count(relationship) AS deleted - } + CALL { WITH event + CALL { WITH event + WITH event WHERE event[0] = 'C' + WITH event[1] AS event + MERGE (start:`LabelA` {id: event.start.keys.id}) + SET start = event.start.properties + SET start += event.start.keys + MERGE (end:`LabelB` {id: event.end.keys.id}) + SET end = event.end.properties + SET end += event.end.keys + MERGE (start)-[relationship:`REL_TYPE` {rel_id: event.keys.rel_id}]->(end) + SET relationship = event.properties + SET relationship += event.keys + RETURN count(relationship) AS created + } + CALL { WITH event + WITH event WHERE event[0] = 'D' + WITH event[1] AS event + MATCH (start:`LabelA` {id: event.start.keys.id})-[relationship:`REL_TYPE` {rel_id: event.keys.rel_id}]->(end:`LabelB` {id: event.end.keys.id}) + DELETE relationship + RETURN count(relationship) AS deleted + } + RETURN created, deleted + } RETURN sum(created) AS created, sum(deleted) AS deleted """ .trimIndent()) @@ -285,24 +303,27 @@ class RelationshipPatternHandlerTest : HandlerTest() { CypherParser.parse( """ UNWIND ${'$'}events AS event - CALL { WITH event - WITH event WHERE event[0] = 'C' - WITH event[1] AS event - MERGE (start:`LabelA` {id1: event.start.keys.id1, id2: event.start.keys.id2}) - SET start += event.start.properties - MERGE (end:`LabelB` {id1: event.end.keys.id1, id2: event.end.keys.id2}) - SET end += event.end.properties - MERGE (start)-[relationship:`REL_TYPE` {rel_id1: event.keys.rel_id1, rel_id2: event.keys.rel_id2}]->(end) - SET relationship += event.properties - RETURN count(relationship) AS created - } - CALL { WITH event - WITH event WHERE event[0] = 'D' - WITH event[1] AS event - MATCH (start:`LabelA` {id1: event.start.keys.id1, id2: event.start.keys.id2})-[relationship:`REL_TYPE` {rel_id1: event.keys.rel_id1, rel_id2: event.keys.rel_id2}]->(end:`LabelB` {id1: event.end.keys.id1, id2: event.end.keys.id2}) - DELETE relationship - RETURN count(relationship) AS deleted - } + CALL { WITH event + CALL { WITH event + WITH event WHERE event[0] = 'C' + WITH event[1] AS event + MERGE (start:`LabelA` {id1: event.start.keys.id1, id2: event.start.keys.id2}) + SET start += event.start.properties + MERGE (end:`LabelB` {id1: event.end.keys.id1, id2: event.end.keys.id2}) + SET end += event.end.properties + MERGE (start)-[relationship:`REL_TYPE` {rel_id1: event.keys.rel_id1, rel_id2: event.keys.rel_id2}]->(end) + SET relationship += event.properties + RETURN count(relationship) AS created + } + CALL { WITH event + WITH event WHERE event[0] = 'D' + WITH event[1] AS event + MATCH (start:`LabelA` {id1: event.start.keys.id1, id2: event.start.keys.id2})-[relationship:`REL_TYPE` {rel_id1: event.keys.rel_id1, rel_id2: event.keys.rel_id2}]->(end:`LabelB` {id1: event.end.keys.id1, id2: event.end.keys.id2}) + DELETE relationship + RETURN count(relationship) AS deleted + } + RETURN created, deleted + } RETURN sum(created) AS created, sum(deleted) AS deleted """ .trimIndent()) @@ -324,24 +345,27 @@ class RelationshipPatternHandlerTest : HandlerTest() { CypherParser.parse( """ UNWIND ${'$'}events AS event - CALL { WITH event - WITH event WHERE event[0] = 'C' - WITH event[1] AS event - MERGE (start:`LabelA`:`LabelC` {id: event.start.keys.id}) - SET start += event.start.properties - MERGE (end:`LabelB`:`LabelD` {id: event.end.keys.id}) - SET end += event.end.properties - MERGE (start)-[relationship:`REL_TYPE` {rel_id: event.keys.rel_id}]->(end) - SET relationship += event.properties - RETURN count(relationship) AS created - } - CALL { WITH event - WITH event WHERE event[0] = 'D' - WITH event[1] AS event - MATCH (start:`LabelA`:`LabelC` {id: event.start.keys.id})-[relationship:`REL_TYPE` {rel_id: event.keys.rel_id}]->(end:`LabelB`:`LabelD` {id: event.end.keys.id}) - DELETE relationship - RETURN count(relationship) AS deleted - } + CALL { WITH event + CALL { WITH event + WITH event WHERE event[0] = 'C' + WITH event[1] AS event + MERGE (start:`LabelA`:`LabelC` {id: event.start.keys.id}) + SET start += event.start.properties + MERGE (end:`LabelB`:`LabelD` {id: event.end.keys.id}) + SET end += event.end.properties + MERGE (start)-[relationship:`REL_TYPE` {rel_id: event.keys.rel_id}]->(end) + SET relationship += event.properties + RETURN count(relationship) AS created + } + CALL { WITH event + WITH event WHERE event[0] = 'D' + WITH event[1] AS event + MATCH (start:`LabelA`:`LabelC` {id: event.start.keys.id})-[relationship:`REL_TYPE` {rel_id: event.keys.rel_id}]->(end:`LabelB`:`LabelD` {id: event.end.keys.id}) + DELETE relationship + RETURN count(relationship) AS deleted + } + RETURN created, deleted + } RETURN sum(created) AS created, sum(deleted) AS deleted """ .trimIndent()) @@ -363,24 +387,27 @@ class RelationshipPatternHandlerTest : HandlerTest() { CypherParser.parse( """ UNWIND ${'$'}events AS event - CALL { WITH event - WITH event WHERE event[0] = 'C' - WITH event[1] AS event - MERGE (start:`Label A`:`Label C` {`a-id`: event.start.keys.`a-id`}) - SET start += event.start.properties - MERGE (end:`Label B`:`Label D` {`b-id`: event.end.keys.`b-id`}) - SET end += event.end.properties - MERGE (start)-[relationship:`REL TYPE` {`rel-id`: event.keys.`rel-id`}]->(end) - SET relationship += event.properties - RETURN count(relationship) AS created - } - CALL { WITH event - WITH event WHERE event[0] = 'D' - WITH event[1] AS event - MATCH (start:`Label A`:`Label C` {`a-id`: event.start.keys.`a-id`})-[relationship:`REL TYPE` {`rel-id`: event.keys.`rel-id`}]->(end:`Label B`:`Label D` {`b-id`: event.end.keys.`b-id`}) - DELETE relationship - RETURN count(relationship) AS deleted - } + CALL { WITH event + CALL { WITH event + WITH event WHERE event[0] = 'C' + WITH event[1] AS event + MERGE (start:`Label A`:`Label C` {`a-id`: event.start.keys.`a-id`}) + SET start += event.start.properties + MERGE (end:`Label B`:`Label D` {`b-id`: event.end.keys.`b-id`}) + SET end += event.end.properties + MERGE (start)-[relationship:`REL TYPE` {`rel-id`: event.keys.`rel-id`}]->(end) + SET relationship += event.properties + RETURN count(relationship) AS created + } + CALL { WITH event + WITH event WHERE event[0] = 'D' + WITH event[1] AS event + MATCH (start:`Label A`:`Label C` {`a-id`: event.start.keys.`a-id`})-[relationship:`REL TYPE` {`rel-id`: event.keys.`rel-id`}]->(end:`Label B`:`Label D` {`b-id`: event.end.keys.`b-id`}) + DELETE relationship + RETURN count(relationship) AS deleted + } + RETURN created, deleted + } RETURN sum(created) AS created, sum(deleted) AS deleted """ .trimIndent()) @@ -830,27 +857,30 @@ class RelationshipPatternHandlerTest : HandlerTest() { CypherParser.parse( """ UNWIND ${'$'}events AS event - CALL { WITH event - WITH event WHERE event[0] = 'C' - WITH event[1] AS event - MERGE (start:`LabelA` {id: event.start.keys.id}) - SET start ${if (mergeNodeProperties) "+" else "" }= event.start.properties - ${if (!mergeNodeProperties) "SET start += event.start.keys" else ""} - MERGE (end:`LabelB` {id: event.end.keys.id}) - SET end ${if (mergeNodeProperties) "+" else "" }= event.end.properties - ${if (!mergeNodeProperties) "SET end += event.end.keys" else ""} - MERGE (start)-[relationship:`REL_TYPE` {id: event.keys.id}]->(end) - SET relationship ${if (mergeRelationshipProperties) "+" else "" }= event.properties - ${if (!mergeRelationshipProperties) "SET relationship += event.keys" else ""} - RETURN count(relationship) AS created - } - CALL { WITH event - WITH event WHERE event[0] = 'D' - WITH event[1] AS event - MATCH (start:`LabelA` {id: event.start.keys.id})-[relationship:`REL_TYPE` {id: event.keys.id}]->(end:`LabelB` {id: event.end.keys.id}) - DELETE relationship - RETURN count(relationship) AS deleted - } + CALL { WITH event + CALL { WITH event + WITH event WHERE event[0] = 'C' + WITH event[1] AS event + MERGE (start:`LabelA` {id: event.start.keys.id}) + SET start ${if (mergeNodeProperties) "+" else "" }= event.start.properties + ${if (!mergeNodeProperties) "SET start += event.start.keys" else ""} + MERGE (end:`LabelB` {id: event.end.keys.id}) + SET end ${if (mergeNodeProperties) "+" else "" }= event.end.properties + ${if (!mergeNodeProperties) "SET end += event.end.keys" else ""} + MERGE (start)-[relationship:`REL_TYPE` {id: event.keys.id}]->(end) + SET relationship ${if (mergeRelationshipProperties) "+" else "" }= event.properties + ${if (!mergeRelationshipProperties) "SET relationship += event.keys" else ""} + RETURN count(relationship) AS created + } + CALL { WITH event + WITH event WHERE event[0] = 'D' + WITH event[1] AS event + MATCH (start:`LabelA` {id: event.start.keys.id})-[relationship:`REL_TYPE` {id: event.keys.id}]->(end:`LabelB` {id: event.end.keys.id}) + DELETE relationship + RETURN count(relationship) AS deleted + } + RETURN created, deleted + } RETURN sum(created) AS created, sum(deleted) AS deleted """ .trimIndent()) From 0bfc9155835c060cb742bc820be6fa9c51a26ab9 Mon Sep 17 00:00:00 2001 From: Ali Ince Date: Sat, 1 Jun 2024 21:18:10 +0100 Subject: [PATCH 13/14] test: make legacy producer also transactional --- .../kafka/testing/sink/LegacyNeo4jSinkExtension.kt | 5 ++++- 1 file changed, 4 insertions(+), 1 deletion(-) diff --git a/testing/src/main/kotlin/org/neo4j/connectors/kafka/testing/sink/LegacyNeo4jSinkExtension.kt b/testing/src/main/kotlin/org/neo4j/connectors/kafka/testing/sink/LegacyNeo4jSinkExtension.kt index cb81c74b2..79847067d 100644 --- a/testing/src/main/kotlin/org/neo4j/connectors/kafka/testing/sink/LegacyNeo4jSinkExtension.kt +++ b/testing/src/main/kotlin/org/neo4j/connectors/kafka/testing/sink/LegacyNeo4jSinkExtension.kt @@ -204,7 +204,10 @@ internal class LegacyNeo4jSinkExtension( ProducerConfig.VALUE_SERIALIZER_CLASS_CONFIG, valueConverter.serializerClass.name, ) - return KafkaProducer(properties) + properties.setProperty(ProducerConfig.TRANSACTIONAL_ID_CONFIG, UUID.randomUUID().toString()) + val producer = KafkaProducer(properties) + producer.initTransactions() + return producer } private fun resolveGenericProducer( From 09b8e834585568df5f1b3060ab0802daa56a794b Mon Sep 17 00:00:00 2001 From: emrehizal Date: Fri, 7 Jun 2024 22:49:21 +0200 Subject: [PATCH 14/14] test: implement requested changes --- .../kafka/sink/Neo4jNodePatternIT.kt | 98 +++++++++++++------ 1 file changed, 66 insertions(+), 32 deletions(-) diff --git a/sink-connector/src/test/kotlin/org/neo4j/connectors/kafka/sink/Neo4jNodePatternIT.kt b/sink-connector/src/test/kotlin/org/neo4j/connectors/kafka/sink/Neo4jNodePatternIT.kt index 5514da0c6..ac654682f 100644 --- a/sink-connector/src/test/kotlin/org/neo4j/connectors/kafka/sink/Neo4jNodePatternIT.kt +++ b/sink-connector/src/test/kotlin/org/neo4j/connectors/kafka/sink/Neo4jNodePatternIT.kt @@ -49,7 +49,9 @@ abstract class Neo4jNodePatternIT { const val TOPIC_2 = "test-2" } - @Neo4jSink(nodePattern = [NodePatternStrategy(TOPIC, "(:User{!id,name,surname})", false)]) + @Neo4jSink( + nodePattern = + [NodePatternStrategy(TOPIC, "(:User{!id,name,surname})", mergeNodeProperties = false)]) @Test fun `should create node from json string`( @TopicProducer(TOPIC) producer: ConvertingKafkaProducer, @@ -71,7 +73,10 @@ abstract class Neo4jNodePatternIT { } @Neo4jSink( - nodePattern = [NodePatternStrategy(TOPIC, "(:User{!id,name,surname,dob,place})", false)]) + nodePattern = + [ + NodePatternStrategy( + TOPIC, "(:User{!id,name,surname,dob,place})", mergeNodeProperties = false)]) @Test fun `should create node from struct`( @TopicProducer(TOPIC) producer: ConvertingKafkaProducer, @@ -119,7 +124,9 @@ abstract class Neo4jNodePatternIT { } } - @Neo4jSink(nodePattern = [NodePatternStrategy(TOPIC, "(:User{!id,name,surname})", false)]) + @Neo4jSink( + nodePattern = + [NodePatternStrategy(TOPIC, "(:User{!id,name,surname})", mergeNodeProperties = false)]) @Test fun `should create node from json byte array`( @TopicProducer(TOPIC) producer: ConvertingKafkaProducer, @@ -142,7 +149,11 @@ abstract class Neo4jNodePatternIT { } } - @Neo4jSink(nodePattern = [NodePatternStrategy(TOPIC, "(:User:Person{!id,name,surname})", false)]) + @Neo4jSink( + nodePattern = + [ + NodePatternStrategy( + TOPIC, "(:User:Person{!id,name,surname})", mergeNodeProperties = false)]) @Test fun `should create node with multiple labels pattern`( @TopicProducer(TOPIC) producer: ConvertingKafkaProducer, @@ -170,7 +181,9 @@ abstract class Neo4jNodePatternIT { nodePattern = [ NodePatternStrategy( - TOPIC, "(:User{!id: old_id,name: first_name,surname: last_name})", false)]) + TOPIC, + "(:User{!id: old_id,name: first_name,surname: last_name})", + mergeNodeProperties = false)]) @Test fun `should create node with aliases`( @TopicProducer(TOPIC) producer: ConvertingKafkaProducer, @@ -197,7 +210,7 @@ abstract class Neo4jNodePatternIT { NodePatternStrategy( TOPIC, "(:User{!id: __value.old_id,name: __key.first_name,surname: last_name})", - false)]) + mergeNodeProperties = false)]) @Test fun `should create and delete node with aliases`( @TopicProducer(TOPIC) producer: ConvertingKafkaProducer, @@ -233,24 +246,26 @@ abstract class Neo4jNodePatternIT { } } - @Neo4jSink(nodePattern = [NodePatternStrategy(TOPIC, "(:User{!id})", false)]) + @Neo4jSink( + nodePattern = [NodePatternStrategy(TOPIC, "(:User{!id})", mergeNodeProperties = false)]) @Test - fun `should create and delete node in the same kafka transaction`( + fun `should delete node`( @TopicProducer(TOPIC) producer: ConvertingKafkaProducer, session: Session ) = runTest { session.run("CREATE CONSTRAINT FOR (n:User) REQUIRE n.id IS KEY").consume() + session + .run( + "CREATE (n:User) SET n = ${'$'}props", + mapOf("props" to mapOf("id" to 1, "name" to "john", "surname" to "doe")), + ) + .consume() + producer.publish( - KafkaMessage( - keySchema = Schema.STRING_SCHEMA, - key = """{"id": 1}""", - valueSchema = Schema.STRING_SCHEMA, - value = """{"name": "john", "surname": "doe"}"""), - KafkaMessage( - keySchema = Schema.STRING_SCHEMA, - key = """{"id": 1}""", - )) + keySchema = Schema.STRING_SCHEMA, + key = """{"id": 1}""", + ) eventually(30.seconds) { session @@ -261,7 +276,9 @@ abstract class Neo4jNodePatternIT { } } - @Neo4jSink(nodePattern = [NodePatternStrategy(TOPIC, "(:User{!id,name,surname})", false)]) + @Neo4jSink( + nodePattern = + [NodePatternStrategy(TOPIC, "(:User{!id,name,surname})", mergeNodeProperties = false)]) @Test fun `should create, delete and recreate node`( @TopicProducer(TOPIC) producer: ConvertingKafkaProducer, @@ -291,7 +308,9 @@ abstract class Neo4jNodePatternIT { } } - @Neo4jSink(nodePattern = [NodePatternStrategy(TOPIC, "(:User{!id,name,surname})", false)]) + @Neo4jSink( + nodePattern = + [NodePatternStrategy(TOPIC, "(:User{!id,name,surname})", mergeNodeProperties = false)]) @Test fun `should create multiple nodes`( @TopicProducer(TOPIC) producer: ConvertingKafkaProducer, @@ -325,13 +344,15 @@ abstract class Neo4jNodePatternIT { } } - @Neo4jSink(nodePattern = [NodePatternStrategy(TOPIC, "(:User{!id,!name,surname})", false)]) + @Neo4jSink( + nodePattern = + [NodePatternStrategy(TOPIC, "(:User{!id,!name,surname})", mergeNodeProperties = false)]) @Test fun `should create node with composite key`( @TopicProducer(TOPIC) producer: ConvertingKafkaProducer, session: Session ) = runTest { - session.run("CREATE CONSTRAINT FOR (n:User) REQUIRE n.id IS KEY").consume() + session.run("CREATE CONSTRAINT FOR (n:User) REQUIRE (n.id, n.name) IS KEY").consume() producer.publish( keySchema = Schema.STRING_SCHEMA, @@ -348,7 +369,8 @@ abstract class Neo4jNodePatternIT { } } - @Neo4jSink(nodePattern = [NodePatternStrategy(TOPIC, "(:User{!id})", false)]) + @Neo4jSink( + nodePattern = [NodePatternStrategy(TOPIC, "(:User{!id})", mergeNodeProperties = false)]) @Test fun `should create node with nested properties`( @TopicProducer(TOPIC) producer: ConvertingKafkaProducer, @@ -376,7 +398,11 @@ abstract class Neo4jNodePatternIT { } } - @Neo4jSink(nodePattern = [NodePatternStrategy(TOPIC, "(:User{!id, -name, -surname})", false)]) + @Neo4jSink( + nodePattern = + [ + NodePatternStrategy( + TOPIC, "(:User{!id, -name, -surname})", mergeNodeProperties = false)]) @Test fun `should create node with excluded properties`( @TopicProducer(TOPIC) producer: ConvertingKafkaProducer, @@ -404,7 +430,10 @@ abstract class Neo4jNodePatternIT { } @Neo4jSink( - nodePattern = [NodePatternStrategy(TOPIC, "(:User{!id, created_at: __timestamp})", false)]) + nodePattern = + [ + NodePatternStrategy( + TOPIC, "(:User{!id, created_at: __timestamp})", mergeNodeProperties = false)]) @Test fun `should create node with timestamp`( @TopicProducer(TOPIC) producer: ConvertingKafkaProducer, @@ -434,7 +463,7 @@ abstract class Neo4jNodePatternIT { NodePatternStrategy( TOPIC, "(:User{!id: __key.old_id, name: __key.first_name, surname: __key.last_name})", - false)]) + mergeNodeProperties = false)]) @Test fun `should create and delete node with explicit properties from message key`( @TopicProducer(TOPIC) producer: ConvertingKafkaProducer, @@ -470,7 +499,8 @@ abstract class Neo4jNodePatternIT { } } - @Neo4jSink(nodePattern = [NodePatternStrategy(TOPIC, "(:User{!id})", false)]) + @Neo4jSink( + nodePattern = [NodePatternStrategy(TOPIC, "(:User{!id})", mergeNodeProperties = false)]) @Test fun `should update node`( @TopicProducer(TOPIC) producer: ConvertingKafkaProducer, @@ -504,7 +534,9 @@ abstract class Neo4jNodePatternIT { nodePattern = [ NodePatternStrategy( - TOPIC, "(:User{!id: old_id, name: first_name, surname: last_name})", false)]) + TOPIC, + "(:User{!id: old_id, name: first_name, surname: last_name})", + mergeNodeProperties = false)]) @Test fun `should update node with aliases`( @TopicProducer(TOPIC) producer: ConvertingKafkaProducer, @@ -535,8 +567,8 @@ abstract class Neo4jNodePatternIT { @Neo4jSink( nodePattern = [ - NodePatternStrategy(TOPIC_1, "(:User{!id})", false), - NodePatternStrategy(TOPIC_2, "(:Account{!id})", false)]) + NodePatternStrategy(TOPIC_1, "(:User{!id})", mergeNodeProperties = false), + NodePatternStrategy(TOPIC_2, "(:Account{!id})", mergeNodeProperties = false)]) @Test fun `should create nodes from multiple topics`( @TopicProducer(TOPIC_1) producer1: ConvertingKafkaProducer, @@ -575,7 +607,8 @@ abstract class Neo4jNodePatternIT { } } - @Neo4jSink(nodePattern = [NodePatternStrategy(TOPIC, "(:User{!id})", false)]) + @Neo4jSink( + nodePattern = [NodePatternStrategy(TOPIC, "(:User{!id})", mergeNodeProperties = false)]) @Test fun `should create 1000 nodes`( @TopicProducer(TOPIC) producer: ConvertingKafkaProducer, @@ -604,7 +637,7 @@ abstract class Neo4jNodePatternIT { } } - @Neo4jSink(nodePattern = [NodePatternStrategy(TOPIC, "(:User{!id})", true)]) + @Neo4jSink(nodePattern = [NodePatternStrategy(TOPIC, "(:User{!id})", mergeNodeProperties = true)]) @Test fun `should merge node properties`( @TopicProducer(TOPIC) producer: ConvertingKafkaProducer, @@ -633,7 +666,8 @@ abstract class Neo4jNodePatternIT { } } - @Neo4jSink(nodePattern = [NodePatternStrategy(TOPIC, "(:User{!id})", false)]) + @Neo4jSink( + nodePattern = [NodePatternStrategy(TOPIC, "(:User{!id})", mergeNodeProperties = false)]) @Test fun `should not merge node properties`( @TopicProducer(TOPIC) producer: ConvertingKafkaProducer,