diff --git a/app/com/lynxanalytics/biggraph/frontend_operations/ExportOperations.scala b/app/com/lynxanalytics/biggraph/frontend_operations/ExportOperations.scala index 4a6bd583fb..3c2b08f96d 100644 --- a/app/com/lynxanalytics/biggraph/frontend_operations/ExportOperations.scala +++ b/app/com/lynxanalytics/biggraph/frontend_operations/ExportOperations.scala @@ -2,6 +2,7 @@ package com.lynxanalytics.biggraph.frontend_operations import com.lynxanalytics.biggraph.SparkFreeEnvironment import com.lynxanalytics.biggraph.graph_api.Scripting._ +import com.lynxanalytics.biggraph.graph_util.Scripting._ import com.lynxanalytics.biggraph.graph_operations import com.lynxanalytics.biggraph.controllers._ @@ -153,4 +154,98 @@ class ExportOperations(env: SparkFreeEnvironment) extends OperationRegistry { .map(_ => wc.createSnapshotFromState(user, params("path"), getState)) } }) + + abstract class Neo4jAttributeExport(context: Operation.Context) extends TriggerableOperation(context) { + params ++= List( + Param("url", "Neo4j connection", defaultValue = "bolt://localhost:7687"), + Param("username", "Neo4j username", defaultValue = "neo4j"), + Param("password", "Neo4j password", defaultValue = "neo4j"), + NonNegInt("version", "Export repetition ID", default = 1)) + lazy val project = projectInput("graph") + val nodesOrRelationships: String + override def enabled = FEStatus.enabled + override def trigger(wc: WorkspaceController, gdc: GraphDrawingController): scala.concurrent.Future[Unit] = { + gdc.getComputeBoxResult(List(exportResult.gUID)) + } + override def getOutputs(): Map[BoxOutput, BoxOutputState] = { + params.validate() + Map(context.box.output( + context.meta.outputs(0)) -> BoxOutputState.from(exportResult, params.toMap - "password")) + } + def getAttribute(a: String): com.lynxanalytics.biggraph.graph_api.Attribute[_] + def exportResult() = { + val keys = splitParam("keys") + val attrs = (splitParam("to_export") ++ keys).toSet.toList + val t = graph_operations.AttributesToTable.run(attrs.map(a => a -> getAttribute(a))) + assert(keys.nonEmpty, "You have to choose one or more attributes to use as the keys for identifying the nodes in Neo4j.") + val op = graph_operations.ExportAttributesToNeo4j( + params("url"), params("username"), params("password"), params("labels"), keys, + params("version").toInt, nodesOrRelationships) + op(op.t, t).result.exportResult + } + } + + registerOp( + "Export vertex attributes to Neo4j", defaultIcon, ExportOperations, + List("graph"), List("exported"), new Neo4jAttributeExport(_) { + import Operation.Implicits._ + params ++= List( + Param("labels", "Node labels", defaultValue = ""), + Choice("keys", "Attribute(s) to use as key", + // Cannot join on internal ID ("") and stuff like that. + options = project.vertexAttrList.filter(!_.id.startsWith("<")), multipleChoice = true), + Choice("to_export", "Exported attributes", options = project.vertexAttrList, multipleChoice = true)) + val nodesOrRelationships = "nodes" + def getAttribute(a: String) = project.vertexAttributes(a) + }) + registerOp( + "Export edge attributes to Neo4j", defaultIcon, ExportOperations, + List("graph"), List("exported"), new Neo4jAttributeExport(_) { + import Operation.Implicits._ + params ++= List( + Param("labels", "Relationship labels", defaultValue = ""), + Choice("keys", "Attribute(s) to use as key", + // Cannot join on internal ID ("") and stuff like that. + options = project.edgeAttrList.filter(!_.id.startsWith("<")), multipleChoice = true), + Choice("to_export", "Exported attributes", options = project.edgeAttrList, multipleChoice = true)) + val nodesOrRelationships = "relationships" + def getAttribute(a: String) = project.edgeAttributes(a) + }) + + registerOp( + "Export graph to Neo4j", defaultIcon, ExportOperations, + List("graph"), List("exported"), new TriggerableOperation(_) { + import Operation.Implicits._ + params ++= List( + Param("url", "Neo4j connection", defaultValue = "bolt://localhost:7687"), + Param("username", "Neo4j username", defaultValue = "neo4j"), + Param("password", "Neo4j password", defaultValue = "neo4j"), + NonNegInt("version", "Export repetition ID", default = 1), + Choice("node_labels", "Attribute with node labels", + options = List(FEOption("", "")) ++ project.vertexAttrList[String]), + Choice("relationship_type", "Attribute with relationship type", + options = List(FEOption("", "")) ++ project.edgeAttrList[String])) + lazy val project = projectInput("graph") + override def enabled = FEStatus.enabled + override def trigger(wc: WorkspaceController, gdc: GraphDrawingController): scala.concurrent.Future[Unit] = { + gdc.getComputeBoxResult(List(exportResult.gUID)) + } + override def getOutputs(): Map[BoxOutput, BoxOutputState] = { + params.validate() + Map(context.box.output( + context.meta.outputs(0)) -> BoxOutputState.from(exportResult, params.toMap - "password")) + } + def exportResult() = { + val vsAttr = project.vertexAttributes.toMap + + (graph_operations.ExportGraphToNeo4j.VID -> project.vertexSet.idAttribute) + val esAttr = project.edgeAttributes.toMap + + (graph_operations.ExportGraphToNeo4j.SRCDST -> project.edgeBundle.srcDstAttribute) + val vs = graph_operations.AttributesToTable.run(vsAttr) + val es = graph_operations.AttributesToTable.run(esAttr) + val op = graph_operations.ExportGraphToNeo4j( + params("url"), params("username"), params("password"), params("node_labels"), + params("relationship_type"), params("version").toInt) + op(op.vs, vs)(op.es, es).result.exportResult + } + }) } diff --git a/app/com/lynxanalytics/biggraph/frontend_operations/ImportOperations.scala b/app/com/lynxanalytics/biggraph/frontend_operations/ImportOperations.scala index 9a9c1230a6..7681003f87 100644 --- a/app/com/lynxanalytics/biggraph/frontend_operations/ImportOperations.scala +++ b/app/com/lynxanalytics/biggraph/frontend_operations/ImportOperations.scala @@ -167,6 +167,15 @@ class ImportOperations(env: SparkFreeEnvironment) extends ProjectOperations(env) Map(context.box.output(context.meta.outputs(0)) -> BoxOutputState.from(state)) } + def isNameValid(name: String) = { + try { + SubProject.validateName(name) + true + } catch { + case e: AssertionError => false + } + } + override def runImport(env: com.lynxanalytics.biggraph.BigGraphEnvironment): ImportResult = { import org.apache.spark.sql.functions._ import org.apache.spark.sql.types @@ -202,6 +211,16 @@ class ImportOperations(env: SparkFreeEnvironment) extends ProjectOperations(env) val esp = table.toProject.viewer val srcAttr = esp.vertexAttributes("").asString val dstAttr = esp.vertexAttributes("").asString + def pullAttributes(embedding: EdgeBundle) = { + for ((name, attr) <- esp.vertexAttributes) { + // LynxKite attribute names cannot have '.'. + val renamed = name.replace(".", "_") + // Skip names that have other issues. + if (isNameValid(renamed)) { + project.edgeAttributes(renamed) = attr.pullVia(embedding) + } + } + } if (params("vertex_query").isEmpty) { val es = { val op = graph_operations.VerticesToEdges() @@ -210,19 +229,13 @@ class ImportOperations(env: SparkFreeEnvironment) extends ProjectOperations(env) project.vertexSet = es.vs project.newVertexAttribute("", es.stringId) project.edgeBundle = es.es - for ((name, attr) <- esp.vertexAttributes) { - // LynxKite attribute names cannot have '.'. - project.edgeAttributes(name.replace(".", "_")) = attr.pullVia(es.embedding) - } + pullAttributes(es.embedding) } else { val idAttr = project.vertexAttributes("").asString val es = graph_operations.ImportEdgesForExistingVertices.run( idAttr, idAttr, srcAttr, dstAttr) project.edgeBundle = es.edges - for ((name, attr) <- esp.vertexAttributes) { - // LynxKite attribute names cannot have '.'. - project.edgeAttributes(name.replace(".", "_")) = attr.pullVia(es.embedding) - } + pullAttributes(es.embedding) } } val state = json.Json.toJson(project.state).toString diff --git a/app/com/lynxanalytics/biggraph/graph_operations/ExportToNeo4j.scala b/app/com/lynxanalytics/biggraph/graph_operations/ExportToNeo4j.scala new file mode 100644 index 0000000000..5e0da5a670 --- /dev/null +++ b/app/com/lynxanalytics/biggraph/graph_operations/ExportToNeo4j.scala @@ -0,0 +1,139 @@ +// Backend operations for Neo4j export. +package com.lynxanalytics.biggraph.graph_operations +import com.lynxanalytics.biggraph.graph_api._ +import com.lynxanalytics.biggraph.graph_util.Timestamp +import org.apache.spark + +object ExportAttributesToNeo4j extends OpFromJson { + class Input extends MagicInputSignature { + val t = table + } + class Output(implicit instance: MetaGraphOperationInstance) extends MagicOutput(instance) { + val exportResult = scalar[String] + } + def fromJson(j: JsValue) = ExportAttributesToNeo4j( + (j \ "url").as[String], (j \ "username").as[String], (j \ "password").as[String], + (j \ "labels").as[String], (j \ "keys").as[Seq[String]], (j \ "version").as[Long], + (j \ "nodesOrRelationships").as[String]) +} + +// Makes it easy to send a DataFrame to a specified Neo4j instance. +case class Neo4jConnectionParameters(url: String, username: String, password: String) { + def send(df: spark.sql.DataFrame, query: String) { + df.write + .format("org.neo4j.spark.DataSource") + .option("authentication.type", "basic") + .option("authentication.basic.username", username) + .option("authentication.basic.password", password) + .option("url", url) + .option("query", query) + .save() + } +} + +case class ExportAttributesToNeo4j( + url: String, username: String, password: String, labels: String, keys: Seq[String], + version: Long, nodesOrRelationships: String) + extends SparkOperation[ExportAttributesToNeo4j.Input, ExportAttributesToNeo4j.Output] { + val neo = Neo4jConnectionParameters(url, username, password) + @transient override lazy val inputs = new ExportAttributesToNeo4j.Input() + def outputMeta(instance: MetaGraphOperationInstance) = new ExportAttributesToNeo4j.Output()(instance) + override def toJson = Json.obj( + "url" -> url, "username" -> username, "password" -> password, "labels" -> labels, + "keys" -> keys, "version" -> version, "nodesOrRelationships" -> nodesOrRelationships) + def execute( + inputDatas: DataSet, + o: ExportAttributesToNeo4j.Output, + output: OutputBuilder, + rc: RuntimeContext): Unit = { + implicit val ds = inputDatas + // Drop null keys. + val df = keys.foldLeft(inputs.t.df)((df, key) => df.filter(df(key).isNotNull)) + val keyMatch = keys.map(k => s"`$k`: event.`$k`").mkString(", ") + val query = nodesOrRelationships match { + case "nodes" => s"MATCH (n$labels {$keyMatch}) SET n += event" + case "relationships" => s"MATCH ()-[r$labels {$keyMatch}]-() SET r += event" + } + neo.send(df, query) + val exportResult = "Export done." + output(o.exportResult, exportResult) + } +} + +object ExportGraphToNeo4j extends OpFromJson { + class Input extends MagicInputSignature { + val vs = table + val es = table + } + class Output(implicit instance: MetaGraphOperationInstance) extends MagicOutput(instance) { + val exportResult = scalar[String] + } + def fromJson(j: JsValue) = ExportGraphToNeo4j( + (j \ "url").as[String], (j \ "username").as[String], (j \ "password").as[String], + (j \ "nodeLabelsColumn").as[String], (j \ "relationshipTypeColumn").as[String], + (j \ "version").as[Long]) + val VID = "!LynxKite ID" + val SRCDST = "!LynxKite endpoint IDs" + val SRCID = "!Source LynxKite ID" + val DSTID = "!Destination LynxKite ID" +} + +case class ExportGraphToNeo4j( + url: String, username: String, password: String, nodeLabelsColumn: String, + relationshipTypeColumn: String, version: Long) + extends SparkOperation[ExportGraphToNeo4j.Input, ExportGraphToNeo4j.Output] { + import ExportGraphToNeo4j._ + val neo = Neo4jConnectionParameters(url, username, password) + @transient override lazy val inputs = new ExportGraphToNeo4j.Input() + def outputMeta(instance: MetaGraphOperationInstance) = new ExportGraphToNeo4j.Output()(instance) + override def toJson = Json.obj( + "url" -> url, "username" -> username, "password" -> password, + "nodeLabelsColumn" -> nodeLabelsColumn, "relationshipTypeColumn" -> relationshipTypeColumn, + "version" -> version) + def execute( + inputDatas: DataSet, + o: ExportGraphToNeo4j.Output, + output: OutputBuilder, + rc: RuntimeContext): Unit = { + implicit val ds = inputDatas + val F = spark.sql.functions + // Prefix the internal IDs with the timestamp so different exports don't collide. + // Also save the timestamp so the created entities can be easily cleaned up. + val timestamp = F.lit(Timestamp.human) + val vs = inputs.vs.df + .withColumn(VID, F.concat(timestamp, F.lit(" "), F.col(VID))) + .withColumn("!LynxKite export timestamp", timestamp) + val es = inputs.es.df + .withColumn(SRCID, F.concat(timestamp, F.lit(" "), F.col(SRCDST + "._1"))) + .withColumn(DSTID, F.concat(timestamp, F.lit(" "), F.col(SRCDST + "._2"))) + .drop(SRCDST) + .withColumn("!LynxKite export timestamp", timestamp) + + if (nodeLabelsColumn.isEmpty) { + neo.send(vs, s""" + CREATE (n) + SET n += event + """) + } else { + neo.send(vs, s""" + CALL apoc.create.node(split(event.`$nodeLabelsColumn`, ','), event) YIELD node + RETURN 1 + """) + } + if (relationshipTypeColumn.isEmpty) { + neo.send(es, s""" + MATCH (src {`$VID`: event.`$SRCID`}), (dst {`$VID`: event.`$DSTID`}) + CREATE (src)-[r:EDGE]->(dst) + SET r += event + """) + } else { + neo.send(es, s""" + MATCH (src {`$VID`: event.`$SRCID`}), (dst {`$VID`: event.`$DSTID`}) + CALL apoc.create.relationship(src, event.`$relationshipTypeColumn`, event, dst) YIELD rel + RETURN 1 + """) + } + val exportResult = "Export done." + output(o.exportResult, exportResult) + } +} diff --git a/app/com/lynxanalytics/biggraph/graph_util/Scripting.scala b/app/com/lynxanalytics/biggraph/graph_util/Scripting.scala index 8f1a5466aa..947fdfc420 100644 --- a/app/com/lynxanalytics/biggraph/graph_util/Scripting.scala +++ b/app/com/lynxanalytics/biggraph/graph_util/Scripting.scala @@ -72,6 +72,11 @@ object Scripting { val op = graph_operations.AddReversedEdges() op(op.es, self).result.esPlus } + + def srcDstAttribute: Attribute[(ID, ID)] = { + val op = graph_operations.EdgeBundleAsAttribute() + op(op.edges, self).result.attr + } } implicit class RichContainedAttribute[T]( diff --git a/build.sbt b/build.sbt index 0aca0c7666..9cc4c6c722 100644 --- a/build.sbt +++ b/build.sbt @@ -95,6 +95,9 @@ libraryDependencies ++= Seq( "javax.media" % "jai_core" % "1.1.3" from "https://repo.osgeo.org/repository/geotools-releases/javax/media/jai_core/1.1.3/jai_core-1.1.3.jar", // Used for working with AVRO files. "org.apache.spark" %% "spark-avro" % sparkVersion.value, + // For Neo4j tests. + "org.testcontainers" % "testcontainers" % "1.14.3" % Test, + "org.testcontainers" % "neo4j" % "1.14.3" % Test, // Used for working with Delta tables. "io.delta" %% "delta-core" % "0.6.1" ) diff --git a/test/com/lynxanalytics/biggraph/frontend_operations/Neo4jExportImportTest.scala b/test/com/lynxanalytics/biggraph/frontend_operations/Neo4jExportImportTest.scala new file mode 100644 index 0000000000..32842d7912 --- /dev/null +++ b/test/com/lynxanalytics/biggraph/frontend_operations/Neo4jExportImportTest.scala @@ -0,0 +1,70 @@ +package com.lynxanalytics.biggraph.frontend_operations + +import com.lynxanalytics.biggraph.controllers.DirectoryEntry +import com.lynxanalytics.biggraph.graph_api.Edge +import com.lynxanalytics.biggraph.graph_api.Scripting._ +import com.lynxanalytics.biggraph.graph_api.GraphTestUtils._ + +class Neo4jContainer + extends org.testcontainers.containers.Neo4jContainer[Neo4jContainer]("neo4j:4.0.8-enterprise") + +class Neo4jExportImportTest extends OperationsTestBase { + val server = new Neo4jContainer() + .withoutAuthentication + .withEnv("NEO4J_ACCEPT_LICENSE_AGREEMENT", "yes") + .withEnv("NEO4JLABS_PLUGINS", "[\"apoc\"]") + + def exportExampleGraph() = { + val res = box("Create example graph") + .box("Export graph to Neo4j", Map("url" -> server.getBoltUrl)).exportResult + dataManager.get(res) + } + + test("full graph export and import") { + server.start() + exportExampleGraph() + val p = importBox("Import from Neo4j", Map("url" -> server.getBoltUrl)).project + assert(p.vertexAttributes.toMap.keySet == Set( + "!LynxKite ID", "!LynxKite export timestamp", "", "", + "age", "gender", "id", "income", "location", "name")) + assert(p.edgeAttributes.toMap.keySet == Set( + "", "", "", "", "comment", "weight")) + assert(get(p.vertexAttributes("name")).values.toSet == Set("Adam", "Bob", "Eve", "Isolated Joe")) + assert(get(p.edgeAttributes("weight")).values.toSet == Set(1.0, 2.0, 3.0, 4.0)) + server.stop() + } + + test("attribute export") { + server.start() + exportExampleGraph() + val g = box("Create example graph").box("Compute PageRank").box("Compute dispersion") + dataManager.get(g.box( + "Export vertex attributes to Neo4j", + Map("url" -> server.getBoltUrl, "keys" -> "name", "to_export" -> "page_rank")).exportResult) + dataManager.get(g.box( + "Export edge attributes to Neo4j", + Map("url" -> server.getBoltUrl, "keys" -> "comment", "to_export" -> "dispersion")).exportResult) + val p = importBox("Import from Neo4j", Map("url" -> server.getBoltUrl)).project + assert(get(p.vertexAttributes("page_rank")).values.toSet == + get(g.project.vertexAttributes("page_rank")).values.toSet) + assert(get(p.edgeAttributes("dispersion")).values.toSet == + get(g.project.edgeAttributes("dispersion")).values.toSet) + server.stop() + } + + test("export with labels and types") { + server.start() + val res = box("Create example graph").box("Export graph to Neo4j", Map( + "url" -> server.getBoltUrl, + "node_labels" -> "gender", "relationship_type" -> "comment")).exportResult + dataManager.get(res) + val p = importBox("Import from Neo4j", Map("url" -> server.getBoltUrl)).project + assert( + get(p.vertexAttributes("")).values.toList + .flatMap(_.asInstanceOf[scala.collection.mutable.WrappedArray[String]]).sorted + == Seq("Female", "Male", "Male", "Male")) + assert(get(p.edgeAttributes("")).values.toSet + == Set("Adam loves Eve", "Bob envies Adam", "Bob loves Eve", "Eve loves Adam")) + server.stop() + } +} diff --git a/web/app/help/operations/export-edge-attributes-to-neo4j.asciidoc b/web/app/help/operations/export-edge-attributes-to-neo4j.asciidoc new file mode 100644 index 0000000000..56c09a3d4c --- /dev/null +++ b/web/app/help/operations/export-edge-attributes-to-neo4j.asciidoc @@ -0,0 +1,46 @@ +### Export edge attributes to Neo4j + +Exports edge attributes from a graph in LynxKite to a +corresponding graph in https://neo4j.com/[Neo4j]. + +The relationships in Neo4j are identified by a key property (or properties). +You must have a corresponding edge attribute in LynxKite by the same name. +This will be used to find the right relationship to update in Neo4j. + +The properties of the Neo4j relationships will be updated with the exported edge attributes +using a Cypher query like this: + + UNWIND $events as event + MATCH ()-[r:TYPE {`key`: event.`key`}]-() + SET r += event + +In the event of duplicate keys on either end this will update the properties of all the matching +Neo4j relationships with the values from the last matching LynxKite edge. + +==== + +[p-url]#Neo4j connection#:: +The Neo4j connection string of the form `bolt://localhost:7687`. + +[p-username]#Neo4j username#:: +Username for the connection. + +[p-password]#Neo4j password#:: +Password for the connection. It will be saved in the workspace and visible to anyone with +access to the workspace. + +[p-version]#Export repetition ID#:: +LynxKite only re-computes outputs if parameters or inputs have changed. +This is true for exports too. If you want to repeat a previous export, you can increase this +export repetition ID parameter. + +[p-labels]#Relationship type#:: +Makes it possible to restrict the export to one relationship type in Neo4j. +This is useful to make sure no other relationship type is accidentally affected. +The format is as in Cypher: `:TYPE`. Leave empty to allow updating any node. + +[p-keys]#Attribute(s) to use as key#:: +Select the attribute (or attributes) to identify the Neo4j relationships by. +The attribute name must match the property name in Neo4j. + +==== diff --git a/web/app/help/operations/export-graph-to-neo4j.asciidoc b/web/app/help/operations/export-graph-to-neo4j.asciidoc new file mode 100644 index 0000000000..da832e70d3 --- /dev/null +++ b/web/app/help/operations/export-graph-to-neo4j.asciidoc @@ -0,0 +1,59 @@ +### Export graph to Neo4j + +Exports a graph from LynxKite to https://neo4j.com/[Neo4j]. +The whole graph will be copied to Neo4j with all attributes. +No existing data is modified in Neo4j. + +A `!LynxKite export timestamp` property is added to each new +node and relationship in Neo4j. This helps clean up the export if needed. + +The Cypher query to export nodes is, depending on whether an attribute specifies the node labels: + + UNWIND $events AS event + // Without node labels: + CREATE (n) + SET n += event + // With node labels taken from the "label" attribute: + CALL apoc.create.node(split(event.`label`, ','), event) YIELD node + RETURN 1 + +The Cypher query to export relationships is, depending on whether an attribute specifies the +relationship types: + + UNWIND $events AS event + MATCH (src {`!LynxKite ID`: event.`!Source LynxKite ID`}), (dst {`!LynxKite ID`: event.`Destination LynxKite ID`}) + // Without relationship types: + CREATE (src)-[r:EDGE]->(dst) + SET r += event + // With relationship types taken from the "type" attribute: + CALL apoc.create.relationship(src, event.`type`, event, dst) YIELD rel + RETURN 1 + +==== + +[p-url]#Neo4j connection#:: +The Neo4j connection string of the form `bolt://localhost:7687`. + +[p-username]#Neo4j username#:: +Username for the connection. + +[p-password]#Neo4j password#:: +Password for the connection. It will be saved in the workspace and visible to anyone with +access to the workspace. + +[p-version]#Export repetition ID#:: +LynxKite only re-computes outputs if parameters or inputs have changed. +This is true for exports too. If you want to repeat a previous export, you can increase this +export repetition ID parameter. + +[p-node_labels]#Node labels#:: +A string vertex attribute that is a comma-separated list of labels to apply to the newly +created nodes. Optional. You must have https://neo4j.com/developer/neo4j-apoc/[Neo4j APOC] +installed on the Neo4j instance to use this. + +[p-relationship_type]#Attribute with relationship type#:: +A string edge attribute that specifies the relationship type for each newly created relationship. +Optional. You must have https://neo4j.com/developer/neo4j-apoc/[Neo4j APOC] +installed on the Neo4j instance to use this. + +==== diff --git a/web/app/help/operations/export-vertex-attributes-to-neo4j.asciidoc b/web/app/help/operations/export-vertex-attributes-to-neo4j.asciidoc new file mode 100644 index 0000000000..a3c4424037 --- /dev/null +++ b/web/app/help/operations/export-vertex-attributes-to-neo4j.asciidoc @@ -0,0 +1,46 @@ +### Export vertex attributes to Neo4j + +Exports vertex attributes from a graph in LynxKite to a +corresponding graph in https://neo4j.com/[Neo4j]. + +The nodes in Neo4j are identified by a key property (or properties). +You must have a corresponding vertex attribute in LynxKite by the same name. +This will be used to find the right nodes to update in Neo4j. + +The properties of the Neo4j nodes will be updated with the exported vertex attributes using +a Cypher query like this: + + UNWIND $events as event + MATCH (n:Label1:Label2 {`key`: event.`key`}) + SET n += event + +In the event of duplicate keys on either end this will update the properties of all the matching +Neo4j nodes with the values from the last matching LynxKite vertex. + +==== + +[p-url]#Neo4j connection#:: +The Neo4j connection string of the form `bolt://localhost:7687`. + +[p-username]#Neo4j username#:: +Username for the connection. + +[p-password]#Neo4j password#:: +Password for the connection. It will be saved in the workspace and visible to anyone with +access to the workspace. + +[p-version]#Export repetition ID#:: +LynxKite only re-computes outputs if parameters or inputs have changed. +This is true for exports too. If you want to repeat a previous export, you can increase this +export repetition ID parameter. + +[p-labels]#Node labels#:: +Makes it possible to restrict the export to one label (or combination of labels) in Neo4j. +This is useful to make sure no other node type is accidentally affected. +The format is as in Cypher: `:Label1:Label2`. Leave empty to allow updating any node. + +[p-keys]#Attribute(s) to use as key#:: +Select the attribute (or attributes) to identify the Neo4j nodes by. +The attribute name must match the property name in Neo4j. + +==== diff --git a/web/app/help/operations/index.asciidoc b/web/app/help/operations/index.asciidoc index 53cff0e58a..0cca51bee6 100644 --- a/web/app/help/operations/index.asciidoc +++ b/web/app/help/operations/index.asciidoc @@ -283,6 +283,10 @@ include::discard-vertex-attributes.asciidoc[] include::embed-vertices.asciidoc[] +include::export-edge-attributes-to-neo4j.asciidoc[] + +include::export-graph-to-neo4j.asciidoc[] + include::export-to-avro.asciidoc[] include::export-to-csv.asciidoc[] @@ -299,6 +303,8 @@ include::export-to-orc.asciidoc[] include::export-to-parquet.asciidoc[] +include::export-vertex-attributes-to-neo4j.asciidoc[] + include::expose-internal-edge-id.asciidoc[] include::expose-internal-vertex-id.asciidoc[]