Skip to content
New issue

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

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

Already on GitHub? Sign in to your account

Neo4j parametrized from existing SCollection #4719

Merged
merged 5 commits into from
Mar 1, 2023
Merged
Show file tree
Hide file tree
Changes from 1 commit
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
43 changes: 35 additions & 8 deletions scio-neo4j/src/it/scala/com/spotify/scio/neo4j/Neo4jIOIT.scala
Original file line number Diff line number Diff line change
Expand Up @@ -68,29 +68,56 @@ class Neo4jIOIT extends PipelineSpec with Eventually with ForAllTestContainer {
options.setRunner(classOf[DirectRunner])

val martin = Person("Martin Sheen")
val expectedRoles = Seq(
val morgan = Person("Morgan Freeman")
val michael = Person("Michael Douglas")

val americanPresident = Movie("American President", 1995)

val queryMovieYears = Seq(1994, 0, 1995)
val expectedRolesMartin = Seq(
Role(martin, Movie("Wall Street", 1987), "Carl Fox"),
Role(martin, Movie("American President", 1995), "A.J. MacInerney")
)
val expectedRolesMovieYears = Seq(
Role(martin, americanPresident, "A.J. MacInerney"),
Role(michael, americanPresident, "President Andrew Shepherd"),
Role(morgan, Movie("The Shawshank Redemption", 1994), "Ellis Boyd 'Red' Redding")
)

val queryRoles =
s"""MATCH (p)-[r: ACTED_IN]->(m)
|WHERE p.name='${martin.name}'
|RETURN p as person, m as movie, r.role as role
|""".stripMargin

val queryMovieYear =
s"""MATCH (p)-[r: ACTED_IN]->(m)
|WHERE m.year = $$movieYear
|RETURN p as person, m as movie, r.role as role
|""".stripMargin

val neo4jOptions = Neo4jOptions(
Neo4jConnectionOptions(container.boltUrl, container.username, container.password)
)

val queryRoles = s"""MATCH (p)-[r: ACTED_IN]->(m)
|WHERE p.name='${martin.name}'
|RETURN p as person, m as movie, r.role as role
|""".stripMargin

implicit val rowMapper = (record: Record) => {
val p = Person(record.get("p").get("name").asString())
val m = Movie(record.get("m").get("name").asString(), record.get("m").get("year").asInt())
Role(p, m, record.get("r").get("role").asString())
}

runWithRealContext(options) { sc =>
val result = sc.neo4jCypher[Role](neo4jOptions, queryRoles)
result should containInAnyOrder(expectedRoles)
val resultQueryRoles = sc.neo4jCypher[Role](neo4jOptions, queryRoles)
resultQueryRoles should containInAnyOrder(expectedRolesMartin)

val resultQueryMovieYear = sc
.parallelize(queryMovieYears)
.neo4jCypherWithParams[Role](
neo4jOptions,
queryMovieYear,
(my: Int) => Map("movieYear" -> java.lang.Integer.valueOf(my))
)
sumitsu marked this conversation as resolved.
Show resolved Hide resolved
resultQueryMovieYear should containInAnyOrder(expectedRolesMovieYears)
}
}

Expand Down
17 changes: 4 additions & 13 deletions scio-neo4j/src/main/scala/com/spotify/scio/neo4j/Neo4jIO.scala
Original file line number Diff line number Diff line change
Expand Up @@ -20,10 +20,11 @@ package com.spotify.scio.neo4j
import com.spotify.scio.ScioContext
import com.spotify.scio.coders.{Coder, CoderMaterializer}
import com.spotify.scio.io._
import com.spotify.scio.neo4j.ops.Neo4jCommon.dataSourceConfiguration
import com.spotify.scio.neo4j.ops.Neo4jCommonImplicits
import com.spotify.scio.values.SCollection
import magnolify.neo4j.ValueType
import org.apache.beam.sdk.io.{neo4j => beam}
import org.neo4j.driver.{Record, Value, Values}

import scala.util.matching.Regex

Expand All @@ -34,9 +35,6 @@ object Neo4jIO {
}
final case class WriteParam(batchSize: Long = WriteParam.BeamDefaultBatchSize)

implicit private def recordConverter(record: Record): Value =
Values.value(record.asMap(identity[Value]))

private[neo4j] val UnwindParameterRegex: Regex = """UNWIND \$(\w+)""".r.unanchored

private[neo4j] def neo4jIoId(opts: Neo4jOptions, cypher: String): String =
Expand All @@ -45,21 +43,14 @@ object Neo4jIO {
private[neo4j] def neo4jIoId(opts: Neo4jConnectionOptions, cypher: String): String =
s"${opts.username}:${opts.password}@${opts.url}:$cypher"

private[neo4j] def dataSourceConfiguration(
sumitsu marked this conversation as resolved.
Show resolved Hide resolved
connectionOptions: Neo4jConnectionOptions
): beam.Neo4jIO.DriverConfiguration =
beam.Neo4jIO.DriverConfiguration.create(
connectionOptions.url,
connectionOptions.username,
connectionOptions.password
)
}

final case class Neo4jIO[T](neo4jOptions: Neo4jOptions, cypher: String)(implicit
neo4jType: ValueType[T],
coder: Coder[T]
) extends ScioIO[T] {

import Neo4jCommonImplicits._
import Neo4jIO._

override type ReadP = Unit
Expand Down Expand Up @@ -97,7 +88,7 @@ final case class Neo4jIO[T](neo4jOptions: Neo4jOptions, cypher: String)(implicit
beam.Neo4jIO
.writeUnwind()
.withUnwindMapName(unwindMapName)
.withDriverConfiguration(Neo4jIO.dataSourceConfiguration(neo4jOptions.connectionOptions))
.withDriverConfiguration(dataSourceConfiguration(neo4jOptions.connectionOptions))
.withSessionConfig(neo4jOptions.sessionConfig)
.withTransactionConfig(neo4jOptions.transactionConfig)
.withBatchSize(params.batchSize)
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,36 @@
/*
* Copyright 2023 Spotify AB.
*
* Licensed under the Apache License, Version 2.0 (the "License");
* you may not use this file except in compliance with the License.
* You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing,
* software distributed under the License is distributed on an
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
* KIND, either express or implied. See the License for the
* specific language governing permissions and limitations
* under the License.
*/

package com.spotify.scio.neo4j.ops

import com.spotify.scio.neo4j.Neo4jConnectionOptions
import org.apache.beam.sdk.io.{neo4j => beam}

/** Shared tooling for Scio-Neo4j components */
object Neo4jCommon {

def dataSourceConfiguration(
connectionOptions: Neo4jConnectionOptions
): beam.Neo4jIO.DriverConfiguration = {
beam.Neo4jIO.DriverConfiguration.create(
connectionOptions.url,
connectionOptions.username,
connectionOptions.password
)
}

}
Original file line number Diff line number Diff line change
@@ -0,0 +1,28 @@
/*
* Copyright 2023 Spotify AB.
*
* Licensed under the Apache License, Version 2.0 (the "License");
* you may not use this file except in compliance with the License.
* You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing,
* software distributed under the License is distributed on an
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
* KIND, either express or implied. See the License for the
* specific language governing permissions and limitations
* under the License.
*/

package com.spotify.scio.neo4j.ops

import org.neo4j.driver.{Record, Value, Values}

/** Implicit conversions for Scio-Neo4j components */
object Neo4jCommonImplicits {

implicit def recordConverter(record: Record): Value =
Values.value(record.asMap(identity[Value]))

}
Original file line number Diff line number Diff line change
@@ -0,0 +1,149 @@
/*
* Copyright 2023 Spotify AB.
*
* Licensed under the Apache License, Version 2.0 (the "License");
* you may not use this file except in compliance with the License.
* You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing,
* software distributed under the License is distributed on an
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
* KIND, either express or implied. See the License for the
* specific language governing permissions and limitations
* under the License.
*/

package com.spotify.scio.neo4j.ops

import com.spotify.scio.coders.{Coder, CoderMaterializer}
import com.spotify.scio.neo4j.Neo4jOptions
import com.spotify.scio.neo4j.ops.Neo4jCommon.dataSourceConfiguration
import com.spotify.scio.values.SCollection
import magnolify.neo4j.ValueType
import org.apache.beam.sdk.io.{neo4j => beam}
import org.apache.beam.sdk.transforms.SerializableFunction

import java.util.{Map => JMap}
import scala.jdk.CollectionConverters._

/**
* Operations for transforming an existing [[SCollection]] via a Neo4j query (read)
* @param self
* existing [[SCollection]] (prior to Neo4j read)
* @tparam X
* type parameter for existing [[SCollection]]
*/
class Neo4jSCollectionReadOps[X](self: SCollection[X]) {
sumitsu marked this conversation as resolved.
Show resolved Hide resolved

/**
* Execute parallel instances of the provided Cypher query to the specified Neo4j database; one
* instance of the query will be executed for each element in this [[SCollection]]. Results from
* each query invocation will be added to the resulting [[SCollection]] as if by a `flatMap`
* transformation (where the Neo4j-query-execution returns an `Iterable`).
*
* This operation parameterizes each query invocation by applying a supplied function to each
* [[SCollection]] element before executing the Cypher query. The function must produce a [[Map]]
* of [[String]] to [[AnyRef]], where the keys correspond to named parameters in the provided
* Cypher query and the corresponding values correspond to the intended value of that parameter
* for a given query invocation. Named parameters must consist of letters and numbers, prepended
* with a `$`, as described in the Neo4j
* [[https://neo4j.com/docs/cypher-manual/current/syntax/parameters Cypher Manual (Syntax / Parameters)]].
*
* @see
* ''Reading from Neo4j'' in the
* [[https://beam.apache.org/releases/javadoc/current/org/apache/beam/sdk/io/neo4j/Neo4jIO.html Beam `Neo4jIO` documentation]]
* @param neo4jConf
* [[Neo4jOptions]] indicating the Neo4j instance on which to run the query
* @param cypher
* [[String]] parameterized Cypher query
* @param elementToParamFunction
* function (`X => Map[String, AnyRef]`) which converts an element of the [[SCollection]] to a
* [[Map]] of parameter values for the given Cypher query
* @param neo4jType
* (implicit) [[ValueType]] converting Neo4j results to the expected [[SCollection]] output type
* @param coder
* (implicit) [[Coder]] for serialization of the result [[SCollection]] type
* @tparam Y
* result [[SCollection]] type
* @return
* [[SCollection]] containing the union of query results from a parameterized query invocation
* for each original [[SCollection]] element
*/
def neo4jCypherWithParams[Y](
neo4jConf: Neo4jOptions,
cypher: String,
elementToParamFunction: X => Map[String, AnyRef]
sumitsu marked this conversation as resolved.
Show resolved Hide resolved
)(implicit
neo4jType: ValueType[Y],
coder: Coder[Y]
): SCollection[Y] = {
Neo4jSCollectionReadOps.neo4jCypherWithParamsImpl(
self,
neo4jConf,
cypher,
elementToParamFunction
)
}

}

/**
* Implementations for operations for transforming an existing [[SCollection]] via a Neo4j query;
* typically used via implicitly-defined [[SCollection]] syntax.
* @see
* [[com.spotify.scio.neo4j.syntax.SCollectionSyntax]]
*/
object Neo4jSCollectionReadOps {

import Neo4jCommonImplicits._

/**
* Convert a provided Scala function to a Beam [[SerializableFunction]] for use in
* [[https://beam.apache.org/releases/javadoc/current/org/apache/beam/sdk/io/neo4j/Neo4jIO.ReadAll.html#withParametersFunction-org.apache.beam.sdk.transforms.SerializableFunction- `Neo4jIO.ReadAll`]]
* @param elementToParamFunction
* `X => Map[String, AnyRef]` Scala function defining how elements of an existing
* [[SCollection]] (`X`) should map to parameters ([[String]] keys in the output [[Map]]) in a
* Cypher query
* @tparam X
* type parameter for input [[SCollection]]
* @return
* [[SerializableFunction]] equivalent of input Scala function
*/
private def beamParamFunction[X](
elementToParamFunction: X => Map[String, AnyRef]
): SerializableFunction[X, JMap[String, AnyRef]] = {
new SerializableFunction[X, JMap[String, AnyRef]] {
override def apply(input: X): JMap[String, AnyRef] = {
val queryParamStringToArgValue: Map[String, AnyRef] = elementToParamFunction.apply(input)
queryParamStringToArgValue.asJava
}
}
}

/** @see [[com.spotify.scio.neo4j.ops.Neo4jSCollectionReadOps.neo4jCypherWithParams]] */
def neo4jCypherWithParamsImpl[X, Y](
sColl: SCollection[X],
neo4jConf: Neo4jOptions,
cypher: String,
elementToParamFunction: X => Map[String, AnyRef]
)(implicit
neo4jType: ValueType[Y],
coder: Coder[Y]
): SCollection[Y] = {
sColl
.applyTransform(
beam.Neo4jIO
.readAll[X, Y]()
.withDriverConfiguration(dataSourceConfiguration(neo4jConf.connectionOptions))
.withSessionConfig(neo4jConf.sessionConfig)
.withTransactionConfig(neo4jConf.transactionConfig)
.withCypher(cypher)
.withParametersFunction(beamParamFunction(elementToParamFunction))
.withRowMapper(neo4jType.from(_))
.withCoder(CoderMaterializer.beam(sColl.context, coder))
)
}

}
Original file line number Diff line number Diff line change
@@ -0,0 +1,51 @@
/*
* Copyright 2023 Spotify AB.
*
* Licensed under the Apache License, Version 2.0 (the "License");
* you may not use this file except in compliance with the License.
* You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing,
* software distributed under the License is distributed on an
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
* KIND, either express or implied. See the License for the
* specific language governing permissions and limitations
* under the License.
*/

package com.spotify.scio.neo4j.ops

import com.spotify.scio.coders.Coder
import com.spotify.scio.io.ClosedTap
import com.spotify.scio.neo4j.Neo4jIO.WriteParam
import com.spotify.scio.neo4j.{Neo4jIO, Neo4jOptions}
import com.spotify.scio.values.SCollection
import magnolify.neo4j.ValueType

/**
* Enhanced version of [[com.spotify.scio.values.SCollection SCollection]] with Neo4J write methods.
*/
class Neo4jSCollectionWriteOps[T](private val self: SCollection[T]) extends AnyVal {

/**
* Save this SCollection as a Neo4J database.
*
* @param neo4jOptions
* options for configuring a Neo4J driver
* @param unwindCypher
* Neo4J cypher query representing an
* [[https://neo4j.com/docs/cypher-manual/current/clauses/unwind/#unwind-creating-nodes-from-a-list-parameter UNWIND parameter]]
* cypher statement
* @param batchSize
* batch size when executing the unwind cypher query. Default batch size of 5000
*/
def saveAsNeo4j(
neo4jOptions: Neo4jOptions,
unwindCypher: String,
batchSize: Long = WriteParam.BeamDefaultBatchSize
)(implicit neo4jType: ValueType[T], coder: Coder[T]): ClosedTap[Nothing] =
self.write(Neo4jIO[T](neo4jOptions, unwindCypher))(WriteParam(batchSize))

}
Loading