From cfcd55dfe06bee972d99b574a6bd3b3ea9c0edf0 Mon Sep 17 00:00:00 2001 From: Patrick McGee Date: Tue, 22 Aug 2023 16:52:57 -0400 Subject: [PATCH 01/28] wip --- build.sbt | 2 + .../scio/extra/voyager/VoyagerUri.scala | 47 +++++++++++++ .../spotify/scio/extra/voyager/package.scala | 66 +++++++++++++++++++ 3 files changed, 115 insertions(+) create mode 100644 scio-extra/src/main/scala/com/spotify/scio/extra/voyager/VoyagerUri.scala create mode 100644 scio-extra/src/main/scala/com/spotify/scio/extra/voyager/package.scala diff --git a/build.sbt b/build.sbt index b27a2e7ff0..5ae6d70e66 100644 --- a/build.sbt +++ b/build.sbt @@ -138,6 +138,7 @@ val shapelessVersion = "2.3.10" val sparkeyVersion = "3.2.5" val tensorFlowVersion = "0.4.2" val testContainersVersion = "0.41.0" +val voyagerVersion = "1.2.0" val zoltarVersion = "0.6.0" // dependent versions val scalatestplusVersion = s"$scalatestVersion.0" @@ -914,6 +915,7 @@ lazy val `scio-extra`: Project = project "org.scalanlp" %% "breeze" % breezeVersion, "org.slf4j" % "slf4j-api" % slf4jVersion, "org.typelevel" %% "algebra" % algebraVersion, + "com.spotify" % "voyager" % voyagerVersion, // test "com.github.ben-manes.caffeine" % "caffeine" % caffeineVersion % "test,it", "org.scalacheck" %% "scalacheck" % scalacheckVersion % "test,it", diff --git a/scio-extra/src/main/scala/com/spotify/scio/extra/voyager/VoyagerUri.scala b/scio-extra/src/main/scala/com/spotify/scio/extra/voyager/VoyagerUri.scala new file mode 100644 index 0000000000..0ac6773d6d --- /dev/null +++ b/scio-extra/src/main/scala/com/spotify/scio/extra/voyager/VoyagerUri.scala @@ -0,0 +1,47 @@ +package com.spotify.scio.extra.voyager + +import com.spotify.scio.coders.Coder +import org.apache.flink.configuration.PipelineOptions + +import java.io.File + +trait VoyagerUri extends Serializable { + val path: String + private[voyager] def getReader(distanceMeasure: VoyagerDistanceMeasure, dim: Int): VoyagerReader + private[voyager] def buildAndValidate(voyagerWriter: VoyagerWriter): Unit + private[voyager] def exists: Boolean +} + +private[voyager] object VoyagerUri { + def apply(path: String, opts: PipelineOptions): VoyagerUri = ??? + implicit val voyagerUriCoder: Coder[VoyagerUri] = Coder.kryo[VoyagerUri] +} + +private class LocalVoyagerUri(val path: String) extends VoyagerUri { + override private[voyager] def getReader( + distanceMeasure: VoyagerDistanceMeasure, + dim: Int + ): VoyagerReader = ??? + + override private[voyager] def buildAndValidate(voyagerWriter: VoyagerWriter): Unit = ??? + + override private[voyager] def exists: Boolean = new File(path).exists() +} + +private class RemoteVoyagerUri(val path: String, option: PipelineOptions) extends VoyagerUri { + override private[voyager] def getReader( + distanceMeasure: VoyagerDistanceMeasure, + dim: Int + ): VoyagerReader = ??? + + override private[voyager] def buildAndValidate(voyagerWriter: VoyagerWriter): Unit = ??? + + override private[voyager] def exists: Boolean = ??? +} + +private[voyager] class VoyagerWriter( + distanceMeasure: VoyagerDistanceMeasure, + dim: Int, + ef: Int, + m: Int +) {} diff --git a/scio-extra/src/main/scala/com/spotify/scio/extra/voyager/package.scala b/scio-extra/src/main/scala/com/spotify/scio/extra/voyager/package.scala new file mode 100644 index 0000000000..d14f1b97c6 --- /dev/null +++ b/scio-extra/src/main/scala/com/spotify/scio/extra/voyager/package.scala @@ -0,0 +1,66 @@ +package com.spotify.scio.extra + +import com.spotify.scio.annotations.experimental +import com.spotify.scio.values.DistCache +import com.spotify.voyager.jni.Index.{SpaceType, StorageDataType} +import com.spotify.voyager.jni.Index +import org.apache.beam.sdk.values.PCollectionView + +package object voyager { + sealed abstract class VoyagerDistanceMeasure + case object Euclidean extends VoyagerDistanceMeasure + case object Cosine extends VoyagerDistanceMeasure + case object Dot extends VoyagerDistanceMeasure + + sealed abstract class VoyagerStorageType + case object Float8 extends VoyagerStorageType + case object Float32 extends VoyagerStorageType + case object E4M3 extends VoyagerStorageType + + class VoyagerReader private[voyager] ( + path: String, + distanceMeasure: VoyagerDistanceMeasure, + storageType: VoyagerStorageType, + dim: Int + ) { + require(dim > 0, "Vector dimension should be > 0") + private val index: Index = { + val spaceType = distanceMeasure match { + case Euclidean => SpaceType.Euclidean + case Cosine => SpaceType.Cosine + case Dot => SpaceType.InnerProduct + } + + val storageDataType = storageType match { + case Float8 => StorageDataType.Float8 + case Float32 => StorageDataType.Float32 + case E4M3 => StorageDataType.E4M3 + } + Index.load(path, spaceType, dim, storageDataType) + } + + def getNearest(v: Array[Float], maxNumResults: Int) = index.query(v, maxNumResults) + + def getItemVector(i: Int) = index.getVector(i) + } + + @experimental + def asVoyager( + path: String, + m: Int, + ef: Int, + voyagerDistanceMeasure: VoyagerDistanceMeasure + ): String = + "" + + def asVoyagerDistCache(): DistCache[VoyagerReader] = ??? + +// private class VoyagerDistCache( +// val view: PCollectionView[VoyagerUri], +// distanceMeasure: VoyagerDistanceMeasure, +// dim: Int +// ) extends DistCache[VoyagerReader] { +// override def apply(): VoyagerReader = ??? +// } + +} From d3f21497e1bcabc7503a830f4f5fdffffa521505 Mon Sep 17 00:00:00 2001 From: Patrick McGee Date: Thu, 24 Aug 2023 11:14:20 -0400 Subject: [PATCH 02/28] WIP --- .../scio/extra/voyager/VoyagerUri.scala | 93 +++++++++++++++++-- .../spotify/scio/extra/voyager/package.scala | 59 ++++++++---- 2 files changed, 128 insertions(+), 24 deletions(-) diff --git a/scio-extra/src/main/scala/com/spotify/scio/extra/voyager/VoyagerUri.scala b/scio-extra/src/main/scala/com/spotify/scio/extra/voyager/VoyagerUri.scala index 0ac6773d6d..b94d256b32 100644 --- a/scio-extra/src/main/scala/com/spotify/scio/extra/voyager/VoyagerUri.scala +++ b/scio-extra/src/main/scala/com/spotify/scio/extra/voyager/VoyagerUri.scala @@ -1,19 +1,33 @@ package com.spotify.scio.extra.voyager import com.spotify.scio.coders.Coder -import org.apache.flink.configuration.PipelineOptions +import com.spotify.scio.util.ScioUtil +import java.nio.charset.StandardCharsets +import com.spotify.voyager.jni.{Index, StringIndex} +import com.spotify.voyager.jni.Index.{SpaceType, StorageDataType} +import org.apache.beam.sdk.options.PipelineOptions +import scala.jdk.CollectionConverters + +import scala.collection.mutable import java.io.File +import java.net.URI +import java.nio.file.{Files, Paths} trait VoyagerUri extends Serializable { val path: String private[voyager] def getReader(distanceMeasure: VoyagerDistanceMeasure, dim: Int): VoyagerReader - private[voyager] def buildAndValidate(voyagerWriter: VoyagerWriter): Unit + private[voyager] def saveAndClose(voyagerWriter: VoyagerWriter): Unit private[voyager] def exists: Boolean } private[voyager] object VoyagerUri { - def apply(path: String, opts: PipelineOptions): VoyagerUri = ??? + def apply(path: String, opts: PipelineOptions): VoyagerUri = + if (ScioUtil.isLocalUri(new URI(path))) { + new LocalVoyagerUri(path) + } else { + new RemoteVoyagerUri(path, opts) + } implicit val voyagerUriCoder: Coder[VoyagerUri] = Coder.kryo[VoyagerUri] } @@ -23,7 +37,7 @@ private class LocalVoyagerUri(val path: String) extends VoyagerUri { dim: Int ): VoyagerReader = ??? - override private[voyager] def buildAndValidate(voyagerWriter: VoyagerWriter): Unit = ??? + override private[voyager] def saveAndClose(voyagerWriter: VoyagerWriter): Unit = ??? override private[voyager] def exists: Boolean = new File(path).exists() } @@ -34,14 +48,77 @@ private class RemoteVoyagerUri(val path: String, option: PipelineOptions) extend dim: Int ): VoyagerReader = ??? - override private[voyager] def buildAndValidate(voyagerWriter: VoyagerWriter): Unit = ??? + override private[voyager] def saveAndClose(voyagerWriter: VoyagerWriter): Unit = ??? override private[voyager] def exists: Boolean = ??? } private[voyager] class VoyagerWriter( distanceMeasure: VoyagerDistanceMeasure, + storageType: VoyagerStorageType, dim: Int, - ef: Int, - m: Int -) {} + ef: Long = 200L, + m: Long = 16L +) { + + // Chunk size experiments - , + // 4096, 6062: 2022-11-16 14:07:07.358 -> 2022-11-16 16:50:59.109 = 2hr 50min. 1.68s per chunk + // 32786, 758: 2022-11-16 15:37:11.374 -> 2022-11-16 16:50:29.396 = 1hr 13min. 5.77s per chunk + // 131072, 190: 2022-11-17 13:38:08.421 -> 2022-11-17 15:42:39.929 = 2hr 6min. 39.79s per chunk + private val chunkSize = 32786 // 2^15 + private val randomSeed = 1L + private[this] val namesOutput = mutable.ListBuffer.empty[String] + + private[this] val index: Index = { + val spaceType = distanceMeasure match { + case Euclidean => SpaceType.Euclidean + case Cosine => SpaceType.Cosine + case Dot => SpaceType.InnerProduct + } + + val storageDataType = storageType match { + case Float8 => StorageDataType.Float8 + case Float32 => StorageDataType.Float32 + case E4M3 => StorageDataType.E4M3 + } + new Index(spaceType, dim, m, ef, randomSeed, chunkSize, storageDataType) + } + + def write(vectors: Iterable[(String, Array[Float])]): Unit = { + var batchNum = 1 + + val nameVectorIndexIterator = vectors.iterator.zipWithIndex + .map { case ((name, vector), idx) => + (name, vector, idx.longValue()) + } + + while (nameVectorIndexIterator.hasNext) { + val (nameArray, vectorArray, indexArray) = nameVectorIndexIterator + .take(chunkSize) + .toArray + .unzip3 + + index.addItems(vectorArray, indexArray, -1) + + batchNum += 1 + namesOutput ++= nameArray + } + + () + } + + def save(indexFileName: String, namesFileName: String): Unit = { + index.saveIndex(indexFileName) + Files.write( + Paths.get(namesFileName), + namesOutput.mkString("[\"", "\",\"", "\"]").getBytes(StandardCharsets.UTF_8) + ) + () + } + + def close(): Unit = { + index.close() + () + } + +} diff --git a/scio-extra/src/main/scala/com/spotify/scio/extra/voyager/package.scala b/scio-extra/src/main/scala/com/spotify/scio/extra/voyager/package.scala index d14f1b97c6..ce73eb6860 100644 --- a/scio-extra/src/main/scala/com/spotify/scio/extra/voyager/package.scala +++ b/scio-extra/src/main/scala/com/spotify/scio/extra/voyager/package.scala @@ -1,10 +1,9 @@ package com.spotify.scio.extra import com.spotify.scio.annotations.experimental -import com.spotify.scio.values.DistCache +import com.spotify.scio.values.{DistCache, SCollection} import com.spotify.voyager.jni.Index.{SpaceType, StorageDataType} -import com.spotify.voyager.jni.Index -import org.apache.beam.sdk.values.PCollectionView +import com.spotify.voyager.jni.{Index, StringIndex} package object voyager { sealed abstract class VoyagerDistanceMeasure @@ -17,6 +16,8 @@ package object voyager { case object Float32 extends VoyagerStorageType case object E4M3 extends VoyagerStorageType + case class VoyagerResult(label: String, distance: Float) + class VoyagerReader private[voyager] ( path: String, distanceMeasure: VoyagerDistanceMeasure, @@ -24,7 +25,11 @@ package object voyager { dim: Int ) { require(dim > 0, "Vector dimension should be > 0") - private val index: Index = { + + private val indexPath: String = path + "/index.hnsw" + private val namesPath: String = path + "/names.json" + + private val index: StringIndex = { val spaceType = distanceMeasure match { case Euclidean => SpaceType.Euclidean case Cosine => SpaceType.Cosine @@ -36,22 +41,44 @@ package object voyager { case Float32 => StorageDataType.Float32 case E4M3 => StorageDataType.E4M3 } - Index.load(path, spaceType, dim, storageDataType) + StringIndex.load(indexPath, namesPath, spaceType, dim, storageDataType) } + // figure out index path + names path - def getNearest(v: Array[Float], maxNumResults: Int) = index.query(v, maxNumResults) - - def getItemVector(i: Int) = index.getVector(i) +// def getNearest(v: Array[Float], maxNumResults: Int) = index.query(v, maxNumResults) +// +// def getItemVector(i: Int) = index.getVector(i) } - @experimental - def asVoyager( - path: String, - m: Int, - ef: Int, - voyagerDistanceMeasure: VoyagerDistanceMeasure - ): String = - "" + implicit class VoyagerPairSCollection( + @transient private val self: SCollection[(String, Array[Float])] + ) extends AnyVal { + + @experimental + def asVoyager( + path: String, + voyagerDistanceMeasure: VoyagerDistanceMeasure, + voyagerStorageType: VoyagerStorageType, + dim: Int, + ef: Long, + m: Long + ): SCollection[VoyagerUri] = { + val uri: VoyagerUri = VoyagerUri(path, self.context.options) + require(!uri.exists, s"Voyager URI ${uri.path} already exists") + self.transform { in => + { + in.groupBy(_ => ()) + .map { case (_, xs) => + val voyagerWriter: VoyagerWriter = + new VoyagerWriter(voyagerDistanceMeasure, voyagerStorageType, dim, ef, m) + voyagerWriter.write(xs) + uri + } + } + } + } + + } def asVoyagerDistCache(): DistCache[VoyagerReader] = ??? From 8fef885247bb9b504f91f9d9c3514f27dc222bb9 Mon Sep 17 00:00:00 2001 From: Patrick McGee Date: Thu, 24 Aug 2023 13:43:38 -0400 Subject: [PATCH 03/28] next steps --- .../scio/extra/voyager/VoyagerUri.scala | 28 ++++++++++++++----- 1 file changed, 21 insertions(+), 7 deletions(-) diff --git a/scio-extra/src/main/scala/com/spotify/scio/extra/voyager/VoyagerUri.scala b/scio-extra/src/main/scala/com/spotify/scio/extra/voyager/VoyagerUri.scala index b94d256b32..e2acc67b11 100644 --- a/scio-extra/src/main/scala/com/spotify/scio/extra/voyager/VoyagerUri.scala +++ b/scio-extra/src/main/scala/com/spotify/scio/extra/voyager/VoyagerUri.scala @@ -1,14 +1,14 @@ package com.spotify.scio.extra.voyager import com.spotify.scio.coders.Coder -import com.spotify.scio.util.ScioUtil +import com.spotify.scio.util.{RemoteFileUtil, ScioUtil} + import java.nio.charset.StandardCharsets import com.spotify.voyager.jni.{Index, StringIndex} import com.spotify.voyager.jni.Index.{SpaceType, StorageDataType} import org.apache.beam.sdk.options.PipelineOptions import scala.jdk.CollectionConverters - import scala.collection.mutable import java.io.File import java.net.URI @@ -28,6 +28,7 @@ private[voyager] object VoyagerUri { } else { new RemoteVoyagerUri(path, opts) } + def files: Seq[String] = Seq("index.hnsw", "names.json") implicit val voyagerUriCoder: Coder[VoyagerUri] = Coder.kryo[VoyagerUri] } @@ -37,20 +38,33 @@ private class LocalVoyagerUri(val path: String) extends VoyagerUri { dim: Int ): VoyagerReader = ??? - override private[voyager] def saveAndClose(voyagerWriter: VoyagerWriter): Unit = ??? + override private[voyager] def saveAndClose(w: VoyagerWriter): Unit = { + val indexPath = path + "/index.hnsw" + val namesPath = path + "/names.json" + w.save(indexPath, namesPath) + w.close() + } - override private[voyager] def exists: Boolean = new File(path).exists() + override private[voyager] def exists: Boolean = + VoyagerUri.files.map(f => new File(path + "/" + f)).exists(_.exists()) } -private class RemoteVoyagerUri(val path: String, option: PipelineOptions) extends VoyagerUri { +private class RemoteVoyagerUri(val path: String, options: PipelineOptions) extends VoyagerUri { + private[this] val rfu: RemoteFileUtil = RemoteFileUtil.create(options) override private[voyager] def getReader( distanceMeasure: VoyagerDistanceMeasure, dim: Int ): VoyagerReader = ??? - override private[voyager] def saveAndClose(voyagerWriter: VoyagerWriter): Unit = ??? + override private[voyager] def saveAndClose(w: VoyagerWriter): Unit = { + val indexPath = path + "/index.hnsw" + val namesPath = path + "/names.json" + w.save(indexPath, namesPath) + w.close() + } - override private[voyager] def exists: Boolean = ??? + override private[voyager] def exists: Boolean = + VoyagerUri.files.exists(f => rfu.remoteExists(new URI(path + "/" + f))) } private[voyager] class VoyagerWriter( From d3257b0483dd31ae3127e90b8701129636482a92 Mon Sep 17 00:00:00 2001 From: Patrick McGee Date: Mon, 28 Aug 2023 10:01:17 -0400 Subject: [PATCH 04/28] start fleshing out reader as well --- .../scio/extra/voyager/VoyagerUri.scala | 16 ++++-- .../spotify/scio/extra/voyager/package.scala | 7 ++- .../scio/extra/voyager/VoyagerTest.scala | 52 +++++++++++++++++++ 3 files changed, 68 insertions(+), 7 deletions(-) create mode 100644 scio-extra/src/test/scala/com/spotify/scio/extra/voyager/VoyagerTest.scala diff --git a/scio-extra/src/main/scala/com/spotify/scio/extra/voyager/VoyagerUri.scala b/scio-extra/src/main/scala/com/spotify/scio/extra/voyager/VoyagerUri.scala index e2acc67b11..9146ad7486 100644 --- a/scio-extra/src/main/scala/com/spotify/scio/extra/voyager/VoyagerUri.scala +++ b/scio-extra/src/main/scala/com/spotify/scio/extra/voyager/VoyagerUri.scala @@ -16,7 +16,11 @@ import java.nio.file.{Files, Paths} trait VoyagerUri extends Serializable { val path: String - private[voyager] def getReader(distanceMeasure: VoyagerDistanceMeasure, dim: Int): VoyagerReader + private[voyager] def getReader( + distanceMeasure: VoyagerDistanceMeasure, + storageType: VoyagerStorageType, + dim: Int + ): VoyagerReader private[voyager] def saveAndClose(voyagerWriter: VoyagerWriter): Unit private[voyager] def exists: Boolean } @@ -35,8 +39,10 @@ private[voyager] object VoyagerUri { private class LocalVoyagerUri(val path: String) extends VoyagerUri { override private[voyager] def getReader( distanceMeasure: VoyagerDistanceMeasure, + storageType: VoyagerStorageType, dim: Int - ): VoyagerReader = ??? + ): VoyagerReader = + new VoyagerReader(path, distanceMeasure, storageType, dim) override private[voyager] def saveAndClose(w: VoyagerWriter): Unit = { val indexPath = path + "/index.hnsw" @@ -53,8 +59,12 @@ private class RemoteVoyagerUri(val path: String, options: PipelineOptions) exten private[this] val rfu: RemoteFileUtil = RemoteFileUtil.create(options) override private[voyager] def getReader( distanceMeasure: VoyagerDistanceMeasure, + storageType: VoyagerStorageType, dim: Int - ): VoyagerReader = ??? + ): VoyagerReader = { + val localPath = rfu.download(new URI(path)) + new VoyagerReader(localPath.toString, distanceMeasure, storageType, dim) + } override private[voyager] def saveAndClose(w: VoyagerWriter): Unit = { val indexPath = path + "/index.hnsw" diff --git a/scio-extra/src/main/scala/com/spotify/scio/extra/voyager/package.scala b/scio-extra/src/main/scala/com/spotify/scio/extra/voyager/package.scala index ce73eb6860..cb21b473a1 100644 --- a/scio-extra/src/main/scala/com/spotify/scio/extra/voyager/package.scala +++ b/scio-extra/src/main/scala/com/spotify/scio/extra/voyager/package.scala @@ -16,8 +16,6 @@ package object voyager { case object Float32 extends VoyagerStorageType case object E4M3 extends VoyagerStorageType - case class VoyagerResult(label: String, distance: Float) - class VoyagerReader private[voyager] ( path: String, distanceMeasure: VoyagerDistanceMeasure, @@ -60,8 +58,8 @@ package object voyager { voyagerDistanceMeasure: VoyagerDistanceMeasure, voyagerStorageType: VoyagerStorageType, dim: Int, - ef: Long, - m: Long + ef: Long = 200L, + m: Long = 16L ): SCollection[VoyagerUri] = { val uri: VoyagerUri = VoyagerUri(path, self.context.options) require(!uri.exists, s"Voyager URI ${uri.path} already exists") @@ -72,6 +70,7 @@ package object voyager { val voyagerWriter: VoyagerWriter = new VoyagerWriter(voyagerDistanceMeasure, voyagerStorageType, dim, ef, m) voyagerWriter.write(xs) + voyagerWriter.save("index.hnsw", "names.json") uri } } diff --git a/scio-extra/src/test/scala/com/spotify/scio/extra/voyager/VoyagerTest.scala b/scio-extra/src/test/scala/com/spotify/scio/extra/voyager/VoyagerTest.scala new file mode 100644 index 0000000000..64b2c85eed --- /dev/null +++ b/scio-extra/src/test/scala/com/spotify/scio/extra/voyager/VoyagerTest.scala @@ -0,0 +1,52 @@ +package com.spotify.scio.extra.voyager + +import com.spotify.scio.ScioContext +import com.spotify.scio.io.ClosedTap +import com.spotify.scio.testing.PipelineSpec +import com.spotify.voyager.jni.Index.{SpaceType, StorageDataType} +import com.spotify.voyager.jni.StringIndex + +import java.io.File +import java.nio.file.Files + +class VoyagerTest extends PipelineSpec { + val distanceMeasure = Cosine + val storageType = E4M3 + val dim = 2 + + val sideData: Seq[(String, Array[Float])] = + Seq(("1", Array(1.1f, 1.1f)), ("2", Array(2.2f, 2.2f)), ("3", Array(3.3f, 3.3f))) + + "SCollection" should "support .asVoyager with specified local file" in { + val tmpDir = Files.createTempDirectory("voyager-test") + val basePath = tmpDir.toString + val sc = ScioContext() + val p: ClosedTap[VoyagerUri] = + sc.parallelize(sideData) + .asVoyager(basePath, distanceMeasure, storageType, dim) + .materialize + + val scioResult = sc.run().waitUntilFinish() + val path = scioResult.tap(p).value.next.path + println(path) + + + val reader = StringIndex.load( + path + "/index.hnsw", + path + "/names.json", + SpaceType.Cosine, + dim, + StorageDataType.E4M3 + ) + + sideData.foreach { s => + println(reader.query(s._2, 1, 1).getNames.toString) + } + + for (file <- Seq("index.hnsw", "names.json")) { + new File(basePath + file).delete() + } + + } + +} From 16cc3e56017f6cc3e234408f5dbffa0479842321 Mon Sep 17 00:00:00 2001 From: Patrick McGee Date: Tue, 29 Aug 2023 16:07:15 -0400 Subject: [PATCH 05/28] basic testing done --- .../scio/extra/voyager/VoyagerUri.scala | 17 +++-- .../spotify/scio/extra/voyager/package.scala | 65 ++++++++++++++----- .../scio/extra/voyager/VoyagerTest.scala | 36 +++++++--- 3 files changed, 83 insertions(+), 35 deletions(-) diff --git a/scio-extra/src/main/scala/com/spotify/scio/extra/voyager/VoyagerUri.scala b/scio-extra/src/main/scala/com/spotify/scio/extra/voyager/VoyagerUri.scala index 9146ad7486..865c595a09 100644 --- a/scio-extra/src/main/scala/com/spotify/scio/extra/voyager/VoyagerUri.scala +++ b/scio-extra/src/main/scala/com/spotify/scio/extra/voyager/VoyagerUri.scala @@ -4,7 +4,7 @@ import com.spotify.scio.coders.Coder import com.spotify.scio.util.{RemoteFileUtil, ScioUtil} import java.nio.charset.StandardCharsets -import com.spotify.voyager.jni.{Index, StringIndex} +import com.spotify.voyager.jni.Index import com.spotify.voyager.jni.Index.{SpaceType, StorageDataType} import org.apache.beam.sdk.options.PipelineOptions @@ -12,7 +12,7 @@ import scala.jdk.CollectionConverters import scala.collection.mutable import java.io.File import java.net.URI -import java.nio.file.{Files, Paths} +import java.nio.file.{Files, Path, Paths} trait VoyagerUri extends Serializable { val path: String @@ -45,9 +45,7 @@ private class LocalVoyagerUri(val path: String) extends VoyagerUri { new VoyagerReader(path, distanceMeasure, storageType, dim) override private[voyager] def saveAndClose(w: VoyagerWriter): Unit = { - val indexPath = path + "/index.hnsw" - val namesPath = path + "/names.json" - w.save(indexPath, namesPath) + w.save(path) w.close() } @@ -67,9 +65,7 @@ private class RemoteVoyagerUri(val path: String, options: PipelineOptions) exten } override private[voyager] def saveAndClose(w: VoyagerWriter): Unit = { - val indexPath = path + "/index.hnsw" - val namesPath = path + "/names.json" - w.save(indexPath, namesPath) + w.save(path) w.close() } @@ -131,7 +127,10 @@ private[voyager] class VoyagerWriter( () } - def save(indexFileName: String, namesFileName: String): Unit = { + def save(path: String): Unit = { + val basePath: Path = Paths.get(path) + val indexFileName: String = basePath.resolve("index.hnsw").toString + val namesFileName: String = basePath.resolve("names.json").toString index.saveIndex(indexFileName) Files.write( Paths.get(namesFileName), diff --git a/scio-extra/src/main/scala/com/spotify/scio/extra/voyager/package.scala b/scio-extra/src/main/scala/com/spotify/scio/extra/voyager/package.scala index cb21b473a1..df4821d1ca 100644 --- a/scio-extra/src/main/scala/com/spotify/scio/extra/voyager/package.scala +++ b/scio-extra/src/main/scala/com/spotify/scio/extra/voyager/package.scala @@ -1,9 +1,14 @@ package com.spotify.scio.extra +import com.spotify.scio.ScioContext import com.spotify.scio.annotations.experimental -import com.spotify.scio.values.{DistCache, SCollection} +import com.spotify.scio.values.{DistCache, SCollection, SideInput} import com.spotify.voyager.jni.Index.{SpaceType, StorageDataType} import com.spotify.voyager.jni.{Index, StringIndex} +import org.apache.beam.sdk.transforms.{DoFn, View} +import org.apache.beam.sdk.values.PCollectionView + +import java.nio.file.{Path, Paths} package object voyager { sealed abstract class VoyagerDistanceMeasure @@ -16,6 +21,8 @@ package object voyager { case object Float32 extends VoyagerStorageType case object E4M3 extends VoyagerStorageType + case class VoyagerResult(value: String, distance: Float) + class VoyagerReader private[voyager] ( path: String, distanceMeasure: VoyagerDistanceMeasure, @@ -24,8 +31,9 @@ package object voyager { ) { require(dim > 0, "Vector dimension should be > 0") - private val indexPath: String = path + "/index.hnsw" - private val namesPath: String = path + "/names.json" + private val basePath: Path = Paths.get(path) + private val indexFileName: String = basePath.resolve("index.hnsw").toString + private val namesFileName: String = basePath.resolve("names.json").toString private val index: StringIndex = { val spaceType = distanceMeasure match { @@ -39,13 +47,17 @@ package object voyager { case Float32 => StorageDataType.Float32 case E4M3 => StorageDataType.E4M3 } - StringIndex.load(indexPath, namesPath, spaceType, dim, storageDataType) + StringIndex.load(indexFileName, namesFileName, spaceType, dim, storageDataType) } - // figure out index path + names path -// def getNearest(v: Array[Float], maxNumResults: Int) = index.query(v, maxNumResults) -// -// def getItemVector(i: Int) = index.getVector(i) + def getNearest(v: Array[Float], maxNumResults: Int, ef: Int): Array[VoyagerResult] = { + val queryResults = index.query(v, maxNumResults, ef) + queryResults.getNames + .zip(queryResults.getDistances) + .map { case (name, distance) => + VoyagerResult(name, distance.toFloat) + } + } } implicit class VoyagerPairSCollection( @@ -70,7 +82,7 @@ package object voyager { val voyagerWriter: VoyagerWriter = new VoyagerWriter(voyagerDistanceMeasure, voyagerStorageType, dim, ef, m) voyagerWriter.write(xs) - voyagerWriter.save("index.hnsw", "names.json") + voyagerWriter.save(path) uri } } @@ -79,14 +91,33 @@ package object voyager { } - def asVoyagerDistCache(): DistCache[VoyagerReader] = ??? + def asVoyagerSideInput(): SideInput[VoyagerReader] = ??? + + /** + * To be used with with side inputs + * @param self + */ + implicit class VoyagerScioContext(private val self: ScioContext) extends AnyVal { + def voyagerSideInput( + path: String, + distanceMeasure: VoyagerDistanceMeasure, + storageType: VoyagerStorageType, + dim: Int + ): SideInput[VoyagerReader] = { + val uri = VoyagerUri(path, self.options) + val view = self.parallelize(Seq(uri)).applyInternal(View.asSingleton()) + new VoyagerSideInput(view, distanceMeasure, storageType, dim) + } + } -// private class VoyagerDistCache( -// val view: PCollectionView[VoyagerUri], -// distanceMeasure: VoyagerDistanceMeasure, -// dim: Int -// ) extends DistCache[VoyagerReader] { -// override def apply(): VoyagerReader = ??? -// } + private class VoyagerSideInput( + val view: PCollectionView[VoyagerUri], + distanceMeasure: VoyagerDistanceMeasure, + storageType: VoyagerStorageType, + dim: Int + ) extends SideInput[VoyagerReader] { + override def get[I, O](context: DoFn[I, O]#ProcessContext): VoyagerReader = + context.sideInput(view).getReader(distanceMeasure, storageType, dim) + } } diff --git a/scio-extra/src/test/scala/com/spotify/scio/extra/voyager/VoyagerTest.scala b/scio-extra/src/test/scala/com/spotify/scio/extra/voyager/VoyagerTest.scala index 64b2c85eed..ecbe20d450 100644 --- a/scio-extra/src/test/scala/com/spotify/scio/extra/voyager/VoyagerTest.scala +++ b/scio-extra/src/test/scala/com/spotify/scio/extra/voyager/VoyagerTest.scala @@ -1,6 +1,6 @@ package com.spotify.scio.extra.voyager -import com.spotify.scio.ScioContext +import com.spotify.scio.{ScioContext, ScioResult} import com.spotify.scio.io.ClosedTap import com.spotify.scio.testing.PipelineSpec import com.spotify.voyager.jni.Index.{SpaceType, StorageDataType} @@ -15,7 +15,7 @@ class VoyagerTest extends PipelineSpec { val dim = 2 val sideData: Seq[(String, Array[Float])] = - Seq(("1", Array(1.1f, 1.1f)), ("2", Array(2.2f, 2.2f)), ("3", Array(3.3f, 3.3f))) + Seq(("1", Array(2.5f, 7.2f)), ("2", Array(1.2f, 2.2f)), ("3", Array(5.6f, 3.4f))) "SCollection" should "support .asVoyager with specified local file" in { val tmpDir = Files.createTempDirectory("voyager-test") @@ -26,12 +26,10 @@ class VoyagerTest extends PipelineSpec { .asVoyager(basePath, distanceMeasure, storageType, dim) .materialize - val scioResult = sc.run().waitUntilFinish() - val path = scioResult.tap(p).value.next.path - println(path) + val scioResult: ScioResult = sc.run().waitUntilFinish() + val path: String = scioResult.tap(p).value.next.path - - val reader = StringIndex.load( + val index: StringIndex = StringIndex.load( path + "/index.hnsw", path + "/names.json", SpaceType.Cosine, @@ -39,14 +37,34 @@ class VoyagerTest extends PipelineSpec { StorageDataType.E4M3 ) - sideData.foreach { s => - println(reader.query(s._2, 1, 1).getNames.toString) + sideData.foreach { data => + val result = index.query(data._2, 2, 100) + result.getNames.length shouldEqual 2 + result.getDistances.length shouldEqual 2 + result.getNames should contain(data._1) } for (file <- Seq("index.hnsw", "names.json")) { new File(basePath + file).delete() } + } + + it should "throw exception when the Voyager files already exists" in { + val tmpDir = Files.createTempDirectory("voyager-test") + + val index = tmpDir.resolve("index.hnsw") + val names = tmpDir.resolve("names.json") + Files.createFile(index) + Files.createFile(names) + + the[IllegalArgumentException] thrownBy { + runWithContext { + _.parallelize(sideData).asVoyager(tmpDir.toString, Cosine, E4M3, dim) + } + } should have message s"requirement failed: Voyager URI $tmpDir already exists" + Files.delete(index) + Files.delete(names) } } From 47e9bcb49007f1e9e9f83bb0f02c5d8eabf3dc95 Mon Sep 17 00:00:00 2001 From: Patrick McGee Date: Fri, 1 Sep 2023 16:06:27 -0400 Subject: [PATCH 06/28] updates galore --- .../scio/extra/voyager/VoyagerIT.scala | 45 ++++++++++++ .../scio/extra/voyager/VoyagerUri.scala | 20 ++++-- .../spotify/scio/extra/voyager/package.scala | 69 +++++++++++++++---- .../scio/extra/voyager/VoyagerTest.scala | 4 +- 4 files changed, 119 insertions(+), 19 deletions(-) create mode 100644 scio-extra/src/it/scala/com/spotify/scio/extra/voyager/VoyagerIT.scala diff --git a/scio-extra/src/it/scala/com/spotify/scio/extra/voyager/VoyagerIT.scala b/scio-extra/src/it/scala/com/spotify/scio/extra/voyager/VoyagerIT.scala new file mode 100644 index 0000000000..0f58a9011c --- /dev/null +++ b/scio-extra/src/it/scala/com/spotify/scio/extra/voyager/VoyagerIT.scala @@ -0,0 +1,45 @@ +package com.spotify.scio.extra.voyager + +import com.spotify.scio.testing.PipelineSpec +import com.spotify.scio.testing.util.ItUtils +import com.spotify.scio.values.{SCollection, SideInput} +import org.apache.beam.sdk.io.FileSystems + +import scala.jdk.CollectionConverters._ +class VoyagerIT extends PipelineSpec { + val dim: Int = 2 + val storageType: VoyagerStorageType = E4M3 + val distanceMeasure: VoyagerDistanceMeasure = Cosine + + val sideData: Seq[(String, Array[Float])] = + Seq(("1", Array(2.5f, 7.2f)), ("2", Array(1.2f, 2.2f)), ("3", Array(5.6f, 3.4f))) + it should "support .asVoyagerSideInput using GCS tempLocation" in { + runWithContext { sc => + FileSystems.setDefaultPipelineOptions(sc.options) + + val tempLocation = ItUtils.gcpTempLocation("voyager-it") + + try { + val p1 = sc.parallelize(sideData) + val p2: SideInput[VoyagerReader] = + sc.parallelize(sideData).asVoyagerSideInput(distanceMeasure, storageType, dim) + val s = p1 + .withSideInputs(p2) + .flatMap { (xs, si) => + si(p2).getNearest(xs._2, 1, 100) + } + .toSCollection + + s should containInAnyOrder(sideData.map(_._1)) + } finally { + val files = FileSystems + .`match`(s"$tempLocation") + .metadata() + .asScala + .map(_.resourceId()) + + FileSystems.delete(files.asJava) + } + } + } +} diff --git a/scio-extra/src/main/scala/com/spotify/scio/extra/voyager/VoyagerUri.scala b/scio-extra/src/main/scala/com/spotify/scio/extra/voyager/VoyagerUri.scala index 865c595a09..788cb1ad70 100644 --- a/scio-extra/src/main/scala/com/spotify/scio/extra/voyager/VoyagerUri.scala +++ b/scio-extra/src/main/scala/com/spotify/scio/extra/voyager/VoyagerUri.scala @@ -7,6 +7,7 @@ import java.nio.charset.StandardCharsets import com.spotify.voyager.jni.Index import com.spotify.voyager.jni.Index.{SpaceType, StorageDataType} import org.apache.beam.sdk.options.PipelineOptions +import org.slf4j.LoggerFactory import scala.jdk.CollectionConverters import scala.collection.mutable @@ -15,6 +16,7 @@ import java.net.URI import java.nio.file.{Files, Path, Paths} trait VoyagerUri extends Serializable { + val logger = LoggerFactory.getLogger(this.getClass) val path: String private[voyager] def getReader( distanceMeasure: VoyagerDistanceMeasure, @@ -65,8 +67,18 @@ private class RemoteVoyagerUri(val path: String, options: PipelineOptions) exten } override private[voyager] def saveAndClose(w: VoyagerWriter): Unit = { - w.save(path) + val tempPath: Path = Files.createTempDirectory("") + logger.info(s"temp path: $path") + w.save(tempPath.toString) w.close() + + VoyagerUri.files.foreach { f => + val filePath = tempPath.resolve(f) + logger.info(s"resolved filePath $filePath") + rfu.upload(Paths.get(filePath.toString), new URI(path + "/" + f)) + Files.delete(filePath) + } + } override private[voyager] def exists: Boolean = @@ -80,6 +92,7 @@ private[voyager] class VoyagerWriter( ef: Long = 200L, m: Long = 16L ) { + private val logger = LoggerFactory.getLogger(this.getClass) // Chunk size experiments - , // 4096, 6062: 2022-11-16 14:07:07.358 -> 2022-11-16 16:50:59.109 = 2hr 50min. 1.68s per chunk @@ -128,9 +141,8 @@ private[voyager] class VoyagerWriter( } def save(path: String): Unit = { - val basePath: Path = Paths.get(path) - val indexFileName: String = basePath.resolve("index.hnsw").toString - val namesFileName: String = basePath.resolve("names.json").toString + val indexFileName: String = path + "/index.hnsw" + val namesFileName: String = path + "/names.json" index.saveIndex(indexFileName) Files.write( Paths.get(namesFileName), diff --git a/scio-extra/src/main/scala/com/spotify/scio/extra/voyager/package.scala b/scio-extra/src/main/scala/com/spotify/scio/extra/voyager/package.scala index df4821d1ca..b98f34c1e8 100644 --- a/scio-extra/src/main/scala/com/spotify/scio/extra/voyager/package.scala +++ b/scio-extra/src/main/scala/com/spotify/scio/extra/voyager/package.scala @@ -7,8 +7,10 @@ import com.spotify.voyager.jni.Index.{SpaceType, StorageDataType} import com.spotify.voyager.jni.{Index, StringIndex} import org.apache.beam.sdk.transforms.{DoFn, View} import org.apache.beam.sdk.values.PCollectionView +import org.slf4j.LoggerFactory import java.nio.file.{Path, Paths} +import java.util.UUID package object voyager { sealed abstract class VoyagerDistanceMeasure @@ -20,6 +22,7 @@ package object voyager { case object Float8 extends VoyagerStorageType case object Float32 extends VoyagerStorageType case object E4M3 extends VoyagerStorageType + private val logger = LoggerFactory.getLogger(this.getClass) case class VoyagerResult(value: String, distance: Float) @@ -60,6 +63,24 @@ package object voyager { } } + /** + * To be used with with side inputs + * + * @param self + */ + implicit class VoyagerScioContext(private val self: ScioContext) extends AnyVal { + def voyagerSideInput( + path: String, + distanceMeasure: VoyagerDistanceMeasure, + storageType: VoyagerStorageType, + dim: Int + ): SideInput[VoyagerReader] = { + val uri = VoyagerUri(path, self.options) + val view = self.parallelize(Seq(uri)).applyInternal(View.asSingleton()) + new VoyagerSideInput(view, distanceMeasure, storageType, dim) + } + } + implicit class VoyagerPairSCollection( @transient private val self: SCollection[(String, Array[Float])] ) extends AnyVal { @@ -70,42 +91,64 @@ package object voyager { voyagerDistanceMeasure: VoyagerDistanceMeasure, voyagerStorageType: VoyagerStorageType, dim: Int, - ef: Long = 200L, - m: Long = 16L + ef: Long, + m: Long ): SCollection[VoyagerUri] = { val uri: VoyagerUri = VoyagerUri(path, self.context.options) require(!uri.exists, s"Voyager URI ${uri.path} already exists") + logger.info(s"Vyager URI :${uri.path}") self.transform { in => { in.groupBy(_ => ()) .map { case (_, xs) => val voyagerWriter: VoyagerWriter = new VoyagerWriter(voyagerDistanceMeasure, voyagerStorageType, dim, ef, m) + voyagerWriter.write(xs) - voyagerWriter.save(path) + uri.saveAndClose(voyagerWriter) uri } } } } + @experimental + def asVoyager( + distanceMeasure: VoyagerDistanceMeasure, + storageType: VoyagerStorageType, + dim: Int, + ef: Long = 200L, + m: Long = 16L + ): SCollection[VoyagerUri] = { + val uuid: UUID = UUID.randomUUID() + val tempLocation: String = self.context.options.getTempLocation + require(tempLocation != null, s"--tempLocation arg is required") + val path = s"$tempLocation/voyager-build-$uuid" + this.asVoyager(path, distanceMeasure, storageType, dim, ef, m) + } + + def asVoyagerSideInput( + distanceMeasure: VoyagerDistanceMeasure, + storageType: VoyagerStorageType, + dim: Int, + ef: Long = 200L, + m: Long = 16L + ): SideInput[VoyagerReader] = + self + .asVoyager(distanceMeasure, storageType, dim, ef, m) + .asVoyagerSideInput(distanceMeasure, storageType, dim) } - def asVoyagerSideInput(): SideInput[VoyagerReader] = ??? + implicit class AnnoySCollection(@transient private val self: SCollection[VoyagerUri]) + extends AnyVal { - /** - * To be used with with side inputs - * @param self - */ - implicit class VoyagerScioContext(private val self: ScioContext) extends AnyVal { - def voyagerSideInput( - path: String, + @experimental + def asVoyagerSideInput( distanceMeasure: VoyagerDistanceMeasure, storageType: VoyagerStorageType, dim: Int ): SideInput[VoyagerReader] = { - val uri = VoyagerUri(path, self.options) - val view = self.parallelize(Seq(uri)).applyInternal(View.asSingleton()) + val view = self.applyInternal(View.asSingleton()) new VoyagerSideInput(view, distanceMeasure, storageType, dim) } } diff --git a/scio-extra/src/test/scala/com/spotify/scio/extra/voyager/VoyagerTest.scala b/scio-extra/src/test/scala/com/spotify/scio/extra/voyager/VoyagerTest.scala index ecbe20d450..78a80a74f7 100644 --- a/scio-extra/src/test/scala/com/spotify/scio/extra/voyager/VoyagerTest.scala +++ b/scio-extra/src/test/scala/com/spotify/scio/extra/voyager/VoyagerTest.scala @@ -23,7 +23,7 @@ class VoyagerTest extends PipelineSpec { val sc = ScioContext() val p: ClosedTap[VoyagerUri] = sc.parallelize(sideData) - .asVoyager(basePath, distanceMeasure, storageType, dim) + .asVoyager(basePath, distanceMeasure, storageType, dim, 200L, 16L) .materialize val scioResult: ScioResult = sc.run().waitUntilFinish() @@ -59,7 +59,7 @@ class VoyagerTest extends PipelineSpec { the[IllegalArgumentException] thrownBy { runWithContext { - _.parallelize(sideData).asVoyager(tmpDir.toString, Cosine, E4M3, dim) + _.parallelize(sideData).asVoyager(tmpDir.toString, Cosine, E4M3, dim, 200L, 16L) } } should have message s"requirement failed: Voyager URI $tmpDir already exists" From 7290bee5ca8e8c1aa21e529dbaac6931c6f0e50a Mon Sep 17 00:00:00 2001 From: Patrick McGee Date: Fri, 1 Sep 2023 16:19:33 -0400 Subject: [PATCH 07/28] copyrights --- .../scio/extra/voyager/VoyagerIT.scala | 19 ++++++++++++++++++- .../scio/extra/voyager/VoyagerUri.scala | 16 ++++++++++++++++ .../spotify/scio/extra/voyager/package.scala | 16 ++++++++++++++++ .../scio/extra/voyager/VoyagerTest.scala | 16 ++++++++++++++++ 4 files changed, 66 insertions(+), 1 deletion(-) diff --git a/scio-extra/src/it/scala/com/spotify/scio/extra/voyager/VoyagerIT.scala b/scio-extra/src/it/scala/com/spotify/scio/extra/voyager/VoyagerIT.scala index 0f58a9011c..fa441b3a70 100644 --- a/scio-extra/src/it/scala/com/spotify/scio/extra/voyager/VoyagerIT.scala +++ b/scio-extra/src/it/scala/com/spotify/scio/extra/voyager/VoyagerIT.scala @@ -1,11 +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.extra.voyager import com.spotify.scio.testing.PipelineSpec import com.spotify.scio.testing.util.ItUtils -import com.spotify.scio.values.{SCollection, SideInput} +import com.spotify.scio.values.SideInput import org.apache.beam.sdk.io.FileSystems import scala.jdk.CollectionConverters._ + class VoyagerIT extends PipelineSpec { val dim: Int = 2 val storageType: VoyagerStorageType = E4M3 diff --git a/scio-extra/src/main/scala/com/spotify/scio/extra/voyager/VoyagerUri.scala b/scio-extra/src/main/scala/com/spotify/scio/extra/voyager/VoyagerUri.scala index 788cb1ad70..ce3554dd48 100644 --- a/scio-extra/src/main/scala/com/spotify/scio/extra/voyager/VoyagerUri.scala +++ b/scio-extra/src/main/scala/com/spotify/scio/extra/voyager/VoyagerUri.scala @@ -1,3 +1,19 @@ +/* + * 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.extra.voyager import com.spotify.scio.coders.Coder diff --git a/scio-extra/src/main/scala/com/spotify/scio/extra/voyager/package.scala b/scio-extra/src/main/scala/com/spotify/scio/extra/voyager/package.scala index b98f34c1e8..b23cdf3784 100644 --- a/scio-extra/src/main/scala/com/spotify/scio/extra/voyager/package.scala +++ b/scio-extra/src/main/scala/com/spotify/scio/extra/voyager/package.scala @@ -1,3 +1,19 @@ +/* + * 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.extra import com.spotify.scio.ScioContext diff --git a/scio-extra/src/test/scala/com/spotify/scio/extra/voyager/VoyagerTest.scala b/scio-extra/src/test/scala/com/spotify/scio/extra/voyager/VoyagerTest.scala index 78a80a74f7..6ad4eaddbc 100644 --- a/scio-extra/src/test/scala/com/spotify/scio/extra/voyager/VoyagerTest.scala +++ b/scio-extra/src/test/scala/com/spotify/scio/extra/voyager/VoyagerTest.scala @@ -1,3 +1,19 @@ +/* + * 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.extra.voyager import com.spotify.scio.{ScioContext, ScioResult} From a3a14b312edc23e944faf4d0eb9a1f858a6cd837 Mon Sep 17 00:00:00 2001 From: Patrick McGee Date: Wed, 6 Sep 2023 15:52:46 -0400 Subject: [PATCH 08/28] add in updates --- .../scio/extra/voyager/VoyagerIT.scala | 28 +++++++++++++++++++ 1 file changed, 28 insertions(+) diff --git a/scio-extra/src/it/scala/com/spotify/scio/extra/voyager/VoyagerIT.scala b/scio-extra/src/it/scala/com/spotify/scio/extra/voyager/VoyagerIT.scala index fa441b3a70..1679425bfc 100644 --- a/scio-extra/src/it/scala/com/spotify/scio/extra/voyager/VoyagerIT.scala +++ b/scio-extra/src/it/scala/com/spotify/scio/extra/voyager/VoyagerIT.scala @@ -20,7 +20,9 @@ import com.spotify.scio.testing.PipelineSpec import com.spotify.scio.testing.util.ItUtils import com.spotify.scio.values.SideInput import org.apache.beam.sdk.io.FileSystems +import org.apache.beam.sdk.util.MimeTypes +import java.nio.ByteBuffer import scala.jdk.CollectionConverters._ class VoyagerIT extends PipelineSpec { @@ -59,4 +61,30 @@ class VoyagerIT extends PipelineSpec { } } } + + it should "support .asVoyagerSideInput using GCS tempLocation" in { + runWithContext { sc => + FileSystems.setDefaultPipelineOptions(sc.options) + + val tempLocation = ItUtils.gcpTempLocation("voyager-it") + val namePath = tempLocation + "/names.json" + val indexPath = tempLocation + "/index.hnsw" + val nameResourceId = FileSystems.matchNewResource(namePath, false) + val indexResourceId = FileSystems.matchNewResource(indexPath, false) + + try { + val f1 = FileSystems.create(nameResourceId, MimeTypes.BINARY) + val f2 = FileSystems.create(indexResourceId, MimeTypes.BINARY) + f1.write(ByteBuffer.wrap("test-data".getBytes())) + f1.close() + f2.write(ByteBuffer.wrap("test-data".getBytes())) + f2.close() + the[IllegalArgumentException] thrownBy { + sc.parallelize(sideData).asVoyager(distanceMeasure, storageType, dim) + } should have message s"" + } finally { + FileSystems.delete(Seq(nameResourceId, indexResourceId).asJava) + } + } + } } From 2923e00208d464279da201f1d311050fd2c12675 Mon Sep 17 00:00:00 2001 From: Patrick McGee Date: Thu, 7 Sep 2023 16:51:23 -0400 Subject: [PATCH 09/28] Update w/ multi pathing might revert it out though --- .../scio/extra/voyager/VoyagerUri.scala | 65 ++++++++++++------- .../spotify/scio/extra/voyager/package.scala | 20 +++--- .../scio/extra/voyager/VoyagerTest.scala | 2 +- 3 files changed, 50 insertions(+), 37 deletions(-) diff --git a/scio-extra/src/main/scala/com/spotify/scio/extra/voyager/VoyagerUri.scala b/scio-extra/src/main/scala/com/spotify/scio/extra/voyager/VoyagerUri.scala index ce3554dd48..01e1cb81ed 100644 --- a/scio-extra/src/main/scala/com/spotify/scio/extra/voyager/VoyagerUri.scala +++ b/scio-extra/src/main/scala/com/spotify/scio/extra/voyager/VoyagerUri.scala @@ -33,7 +33,8 @@ import java.nio.file.{Files, Path, Paths} trait VoyagerUri extends Serializable { val logger = LoggerFactory.getLogger(this.getClass) - val path: String + val indexPath: String + val namesPath: String private[voyager] def getReader( distanceMeasure: VoyagerDistanceMeasure, storageType: VoyagerStorageType, @@ -44,61 +45,75 @@ trait VoyagerUri extends Serializable { } private[voyager] object VoyagerUri { - def apply(path: String, opts: PipelineOptions): VoyagerUri = - if (ScioUtil.isLocalUri(new URI(path))) { - new LocalVoyagerUri(path) + def apply(indexPath: String, namesPath: String = null, opts: PipelineOptions): VoyagerUri = { + val actualNamesPath = Option(namesPath).getOrElse(indexPath) + if (ScioUtil.isLocalUri(new URI(indexPath))) { + new LocalVoyagerUri(indexPath, actualNamesPath) } else { - new RemoteVoyagerUri(path, opts) + new RemoteVoyagerUri(indexPath, actualNamesPath, opts) } + } + def files: Seq[String] = Seq("index.hnsw", "names.json") implicit val voyagerUriCoder: Coder[VoyagerUri] = Coder.kryo[VoyagerUri] } -private class LocalVoyagerUri(val path: String) extends VoyagerUri { +private class LocalVoyagerUri(val indexPath: String, val namesPath: String) extends VoyagerUri { override private[voyager] def getReader( distanceMeasure: VoyagerDistanceMeasure, storageType: VoyagerStorageType, dim: Int - ): VoyagerReader = - new VoyagerReader(path, distanceMeasure, storageType, dim) + ): VoyagerReader = { + + val indexFileName: String = indexPath + "/index.hnsw" + val namesFileName: String = namesPath + "/names.json" + new VoyagerReader(indexFileName, namesFileName, distanceMeasure, storageType, dim) + } override private[voyager] def saveAndClose(w: VoyagerWriter): Unit = { - w.save(path) + w.save(indexPath, namesPath) w.close() } override private[voyager] def exists: Boolean = - VoyagerUri.files.map(f => new File(path + "/" + f)).exists(_.exists()) + new File(indexPath + "/index.hnsw").exists() && new File(namesPath + "/names.json").exists() } -private class RemoteVoyagerUri(val path: String, options: PipelineOptions) extends VoyagerUri { +private class RemoteVoyagerUri( + val indexPath: String, + val namesPath: String, + options: PipelineOptions +) extends VoyagerUri { private[this] val rfu: RemoteFileUtil = RemoteFileUtil.create(options) override private[voyager] def getReader( distanceMeasure: VoyagerDistanceMeasure, storageType: VoyagerStorageType, dim: Int ): VoyagerReader = { - val localPath = rfu.download(new URI(path)) - new VoyagerReader(localPath.toString, distanceMeasure, storageType, dim) + val indexFileName: String = rfu.download(new URI(indexPath + "/index.hnsw")).toString + val namesFileName: String = rfu.download(new URI(namesPath + "/names.json")).toString + new VoyagerReader(indexFileName, namesFileName, distanceMeasure, storageType, dim) } override private[voyager] def saveAndClose(w: VoyagerWriter): Unit = { val tempPath: Path = Files.createTempDirectory("") - logger.info(s"temp path: $path") - w.save(tempPath.toString) + logger.info(s"temp path: $tempPath") + w.save(tempPath.toString, tempPath.toString) w.close() - VoyagerUri.files.foreach { f => - val filePath = tempPath.resolve(f) - logger.info(s"resolved filePath $filePath") - rfu.upload(Paths.get(filePath.toString), new URI(path + "/" + f)) - Files.delete(filePath) - } + val tempIndexPath = tempPath.resolve("index.hnsw") + rfu.upload(Paths.get(tempIndexPath.toString), new URI(indexPath + "/index.hnsw")) + val tempNamesPath = tempPath.resolve("names.json") + rfu.upload(Paths.get(tempNamesPath.toString), new URI(namesPath + "/names.json")) + Files.delete(tempIndexPath) + Files.delete(tempNamesPath) } override private[voyager] def exists: Boolean = - VoyagerUri.files.exists(f => rfu.remoteExists(new URI(path + "/" + f))) + rfu.remoteExists(new URI(indexPath + "/index.hnsw")) && rfu.remoteExists( + new URI(namesPath + "/names.json") + ) } private[voyager] class VoyagerWriter( @@ -156,9 +171,9 @@ private[voyager] class VoyagerWriter( () } - def save(path: String): Unit = { - val indexFileName: String = path + "/index.hnsw" - val namesFileName: String = path + "/names.json" + def save(indexPath: String, namesPath: String): Unit = { + val indexFileName: String = indexPath + "/index.hnsw" + val namesFileName: String = namesPath + "/names.json" index.saveIndex(indexFileName) Files.write( Paths.get(namesFileName), diff --git a/scio-extra/src/main/scala/com/spotify/scio/extra/voyager/package.scala b/scio-extra/src/main/scala/com/spotify/scio/extra/voyager/package.scala index b23cdf3784..e76b531088 100644 --- a/scio-extra/src/main/scala/com/spotify/scio/extra/voyager/package.scala +++ b/scio-extra/src/main/scala/com/spotify/scio/extra/voyager/package.scala @@ -43,17 +43,14 @@ package object voyager { case class VoyagerResult(value: String, distance: Float) class VoyagerReader private[voyager] ( - path: String, + indexFileName: String, + namesFileName: String, distanceMeasure: VoyagerDistanceMeasure, storageType: VoyagerStorageType, dim: Int ) { require(dim > 0, "Vector dimension should be > 0") - private val basePath: Path = Paths.get(path) - private val indexFileName: String = basePath.resolve("index.hnsw").toString - private val namesFileName: String = basePath.resolve("names.json").toString - private val index: StringIndex = { val spaceType = distanceMeasure match { case Euclidean => SpaceType.Euclidean @@ -86,12 +83,13 @@ package object voyager { */ implicit class VoyagerScioContext(private val self: ScioContext) extends AnyVal { def voyagerSideInput( - path: String, + indexPath: String, + namesPath: String = null, distanceMeasure: VoyagerDistanceMeasure, storageType: VoyagerStorageType, dim: Int ): SideInput[VoyagerReader] = { - val uri = VoyagerUri(path, self.options) + val uri = VoyagerUri(indexPath, namesPath, self.options) val view = self.parallelize(Seq(uri)).applyInternal(View.asSingleton()) new VoyagerSideInput(view, distanceMeasure, storageType, dim) } @@ -103,16 +101,16 @@ package object voyager { @experimental def asVoyager( - path: String, + indexPath: String, voyagerDistanceMeasure: VoyagerDistanceMeasure, voyagerStorageType: VoyagerStorageType, dim: Int, ef: Long, m: Long ): SCollection[VoyagerUri] = { - val uri: VoyagerUri = VoyagerUri(path, self.context.options) - require(!uri.exists, s"Voyager URI ${uri.path} already exists") - logger.info(s"Vyager URI :${uri.path}") + val uri: VoyagerUri = VoyagerUri(indexPath, indexPath, self.context.options) + require(!uri.exists, s"Voyager URI ${uri.indexPath} already exists") + logger.info(s"Voyager URI :${uri.indexPath}") self.transform { in => { in.groupBy(_ => ()) diff --git a/scio-extra/src/test/scala/com/spotify/scio/extra/voyager/VoyagerTest.scala b/scio-extra/src/test/scala/com/spotify/scio/extra/voyager/VoyagerTest.scala index 6ad4eaddbc..bd0260ef78 100644 --- a/scio-extra/src/test/scala/com/spotify/scio/extra/voyager/VoyagerTest.scala +++ b/scio-extra/src/test/scala/com/spotify/scio/extra/voyager/VoyagerTest.scala @@ -43,7 +43,7 @@ class VoyagerTest extends PipelineSpec { .materialize val scioResult: ScioResult = sc.run().waitUntilFinish() - val path: String = scioResult.tap(p).value.next.path + val path: String = scioResult.tap(p).value.next.indexPath val index: StringIndex = StringIndex.load( path + "/index.hnsw", From 9f5a170ecb0c80bda8717309de4f9b6a7b74cbe9 Mon Sep 17 00:00:00 2001 From: Patrick McGee Date: Mon, 11 Sep 2023 15:18:46 -0400 Subject: [PATCH 10/28] Add in refactor + updates to get working for reading indicies --- .../scio/extra/voyager/VoyagerUri.scala | 53 ++++++++----------- .../spotify/scio/extra/voyager/package.scala | 29 ++++++---- .../scio/extra/voyager/VoyagerTest.scala | 2 +- 3 files changed, 43 insertions(+), 41 deletions(-) diff --git a/scio-extra/src/main/scala/com/spotify/scio/extra/voyager/VoyagerUri.scala b/scio-extra/src/main/scala/com/spotify/scio/extra/voyager/VoyagerUri.scala index 01e1cb81ed..10b7eefbac 100644 --- a/scio-extra/src/main/scala/com/spotify/scio/extra/voyager/VoyagerUri.scala +++ b/scio-extra/src/main/scala/com/spotify/scio/extra/voyager/VoyagerUri.scala @@ -33,8 +33,7 @@ import java.nio.file.{Files, Path, Paths} trait VoyagerUri extends Serializable { val logger = LoggerFactory.getLogger(this.getClass) - val indexPath: String - val namesPath: String + val path: String private[voyager] def getReader( distanceMeasure: VoyagerDistanceMeasure, storageType: VoyagerStorageType, @@ -45,12 +44,11 @@ trait VoyagerUri extends Serializable { } private[voyager] object VoyagerUri { - def apply(indexPath: String, namesPath: String = null, opts: PipelineOptions): VoyagerUri = { - val actualNamesPath = Option(namesPath).getOrElse(indexPath) - if (ScioUtil.isLocalUri(new URI(indexPath))) { - new LocalVoyagerUri(indexPath, actualNamesPath) + def apply(path: String, opts: PipelineOptions): VoyagerUri = { + if (ScioUtil.isLocalUri(new URI(path))) { + new LocalVoyagerUri(path) } else { - new RemoteVoyagerUri(indexPath, actualNamesPath, opts) + new RemoteVoyagerUri(path, opts) } } @@ -58,30 +56,29 @@ private[voyager] object VoyagerUri { implicit val voyagerUriCoder: Coder[VoyagerUri] = Coder.kryo[VoyagerUri] } -private class LocalVoyagerUri(val indexPath: String, val namesPath: String) extends VoyagerUri { +private class LocalVoyagerUri(val path: String) extends VoyagerUri { override private[voyager] def getReader( distanceMeasure: VoyagerDistanceMeasure, storageType: VoyagerStorageType, dim: Int ): VoyagerReader = { - val indexFileName: String = indexPath + "/index.hnsw" - val namesFileName: String = namesPath + "/names.json" + val indexFileName: String = path + "/index.hnsw" + val namesFileName: String = path + "/names.json" new VoyagerReader(indexFileName, namesFileName, distanceMeasure, storageType, dim) } override private[voyager] def saveAndClose(w: VoyagerWriter): Unit = { - w.save(indexPath, namesPath) + w.save(path) w.close() } override private[voyager] def exists: Boolean = - new File(indexPath + "/index.hnsw").exists() && new File(namesPath + "/names.json").exists() + VoyagerUri.files.exists(f => new File(path + "/" + f).exists()) } private class RemoteVoyagerUri( - val indexPath: String, - val namesPath: String, + val path: String, options: PipelineOptions ) extends VoyagerUri { private[this] val rfu: RemoteFileUtil = RemoteFileUtil.create(options) @@ -90,30 +87,26 @@ private class RemoteVoyagerUri( storageType: VoyagerStorageType, dim: Int ): VoyagerReader = { - val indexFileName: String = rfu.download(new URI(indexPath + "/index.hnsw")).toString - val namesFileName: String = rfu.download(new URI(namesPath + "/names.json")).toString + val indexFileName: String = rfu.download(new URI(path + "/index.hnsw")).toString + val namesFileName: String = rfu.download(new URI(path + "/names.json")).toString new VoyagerReader(indexFileName, namesFileName, distanceMeasure, storageType, dim) } override private[voyager] def saveAndClose(w: VoyagerWriter): Unit = { val tempPath: Path = Files.createTempDirectory("") logger.info(s"temp path: $tempPath") - w.save(tempPath.toString, tempPath.toString) + w.save(tempPath.toString) w.close() - val tempIndexPath = tempPath.resolve("index.hnsw") - rfu.upload(Paths.get(tempIndexPath.toString), new URI(indexPath + "/index.hnsw")) - val tempNamesPath = tempPath.resolve("names.json") - rfu.upload(Paths.get(tempNamesPath.toString), new URI(namesPath + "/names.json")) - - Files.delete(tempIndexPath) - Files.delete(tempNamesPath) + VoyagerUri.files.foreach { f => + val tf = tempPath.resolve(f) + rfu.upload(Paths.get(tf.toString), new URI(path + "/" + f)) + Files.delete(tf) + } } override private[voyager] def exists: Boolean = - rfu.remoteExists(new URI(indexPath + "/index.hnsw")) && rfu.remoteExists( - new URI(namesPath + "/names.json") - ) + VoyagerUri.files.exists(f => rfu.remoteExists(new URI(path + "/" + f))) } private[voyager] class VoyagerWriter( @@ -171,9 +164,9 @@ private[voyager] class VoyagerWriter( () } - def save(indexPath: String, namesPath: String): Unit = { - val indexFileName: String = indexPath + "/index.hnsw" - val namesFileName: String = namesPath + "/names.json" + def save(path: String): Unit = { + val indexFileName: String = path + "/index.hnsw" + val namesFileName: String = path + "/names.json" index.saveIndex(indexFileName) Files.write( Paths.get(namesFileName), diff --git a/scio-extra/src/main/scala/com/spotify/scio/extra/voyager/package.scala b/scio-extra/src/main/scala/com/spotify/scio/extra/voyager/package.scala index e76b531088..459745fe04 100644 --- a/scio-extra/src/main/scala/com/spotify/scio/extra/voyager/package.scala +++ b/scio-extra/src/main/scala/com/spotify/scio/extra/voyager/package.scala @@ -25,8 +25,8 @@ import org.apache.beam.sdk.transforms.{DoFn, View} import org.apache.beam.sdk.values.PCollectionView import org.slf4j.LoggerFactory -import java.nio.file.{Path, Paths} import java.util.UUID +import scala.collection.mutable package object voyager { sealed abstract class VoyagerDistanceMeasure @@ -42,6 +42,8 @@ package object voyager { case class VoyagerResult(value: String, distance: Float) + val VOYAGER_URI_MAP: mutable.Map[VoyagerUri, VoyagerReader] = mutable.HashMap.empty + class VoyagerReader private[voyager] ( indexFileName: String, namesFileName: String, @@ -83,13 +85,12 @@ package object voyager { */ implicit class VoyagerScioContext(private val self: ScioContext) extends AnyVal { def voyagerSideInput( - indexPath: String, - namesPath: String = null, + path: String, distanceMeasure: VoyagerDistanceMeasure, storageType: VoyagerStorageType, dim: Int ): SideInput[VoyagerReader] = { - val uri = VoyagerUri(indexPath, namesPath, self.options) + val uri = VoyagerUri(path, self.options) val view = self.parallelize(Seq(uri)).applyInternal(View.asSingleton()) new VoyagerSideInput(view, distanceMeasure, storageType, dim) } @@ -101,16 +102,16 @@ package object voyager { @experimental def asVoyager( - indexPath: String, + path: String, voyagerDistanceMeasure: VoyagerDistanceMeasure, voyagerStorageType: VoyagerStorageType, dim: Int, ef: Long, m: Long ): SCollection[VoyagerUri] = { - val uri: VoyagerUri = VoyagerUri(indexPath, indexPath, self.context.options) - require(!uri.exists, s"Voyager URI ${uri.indexPath} already exists") - logger.info(s"Voyager URI :${uri.indexPath}") + val uri: VoyagerUri = VoyagerUri(path, self.context.options) + require(!uri.exists, s"Voyager URI ${uri.path} already exists") + logger.info(s"Voyager URI :${uri.path}") self.transform { in => { in.groupBy(_ => ()) @@ -173,8 +174,16 @@ package object voyager { storageType: VoyagerStorageType, dim: Int ) extends SideInput[VoyagerReader] { - override def get[I, O](context: DoFn[I, O]#ProcessContext): VoyagerReader = - context.sideInput(view).getReader(distanceMeasure, storageType, dim) + override def get[I, O](context: DoFn[I, O]#ProcessContext): VoyagerReader = { + val uri = context.sideInput(view) + VOYAGER_URI_MAP.synchronized { + if (!VOYAGER_URI_MAP.contains(uri)) { + VOYAGER_URI_MAP.put(uri, uri.getReader(distanceMeasure, storageType, dim)) + } + VOYAGER_URI_MAP(uri) + } + + } } } diff --git a/scio-extra/src/test/scala/com/spotify/scio/extra/voyager/VoyagerTest.scala b/scio-extra/src/test/scala/com/spotify/scio/extra/voyager/VoyagerTest.scala index bd0260ef78..6ad4eaddbc 100644 --- a/scio-extra/src/test/scala/com/spotify/scio/extra/voyager/VoyagerTest.scala +++ b/scio-extra/src/test/scala/com/spotify/scio/extra/voyager/VoyagerTest.scala @@ -43,7 +43,7 @@ class VoyagerTest extends PipelineSpec { .materialize val scioResult: ScioResult = sc.run().waitUntilFinish() - val path: String = scioResult.tap(p).value.next.indexPath + val path: String = scioResult.tap(p).value.next.path val index: StringIndex = StringIndex.load( path + "/index.hnsw", From 5dcbc52afa2f48615250638a231259c7b9b14e24 Mon Sep 17 00:00:00 2001 From: Patrick McGee Date: Tue, 12 Sep 2023 14:55:03 -0400 Subject: [PATCH 11/28] add in doc strings --- .../scio/extra/voyager/VoyagerUri.scala | 15 +- .../spotify/scio/extra/voyager/package.scala | 142 ++++++++++++++++-- 2 files changed, 140 insertions(+), 17 deletions(-) diff --git a/scio-extra/src/main/scala/com/spotify/scio/extra/voyager/VoyagerUri.scala b/scio-extra/src/main/scala/com/spotify/scio/extra/voyager/VoyagerUri.scala index 10b7eefbac..8606c50227 100644 --- a/scio-extra/src/main/scala/com/spotify/scio/extra/voyager/VoyagerUri.scala +++ b/scio-extra/src/main/scala/com/spotify/scio/extra/voyager/VoyagerUri.scala @@ -18,19 +18,23 @@ package com.spotify.scio.extra.voyager import com.spotify.scio.coders.Coder import com.spotify.scio.util.{RemoteFileUtil, ScioUtil} - -import java.nio.charset.StandardCharsets import com.spotify.voyager.jni.Index import com.spotify.voyager.jni.Index.{SpaceType, StorageDataType} import org.apache.beam.sdk.options.PipelineOptions import org.slf4j.LoggerFactory -import scala.jdk.CollectionConverters -import scala.collection.mutable import java.io.File import java.net.URI +import java.nio.charset.StandardCharsets import java.nio.file.{Files, Path, Paths} +import scala.collection.mutable +/** + * Represents the base URI for a voyager index, either on a local or a remote file system. For + * remote file systems, the `path` should be in the form 'scheme:////'. For local + * files, it should be in the form '//'. The `path` specified represents the directory where + * the `index.hnsw` and `names.json` are. + */ trait VoyagerUri extends Serializable { val logger = LoggerFactory.getLogger(this.getClass) val path: String @@ -99,7 +103,7 @@ private class RemoteVoyagerUri( w.close() VoyagerUri.files.foreach { f => - val tf = tempPath.resolve(f) + val tf: Path = tempPath.resolve(f) rfu.upload(Paths.get(tf.toString), new URI(path + "/" + f)) Files.delete(tf) } @@ -116,7 +120,6 @@ private[voyager] class VoyagerWriter( ef: Long = 200L, m: Long = 16L ) { - private val logger = LoggerFactory.getLogger(this.getClass) // Chunk size experiments - , // 4096, 6062: 2022-11-16 14:07:07.358 -> 2022-11-16 16:50:59.109 = 2hr 50min. 1.68s per chunk diff --git a/scio-extra/src/main/scala/com/spotify/scio/extra/voyager/package.scala b/scio-extra/src/main/scala/com/spotify/scio/extra/voyager/package.scala index 459745fe04..3e08e25b66 100644 --- a/scio-extra/src/main/scala/com/spotify/scio/extra/voyager/package.scala +++ b/scio-extra/src/main/scala/com/spotify/scio/extra/voyager/package.scala @@ -18,9 +18,9 @@ package com.spotify.scio.extra import com.spotify.scio.ScioContext import com.spotify.scio.annotations.experimental -import com.spotify.scio.values.{DistCache, SCollection, SideInput} +import com.spotify.scio.values.{SCollection, SideInput} import com.spotify.voyager.jni.Index.{SpaceType, StorageDataType} -import com.spotify.voyager.jni.{Index, StringIndex} +import com.spotify.voyager.jni.StringIndex import org.apache.beam.sdk.transforms.{DoFn, View} import org.apache.beam.sdk.values.PCollectionView import org.slf4j.LoggerFactory @@ -28,6 +28,7 @@ import org.slf4j.LoggerFactory import java.util.UUID import scala.collection.mutable +/** Main package for Voyager side input APIs. Import all. */ package object voyager { sealed abstract class VoyagerDistanceMeasure case object Euclidean extends VoyagerDistanceMeasure @@ -44,6 +45,22 @@ package object voyager { val VOYAGER_URI_MAP: mutable.Map[VoyagerUri, VoyagerReader] = mutable.HashMap.empty + /** + * Voyager reader class for nearest neighbor lookups. Supports looking up neighbors for a vector + * and returning the string labels and distances associated. + * + * @param indexFileName + * The path to the `index.hnsw` local or remote file. + * @param namesFileName + * The path to the `names.json` local or remote file. + * @param distanceMeasure + * The measurement for computing distance between entities. One of Euclidean, Cosine or Dot + * (inner product). + * @param storageType + * The Storage type of the vectors at rest. One of Float8, Float32 or E4M3. + * @param dim + * Number of dimensions in vectors. + */ class VoyagerReader private[voyager] ( indexFileName: String, namesFileName: String, @@ -68,22 +85,40 @@ package object voyager { StringIndex.load(indexFileName, namesFileName, spaceType, dim, storageDataType) } + /** + * Gets maxNumResults nearest neighbors for vector v using ef (where ef is the size of the + * dynamic list for the nearest neighbors during search). + */ def getNearest(v: Array[Float], maxNumResults: Int, ef: Int): Array[VoyagerResult] = { val queryResults = index.query(v, maxNumResults, ef) queryResults.getNames .zip(queryResults.getDistances) .map { case (name, distance) => - VoyagerResult(name, distance.toFloat) + VoyagerResult(name, distance) } } } - /** - * To be used with with side inputs - * - * @param self - */ + /** Enhanced version of [[ScioContext]] with Voyager methods */ implicit class VoyagerScioContext(private val self: ScioContext) extends AnyVal { + + /** + * Creates a SideInput of [[VoyagerReader]] from an [[VoyagerUri]] base path. To be used with + * [[com.spotify.scio.values.SCollection.withSideInputs SCollection.withSideInputs]] + * + * @param path + * The directory path to be used for the [[VoyagerUri]]. + * @param distanceMeasure + * The measurement for computing distance between entities. One of Euclidean, Cosine or Dot + * (inner product). + * @param storageType + * The Storage type of the vectors at rest. One of Float8, Float32 or E4M3. + * @param dim + * Number of dimensions in vectors. + * @return + * A [[SideInput]] of the [[VoyagerReader]] to be used for querying. + */ + @experimental def voyagerSideInput( path: String, distanceMeasure: VoyagerDistanceMeasure, @@ -100,11 +135,33 @@ package object voyager { @transient private val self: SCollection[(String, Array[Float])] ) extends AnyVal { + /** + * Write the key-value pairs of this SCollection as a Voyager index to a specified location + * using the parameters specified. + * + * @param path + * The directory path to be used for the [[VoyagerUri]]. + * @param distanceMeasure + * The measurement for computing distance between entities. One of Euclidean, Cosine or Dot + * (inner product). + * @param storageType + * The Storage type of the vectors at rest. One of Float8, Float32 or E4M3. + * @param dim + * Number of dimensions in vectors. + * @param ef + * The size of the dynamic list of neighbors used during construction time. This parameter + * controls query time/accuracy tradeoff. More information can be found in the hnswlib + * documentation https://github.com/nmslib/hnswlib. + * @param m + * The number of outgoing connections in the graph. + * @return + * A [[VoyagerUri]] representing where the index was written to. + */ @experimental def asVoyager( path: String, - voyagerDistanceMeasure: VoyagerDistanceMeasure, - voyagerStorageType: VoyagerStorageType, + distanceMeasure: VoyagerDistanceMeasure, + storageType: VoyagerStorageType, dim: Int, ef: Long, m: Long @@ -117,7 +174,7 @@ package object voyager { in.groupBy(_ => ()) .map { case (_, xs) => val voyagerWriter: VoyagerWriter = - new VoyagerWriter(voyagerDistanceMeasure, voyagerStorageType, dim, ef, m) + new VoyagerWriter(distanceMeasure, storageType, dim, ef, m) voyagerWriter.write(xs) uri.saveAndClose(voyagerWriter) @@ -127,6 +184,26 @@ package object voyager { } } + /** + * Write the key-value pairs of this SCollection as a Voyager index to a temporary location and + * building the index using the parameters specified. + * + * @param distanceMeasure + * The measurement for computing distance between entities. One of Euclidean, Cosine or Dot + * (inner product). + * @param storageType + * The Storage type of the vectors at rest. One of Float8, Float32 or E4M3. + * @param dim + * Number of dimensions in vectors. + * @param ef + * The size of the dynamic list of neighbors used during construction time. This parameter + * controls query time/accuracy tradeoff. More information can be found in the hnswlib + * documentation https://github.com/nmslib/hnswlib. + * @param m + * The number of outgoing connections in the graph. + * @return + * A [[VoyagerUri]] representing where the index was written to. + */ @experimental def asVoyager( distanceMeasure: VoyagerDistanceMeasure, @@ -142,6 +219,28 @@ package object voyager { this.asVoyager(path, distanceMeasure, storageType, dim, ef, m) } + /** + * Write the key-value pairs of this SCollection as a Voyager index to a temporary location, + * building the index using the parameters specified and then loading the reader into a side + * input. + * + * @param distanceMeasure + * The measurement for computing distance between entities. One of Euclidean, Cosine or Dot + * (inner product). + * @param storageType + * The Storage type of the vectors at rest. One of Float8, Float32 or E4M3. + * @param dim + * Number of dimensions in vectors. + * @param ef + * The size of the dynamic list of neighbors used during construction time. This parameter + * controls query time/accuracy tradeoff. More information can be found in the hnswlib + * documentation https://github.com/nmslib/hnswlib. + * @param m + * The number of outgoing connections in the graph. + * @return + * A SideInput with a [[VoyagerReader]] + */ + @experimental def asVoyagerSideInput( distanceMeasure: VoyagerDistanceMeasure, storageType: VoyagerStorageType, @@ -154,9 +253,26 @@ package object voyager { .asVoyagerSideInput(distanceMeasure, storageType, dim) } + /** + * Enhanced version of [[com.spotify.scio.values.SCollection SCollection]] with Voyager methods. + */ implicit class AnnoySCollection(@transient private val self: SCollection[VoyagerUri]) extends AnyVal { + /** + * Load the Voyager index stored at [[VoyagerUri]] in this + * [[com.spotify.scio.values.SCollection SCollection]]. + * + * @param distanceMeasure + * The measurement for computing distance between entities. One of Euclidean, Cosine or Dot + * (inner product). + * @param storageType + * The Storage type of the vectors at rest. One of Float8, Float32 or E4M3. + * @param dim + * Number of dimensions in vectors. + * @return + * SideInput[VoyagerReader] + */ @experimental def asVoyagerSideInput( distanceMeasure: VoyagerDistanceMeasure, @@ -168,6 +284,10 @@ package object voyager { } } + /** + * Construction for a VoyagerSide input that leverages a synchronized map to ensure that the + * reader is only loaded once per [[VoyagerUri]]. + */ private class VoyagerSideInput( val view: PCollectionView[VoyagerUri], distanceMeasure: VoyagerDistanceMeasure, From 2975d763d8ac71ee76e55d0225943f7886a8c133 Mon Sep 17 00:00:00 2001 From: Patrick McGee Date: Tue, 12 Sep 2023 15:44:38 -0400 Subject: [PATCH 12/28] bump voyager version --- build.sbt | 21 ++++++++++----------- 1 file changed, 10 insertions(+), 11 deletions(-) diff --git a/build.sbt b/build.sbt index 5ae6d70e66..ad69e48401 100644 --- a/build.sbt +++ b/build.sbt @@ -15,16 +15,15 @@ * under the License. */ -import sbt._ -import Keys._ -import explicitdeps.ExplicitDepsPlugin.autoImport.moduleFilterRemoveValue -import sbtassembly.AssemblyPlugin.autoImport._ -import com.github.sbt.git.SbtGit.GitKeys.gitRemoteRepo -import com.typesafe.tools.mima.core._ -import org.scalafmt.sbt.ScalafmtPlugin.scalafmtConfigSettings +import _root_.io.github.davidgregory084.DevMode import bloop.integrations.sbt.BloopDefaults +import com.github.sbt.git.SbtGit.GitKeys.gitRemoteRepo import de.heikoseeberger.sbtheader.CommentCreator -import _root_.io.github.davidgregory084.DevMode +import explicitdeps.ExplicitDepsPlugin.autoImport.moduleFilterRemoveValue +import org.scalafmt.sbt.ScalafmtPlugin.scalafmtConfigSettings +import sbt.* +import sbt.Keys.* +import sbtassembly.AssemblyPlugin.autoImport.* ThisBuild / turbo := true @@ -138,7 +137,7 @@ val shapelessVersion = "2.3.10" val sparkeyVersion = "3.2.5" val tensorFlowVersion = "0.4.2" val testContainersVersion = "0.41.0" -val voyagerVersion = "1.2.0" +val voyagerVersion = "1.2.6" val zoltarVersion = "0.6.0" // dependent versions val scalatestplusVersion = s"$scalatestVersion.0" @@ -1242,7 +1241,7 @@ lazy val `scio-repl`: Project = project case PathList("org", "apache", "beam", "sdk", "extensions", "avro", _*) => // prefer beam avro classes from extensions lib instead of ones shipped in runners CustomMergeStrategy("BeamAvro") { conflicts => - import sbtassembly.Assembly._ + import sbtassembly.Assembly.* conflicts.collectFirst { case Library(ModuleCoordinate(_, "beam-sdks-java-extensions-avro", _), _, t, s) => JarEntry(t, s) @@ -1254,7 +1253,7 @@ lazy val `scio-repl`: Project = project case PathList("org", "checkerframework", _*) => // prefer checker-qual classes packaged in checkerframework libs CustomMergeStrategy("CheckerQual") { conflicts => - import sbtassembly.Assembly._ + import sbtassembly.Assembly.* conflicts.collectFirst { case Library(ModuleCoordinate("org.checkerframework", _, _), _, t, s) => JarEntry(t, s) From 6a965fa6febfe8386a70ef185542acff11a2b328 Mon Sep 17 00:00:00 2001 From: Patrick McGee Date: Tue, 12 Sep 2023 15:48:04 -0400 Subject: [PATCH 13/28] fix formatting issues --- build.sbt | 19 ++++++++++--------- 1 file changed, 10 insertions(+), 9 deletions(-) diff --git a/build.sbt b/build.sbt index ad69e48401..9e6080e553 100644 --- a/build.sbt +++ b/build.sbt @@ -15,15 +15,16 @@ * under the License. */ -import _root_.io.github.davidgregory084.DevMode -import bloop.integrations.sbt.BloopDefaults -import com.github.sbt.git.SbtGit.GitKeys.gitRemoteRepo -import de.heikoseeberger.sbtheader.CommentCreator +import sbt._ +import Keys._ import explicitdeps.ExplicitDepsPlugin.autoImport.moduleFilterRemoveValue +import sbtassembly.AssemblyPlugin.autoImport._ +import com.github.sbt.git.SbtGit.GitKeys.gitRemoteRepo +import com.typesafe.tools.mima.core._ import org.scalafmt.sbt.ScalafmtPlugin.scalafmtConfigSettings -import sbt.* -import sbt.Keys.* -import sbtassembly.AssemblyPlugin.autoImport.* +import bloop.integrations.sbt.BloopDefaults +import de.heikoseeberger.sbtheader.CommentCreator +import _root_.io.github.davidgregory084.DevMode ThisBuild / turbo := true @@ -1241,7 +1242,7 @@ lazy val `scio-repl`: Project = project case PathList("org", "apache", "beam", "sdk", "extensions", "avro", _*) => // prefer beam avro classes from extensions lib instead of ones shipped in runners CustomMergeStrategy("BeamAvro") { conflicts => - import sbtassembly.Assembly.* + import sbtassembly.Assembly._ conflicts.collectFirst { case Library(ModuleCoordinate(_, "beam-sdks-java-extensions-avro", _), _, t, s) => JarEntry(t, s) @@ -1253,7 +1254,7 @@ lazy val `scio-repl`: Project = project case PathList("org", "checkerframework", _*) => // prefer checker-qual classes packaged in checkerframework libs CustomMergeStrategy("CheckerQual") { conflicts => - import sbtassembly.Assembly.* + import sbtassembly.Assembly._ conflicts.collectFirst { case Library(ModuleCoordinate("org.checkerframework", _, _), _, t, s) => JarEntry(t, s) From 95b66145d642d814ebda65ed906e9b66c6d5d5c7 Mon Sep 17 00:00:00 2001 From: Patrick McGee Date: Wed, 13 Sep 2023 11:28:06 -0400 Subject: [PATCH 14/28] Address feedback --- .../scio/extra/voyager/VoyagerIT.scala | 5 +- .../scio/extra/voyager/VoyagerUri.scala | 50 +++-------- .../spotify/scio/extra/voyager/package.scala | 90 ++++++++----------- .../scio/extra/voyager/VoyagerTest.scala | 18 ++-- 4 files changed, 65 insertions(+), 98 deletions(-) diff --git a/scio-extra/src/it/scala/com/spotify/scio/extra/voyager/VoyagerIT.scala b/scio-extra/src/it/scala/com/spotify/scio/extra/voyager/VoyagerIT.scala index 1679425bfc..23a9de83e7 100644 --- a/scio-extra/src/it/scala/com/spotify/scio/extra/voyager/VoyagerIT.scala +++ b/scio-extra/src/it/scala/com/spotify/scio/extra/voyager/VoyagerIT.scala @@ -19,6 +19,7 @@ package com.spotify.scio.extra.voyager import com.spotify.scio.testing.PipelineSpec import com.spotify.scio.testing.util.ItUtils import com.spotify.scio.values.SideInput +import com.spotify.voyager.jni.Index.{SpaceType, StorageDataType} import org.apache.beam.sdk.io.FileSystems import org.apache.beam.sdk.util.MimeTypes @@ -27,8 +28,8 @@ import scala.jdk.CollectionConverters._ class VoyagerIT extends PipelineSpec { val dim: Int = 2 - val storageType: VoyagerStorageType = E4M3 - val distanceMeasure: VoyagerDistanceMeasure = Cosine + val storageType: StorageDataType = StorageDataType.E4M3 + val distanceMeasure: SpaceType = SpaceType.Cosine val sideData: Seq[(String, Array[Float])] = Seq(("1", Array(2.5f, 7.2f)), ("2", Array(1.2f, 2.2f)), ("3", Array(5.6f, 3.4f))) diff --git a/scio-extra/src/main/scala/com/spotify/scio/extra/voyager/VoyagerUri.scala b/scio-extra/src/main/scala/com/spotify/scio/extra/voyager/VoyagerUri.scala index 8606c50227..3e2b8185a9 100644 --- a/scio-extra/src/main/scala/com/spotify/scio/extra/voyager/VoyagerUri.scala +++ b/scio-extra/src/main/scala/com/spotify/scio/extra/voyager/VoyagerUri.scala @@ -14,6 +14,7 @@ * specific language governing permissions and limitations * under the License. */ + package com.spotify.scio.extra.voyager import com.spotify.scio.coders.Coder @@ -21,7 +22,6 @@ import com.spotify.scio.util.{RemoteFileUtil, ScioUtil} import com.spotify.voyager.jni.Index import com.spotify.voyager.jni.Index.{SpaceType, StorageDataType} import org.apache.beam.sdk.options.PipelineOptions -import org.slf4j.LoggerFactory import java.io.File import java.net.URI @@ -36,11 +36,10 @@ import scala.collection.mutable * the `index.hnsw` and `names.json` are. */ trait VoyagerUri extends Serializable { - val logger = LoggerFactory.getLogger(this.getClass) - val path: String + def path: String private[voyager] def getReader( - distanceMeasure: VoyagerDistanceMeasure, - storageType: VoyagerStorageType, + distanceMeasure: SpaceType, + storageDataType: StorageDataType, dim: Int ): VoyagerReader private[voyager] def saveAndClose(voyagerWriter: VoyagerWriter): Unit @@ -62,8 +61,8 @@ private[voyager] object VoyagerUri { private class LocalVoyagerUri(val path: String) extends VoyagerUri { override private[voyager] def getReader( - distanceMeasure: VoyagerDistanceMeasure, - storageType: VoyagerStorageType, + distanceMeasure: SpaceType, + storageType: StorageDataType, dim: Int ): VoyagerReader = { @@ -87,8 +86,8 @@ private class RemoteVoyagerUri( ) extends VoyagerUri { private[this] val rfu: RemoteFileUtil = RemoteFileUtil.create(options) override private[voyager] def getReader( - distanceMeasure: VoyagerDistanceMeasure, - storageType: VoyagerStorageType, + distanceMeasure: SpaceType, + storageType: StorageDataType, dim: Int ): VoyagerReader = { val indexFileName: String = rfu.download(new URI(path + "/index.hnsw")).toString @@ -114,39 +113,18 @@ private class RemoteVoyagerUri( } private[voyager] class VoyagerWriter( - distanceMeasure: VoyagerDistanceMeasure, - storageType: VoyagerStorageType, + spaceType: SpaceType, + storageDataType: StorageDataType, dim: Int, ef: Long = 200L, m: Long = 16L ) { - - // Chunk size experiments - , - // 4096, 6062: 2022-11-16 14:07:07.358 -> 2022-11-16 16:50:59.109 = 2hr 50min. 1.68s per chunk - // 32786, 758: 2022-11-16 15:37:11.374 -> 2022-11-16 16:50:29.396 = 1hr 13min. 5.77s per chunk - // 131072, 190: 2022-11-17 13:38:08.421 -> 2022-11-17 15:42:39.929 = 2hr 6min. 39.79s per chunk - private val chunkSize = 32786 // 2^15 - private val randomSeed = 1L private[this] val namesOutput = mutable.ListBuffer.empty[String] - private[this] val index: Index = { - val spaceType = distanceMeasure match { - case Euclidean => SpaceType.Euclidean - case Cosine => SpaceType.Cosine - case Dot => SpaceType.InnerProduct - } - - val storageDataType = storageType match { - case Float8 => StorageDataType.Float8 - case Float32 => StorageDataType.Float32 - case E4M3 => StorageDataType.E4M3 - } - new Index(spaceType, dim, m, ef, randomSeed, chunkSize, storageDataType) - } + private[this] val index: Index = + new Index(spaceType, dim, m, ef, RANDOM_SEED, CHUNK_SIZE, storageDataType) def write(vectors: Iterable[(String, Array[Float])]): Unit = { - var batchNum = 1 - val nameVectorIndexIterator = vectors.iterator.zipWithIndex .map { case ((name, vector), idx) => (name, vector, idx.longValue()) @@ -154,13 +132,11 @@ private[voyager] class VoyagerWriter( while (nameVectorIndexIterator.hasNext) { val (nameArray, vectorArray, indexArray) = nameVectorIndexIterator - .take(chunkSize) + .take(CHUNK_SIZE) .toArray .unzip3 index.addItems(vectorArray, indexArray, -1) - - batchNum += 1 namesOutput ++= nameArray } diff --git a/scio-extra/src/main/scala/com/spotify/scio/extra/voyager/package.scala b/scio-extra/src/main/scala/com/spotify/scio/extra/voyager/package.scala index 3e08e25b66..3f2c8a1c08 100644 --- a/scio-extra/src/main/scala/com/spotify/scio/extra/voyager/package.scala +++ b/scio-extra/src/main/scala/com/spotify/scio/extra/voyager/package.scala @@ -14,6 +14,7 @@ * specific language governing permissions and limitations * under the License. */ + package com.spotify.scio.extra import com.spotify.scio.ScioContext @@ -30,20 +31,13 @@ import scala.collection.mutable /** Main package for Voyager side input APIs. Import all. */ package object voyager { - sealed abstract class VoyagerDistanceMeasure - case object Euclidean extends VoyagerDistanceMeasure - case object Cosine extends VoyagerDistanceMeasure - case object Dot extends VoyagerDistanceMeasure - - sealed abstract class VoyagerStorageType - case object Float8 extends VoyagerStorageType - case object Float32 extends VoyagerStorageType - case object E4M3 extends VoyagerStorageType - private val logger = LoggerFactory.getLogger(this.getClass) + @transient lazy val logger = LoggerFactory.getLogger(this.getClass) case class VoyagerResult(value: String, distance: Float) - val VOYAGER_URI_MAP: mutable.Map[VoyagerUri, VoyagerReader] = mutable.HashMap.empty + private val VOYAGER_URI_MAP: mutable.Map[VoyagerUri, VoyagerReader] = mutable.HashMap.empty + private[voyager] val RANDOM_SEED: Long = 1L + private[voyager] val CHUNK_SIZE: Int = 32786 // 2^15 /** * Voyager reader class for nearest neighbor lookups. Supports looking up neighbors for a vector @@ -53,10 +47,10 @@ package object voyager { * The path to the `index.hnsw` local or remote file. * @param namesFileName * The path to the `names.json` local or remote file. - * @param distanceMeasure + * @param spaceType * The measurement for computing distance between entities. One of Euclidean, Cosine or Dot * (inner product). - * @param storageType + * @param storageDataType * The Storage type of the vectors at rest. One of Float8, Float32 or E4M3. * @param dim * Number of dimensions in vectors. @@ -64,26 +58,14 @@ package object voyager { class VoyagerReader private[voyager] ( indexFileName: String, namesFileName: String, - distanceMeasure: VoyagerDistanceMeasure, - storageType: VoyagerStorageType, + spaceType: SpaceType, + storageDataType: StorageDataType, dim: Int ) { require(dim > 0, "Vector dimension should be > 0") - private val index: StringIndex = { - val spaceType = distanceMeasure match { - case Euclidean => SpaceType.Euclidean - case Cosine => SpaceType.Cosine - case Dot => SpaceType.InnerProduct - } - - val storageDataType = storageType match { - case Float8 => StorageDataType.Float8 - case Float32 => StorageDataType.Float32 - case E4M3 => StorageDataType.E4M3 - } + private val index: StringIndex = StringIndex.load(indexFileName, namesFileName, spaceType, dim, storageDataType) - } /** * Gets maxNumResults nearest neighbors for vector v using ef (where ef is the size of the @@ -108,10 +90,10 @@ package object voyager { * * @param path * The directory path to be used for the [[VoyagerUri]]. - * @param distanceMeasure + * @param spaceType * The measurement for computing distance between entities. One of Euclidean, Cosine or Dot * (inner product). - * @param storageType + * @param storageDataType * The Storage type of the vectors at rest. One of Float8, Float32 or E4M3. * @param dim * Number of dimensions in vectors. @@ -121,13 +103,13 @@ package object voyager { @experimental def voyagerSideInput( path: String, - distanceMeasure: VoyagerDistanceMeasure, - storageType: VoyagerStorageType, + spaceType: SpaceType, + storageDataType: StorageDataType, dim: Int ): SideInput[VoyagerReader] = { val uri = VoyagerUri(path, self.options) val view = self.parallelize(Seq(uri)).applyInternal(View.asSingleton()) - new VoyagerSideInput(view, distanceMeasure, storageType, dim) + new VoyagerSideInput(view, spaceType, storageDataType, dim) } } @@ -141,11 +123,11 @@ package object voyager { * * @param path * The directory path to be used for the [[VoyagerUri]]. - * @param distanceMeasure + * @param spaceType * The measurement for computing distance between entities. One of Euclidean, Cosine or Dot * (inner product). - * @param storageType - * The Storage type of the vectors at rest. One of Float8, Float32 or E4M3. + * @param storageDataType + * The storage data type of the vectors at rest. One of Float8, Float32 or E4M3. * @param dim * Number of dimensions in vectors. * @param ef @@ -160,8 +142,8 @@ package object voyager { @experimental def asVoyager( path: String, - distanceMeasure: VoyagerDistanceMeasure, - storageType: VoyagerStorageType, + spaceType: SpaceType, + storageDataType: StorageDataType, dim: Int, ef: Long, m: Long @@ -174,7 +156,7 @@ package object voyager { in.groupBy(_ => ()) .map { case (_, xs) => val voyagerWriter: VoyagerWriter = - new VoyagerWriter(distanceMeasure, storageType, dim, ef, m) + new VoyagerWriter(spaceType, storageDataType, dim, ef, m) voyagerWriter.write(xs) uri.saveAndClose(voyagerWriter) @@ -191,7 +173,7 @@ package object voyager { * @param distanceMeasure * The measurement for computing distance between entities. One of Euclidean, Cosine or Dot * (inner product). - * @param storageType + * @param storageDataType * The Storage type of the vectors at rest. One of Float8, Float32 or E4M3. * @param dim * Number of dimensions in vectors. @@ -206,8 +188,8 @@ package object voyager { */ @experimental def asVoyager( - distanceMeasure: VoyagerDistanceMeasure, - storageType: VoyagerStorageType, + distanceMeasure: SpaceType, + storageDataType: StorageDataType, dim: Int, ef: Long = 200L, m: Long = 16L @@ -216,7 +198,7 @@ package object voyager { val tempLocation: String = self.context.options.getTempLocation require(tempLocation != null, s"--tempLocation arg is required") val path = s"$tempLocation/voyager-build-$uuid" - this.asVoyager(path, distanceMeasure, storageType, dim, ef, m) + this.asVoyager(path, distanceMeasure, storageDataType, dim, ef, m) } /** @@ -224,7 +206,7 @@ package object voyager { * building the index using the parameters specified and then loading the reader into a side * input. * - * @param distanceMeasure + * @param spaceType * The measurement for computing distance between entities. One of Euclidean, Cosine or Dot * (inner product). * @param storageType @@ -242,15 +224,15 @@ package object voyager { */ @experimental def asVoyagerSideInput( - distanceMeasure: VoyagerDistanceMeasure, - storageType: VoyagerStorageType, + spaceType: SpaceType, + storageType: StorageDataType, dim: Int, ef: Long = 200L, m: Long = 16L ): SideInput[VoyagerReader] = self - .asVoyager(distanceMeasure, storageType, dim, ef, m) - .asVoyagerSideInput(distanceMeasure, storageType, dim) + .asVoyager(spaceType, storageType, dim, ef, m) + .asVoyagerSideInput(spaceType, storageType, dim) } /** @@ -263,7 +245,7 @@ package object voyager { * Load the Voyager index stored at [[VoyagerUri]] in this * [[com.spotify.scio.values.SCollection SCollection]]. * - * @param distanceMeasure + * @param spaceType * The measurement for computing distance between entities. One of Euclidean, Cosine or Dot * (inner product). * @param storageType @@ -275,12 +257,12 @@ package object voyager { */ @experimental def asVoyagerSideInput( - distanceMeasure: VoyagerDistanceMeasure, - storageType: VoyagerStorageType, + spaceType: SpaceType, + storageType: StorageDataType, dim: Int ): SideInput[VoyagerReader] = { val view = self.applyInternal(View.asSingleton()) - new VoyagerSideInput(view, distanceMeasure, storageType, dim) + new VoyagerSideInput(view, spaceType, storageType, dim) } } @@ -290,8 +272,8 @@ package object voyager { */ private class VoyagerSideInput( val view: PCollectionView[VoyagerUri], - distanceMeasure: VoyagerDistanceMeasure, - storageType: VoyagerStorageType, + distanceMeasure: SpaceType, + storageType: StorageDataType, dim: Int ) extends SideInput[VoyagerReader] { override def get[I, O](context: DoFn[I, O]#ProcessContext): VoyagerReader = { diff --git a/scio-extra/src/test/scala/com/spotify/scio/extra/voyager/VoyagerTest.scala b/scio-extra/src/test/scala/com/spotify/scio/extra/voyager/VoyagerTest.scala index 6ad4eaddbc..a46b9de586 100644 --- a/scio-extra/src/test/scala/com/spotify/scio/extra/voyager/VoyagerTest.scala +++ b/scio-extra/src/test/scala/com/spotify/scio/extra/voyager/VoyagerTest.scala @@ -14,6 +14,7 @@ * specific language governing permissions and limitations * under the License. */ + package com.spotify.scio.extra.voyager import com.spotify.scio.{ScioContext, ScioResult} @@ -26,9 +27,9 @@ import java.io.File import java.nio.file.Files class VoyagerTest extends PipelineSpec { - val distanceMeasure = Cosine - val storageType = E4M3 - val dim = 2 + val spaceType: SpaceType = SpaceType.Cosine + val storageDataType: StorageDataType = StorageDataType.E4M3 + val dim: Int = 2 val sideData: Seq[(String, Array[Float])] = Seq(("1", Array(2.5f, 7.2f)), ("2", Array(1.2f, 2.2f)), ("3", Array(5.6f, 3.4f))) @@ -39,7 +40,7 @@ class VoyagerTest extends PipelineSpec { val sc = ScioContext() val p: ClosedTap[VoyagerUri] = sc.parallelize(sideData) - .asVoyager(basePath, distanceMeasure, storageType, dim, 200L, 16L) + .asVoyager(basePath, spaceType, storageDataType, dim, 200L, 16L) .materialize val scioResult: ScioResult = sc.run().waitUntilFinish() @@ -75,7 +76,14 @@ class VoyagerTest extends PipelineSpec { the[IllegalArgumentException] thrownBy { runWithContext { - _.parallelize(sideData).asVoyager(tmpDir.toString, Cosine, E4M3, dim, 200L, 16L) + _.parallelize(sideData).asVoyager( + tmpDir.toString, + spaceType, + storageDataType, + dim, + 200L, + 16L + ) } } should have message s"requirement failed: Voyager URI $tmpDir already exists" From 3d277cd07db3fa0a4813a82c3f81cf48080b8764 Mon Sep 17 00:00:00 2001 From: Patrick McGee Date: Mon, 18 Sep 2023 14:57:14 -0400 Subject: [PATCH 15/28] add in more docs --- site/src/main/paradox/extras/Voyager.md | 74 +++++++++++++++++++++++++ 1 file changed, 74 insertions(+) create mode 100644 site/src/main/paradox/extras/Voyager.md diff --git a/site/src/main/paradox/extras/Voyager.md b/site/src/main/paradox/extras/Voyager.md new file mode 100644 index 0000000000..4a962c7c31 --- /dev/null +++ b/site/src/main/paradox/extras/Voyager.md @@ -0,0 +1,74 @@ +# Voyager + +Scio supports Spotify's [Voyager](https://github.com/spotify/voyager), which provides an easy to use API on top of `hnswlib` that that +works in python and java. + +## Write + +A keyed `SCollection` with `String` keys and `Array[Float]` vector values can be saved with @scaladoc[asVoyager](com.spotify.scio.extra.voyager.VoyagerPairSCollection#asVoyager(path:String,spaceType:com.spotify.voyager.jni.Index.SpaceType,storageDataType:com.spotify.voyager.jni.Index.StorageDataType,dim:Int,ef:Long,m:Long)): + +```scala +import com.spotify.scio.values.SCollection +import com.spotify.scio.extra.voyager._ +import com.spotify.voyager.jni.Index.{SpaceType, StorageDataType} + +val dim: Int = ??? +val ef: Long = ??? +val m: Int = ??? +val storageType: StorageDataType = ??? +val spaceType: SpaceType = ??? +val itemVectors: SCollection[(String, Array[Float])] = ??? +itemVectors.asVoyager("gs://output-path", spaceType, storageType, dim, ef, m) +``` + +## Side Input + +A Voyager index can be read directly as a `SideInput` with @scaladoc[voyagerSideInput](com.spotify.scio.extra.voyager.VoyagerScioContext#voyagerSideInput(com.spotify.voyager.jni.Index.SpaceType,storageDataType:com.spotify.voyager.jni.Index.StorageDataType,dim:Int)): + +```scala +import com.spotify.scio._ +import com.spotify.scio.values.SCollection +import com.spotify.scio.extra.voyager._ +import com.spotify.voyager.jni.Index.{SpaceType, StorageDataType} + +val sc: ScioContext = ??? + +val dim: Int = ??? +val storageType: StorageDataType = ??? +val spaceType: SpaceType = ??? +val itemVectors: SCollection[(String, Array[Float])] = ??? +val voyagerSI: SideInput[VoyagerReader] = sc.voyagerSideInput("gs://input-path", spaceType, storageType, dim) +``` + +Alternatively, an `SCollection` can be converted directly to a `SideInput` with @scaladoc[asVoyagerSideInput](com.spotify.scio.extra.voyager.VoyagerPairSCollection#asVoyagerSideInput(spaceType:com.spotify.voyager.jni.Index.SpaceType,storageDataType:com.spotify.voyager.jni.Index.StorageDataType,dim:Int,ef:Long,m:Long)): +```scala +import com.spotify.scio.values.SCollection +import com.spotify.scio.extra.voyager._ +import com.spotify.voyager.jni.Index.{SpaceType, StorageDataType} + +val dim: Int = ??? +val ef: Long = ??? +val m: Int = ??? +val storageType: StorageDataType = ??? +val spaceType: SpaceType = ??? +val itemVectors: SCollection[(String, Array[Float])] = ??? +val voyagerSI: SideInput[VoyagerReader] = itemVectors.asVoyagerSideInput(spaceType, storageType, dim, ef, m) +``` + +An @scaladoc[VoyagerReader](com.spotify.scio.extra.voyager.VoyagerReader) provides access to querying the Voyager index to get their nearest neighbors. +```scala +import com.spotify.scio.values.{SCollection, SideInput} +import com.spotify.scio.extra.voyager._ + +val voyagerSI: SideInput[VoyagerReader] = ??? +val elements: SCollection[(String, Array[Float])] = ??? +val maxNumResults: Int = ??? +val ef: Int = ??? + +val queryResults: SCollection[(String, Array[VoyagerResult])] = elements + .withSideInputs(voyagerSI) + .map { case ((label, vector), ctx) => + val voyagerReader: VoyagerReader = ctx(voyagerSI) + (label, voyagerReader.getNearest(vector, maxNumResults, ef)) + } +``` From d14ad207d85f139270c7df0acc1c1cee3c7a4668 Mon Sep 17 00:00:00 2001 From: Patrick McGee Date: Tue, 26 Sep 2023 10:50:04 -0400 Subject: [PATCH 16/28] coder updates --- .../scio/extra/voyager/VoyagerUri.scala | 30 +++++++++++-------- .../scio/extra/voyager/VoyagerTest.scala | 10 +++++++ 2 files changed, 27 insertions(+), 13 deletions(-) diff --git a/scio-extra/src/main/scala/com/spotify/scio/extra/voyager/VoyagerUri.scala b/scio-extra/src/main/scala/com/spotify/scio/extra/voyager/VoyagerUri.scala index 3e2b8185a9..0194a8beac 100644 --- a/scio-extra/src/main/scala/com/spotify/scio/extra/voyager/VoyagerUri.scala +++ b/scio-extra/src/main/scala/com/spotify/scio/extra/voyager/VoyagerUri.scala @@ -35,7 +35,7 @@ import scala.collection.mutable * files, it should be in the form '//'. The `path` specified represents the directory where * the `index.hnsw` and `names.json` are. */ -trait VoyagerUri extends Serializable { +sealed trait VoyagerUri { def path: String private[voyager] def getReader( distanceMeasure: SpaceType, @@ -49,17 +49,17 @@ trait VoyagerUri extends Serializable { private[voyager] object VoyagerUri { def apply(path: String, opts: PipelineOptions): VoyagerUri = { if (ScioUtil.isLocalUri(new URI(path))) { - new LocalVoyagerUri(path) + LocalVoyagerUri(path) } else { - new RemoteVoyagerUri(path, opts) + val rfu: RemoteFileUtil = RemoteFileUtil.create(opts) + RemoteVoyagerUri(path, rfu) } } def files: Seq[String] = Seq("index.hnsw", "names.json") - implicit val voyagerUriCoder: Coder[VoyagerUri] = Coder.kryo[VoyagerUri] } -private class LocalVoyagerUri(val path: String) extends VoyagerUri { +case class LocalVoyagerUri(path: String) extends VoyagerUri { override private[voyager] def getReader( distanceMeasure: SpaceType, storageType: StorageDataType, @@ -80,18 +80,17 @@ private class LocalVoyagerUri(val path: String) extends VoyagerUri { VoyagerUri.files.exists(f => new File(path + "/" + f).exists()) } -private class RemoteVoyagerUri( - val path: String, - options: PipelineOptions +case class RemoteVoyagerUri( + path: String, + remoteFileUtil: RemoteFileUtil ) extends VoyagerUri { - private[this] val rfu: RemoteFileUtil = RemoteFileUtil.create(options) override private[voyager] def getReader( distanceMeasure: SpaceType, storageType: StorageDataType, dim: Int ): VoyagerReader = { - val indexFileName: String = rfu.download(new URI(path + "/index.hnsw")).toString - val namesFileName: String = rfu.download(new URI(path + "/names.json")).toString + val indexFileName: String = remoteFileUtil.download(new URI(path + "/index.hnsw")).toString + val namesFileName: String = remoteFileUtil.download(new URI(path + "/names.json")).toString new VoyagerReader(indexFileName, namesFileName, distanceMeasure, storageType, dim) } @@ -103,13 +102,18 @@ private class RemoteVoyagerUri( VoyagerUri.files.foreach { f => val tf: Path = tempPath.resolve(f) - rfu.upload(Paths.get(tf.toString), new URI(path + "/" + f)) + remoteFileUtil.upload(Paths.get(tf.toString), new URI(path + "/" + f)) Files.delete(tf) } } override private[voyager] def exists: Boolean = - VoyagerUri.files.exists(f => rfu.remoteExists(new URI(path + "/" + f))) + VoyagerUri.files.exists(f => remoteFileUtil.remoteExists(new URI(path + "/" + f))) +} + +object RemoteVoyagerUri { + def apply(path: String, options: PipelineOptions): RemoteVoyagerUri = + RemoteVoyagerUri(path, RemoteFileUtil.create(options)) } private[voyager] class VoyagerWriter( diff --git a/scio-extra/src/test/scala/com/spotify/scio/extra/voyager/VoyagerTest.scala b/scio-extra/src/test/scala/com/spotify/scio/extra/voyager/VoyagerTest.scala index a46b9de586..c922bcdd33 100644 --- a/scio-extra/src/test/scala/com/spotify/scio/extra/voyager/VoyagerTest.scala +++ b/scio-extra/src/test/scala/com/spotify/scio/extra/voyager/VoyagerTest.scala @@ -19,9 +19,12 @@ package com.spotify.scio.extra.voyager import com.spotify.scio.{ScioContext, ScioResult} import com.spotify.scio.io.ClosedTap +import com.spotify.scio.testing.CoderAssertions.{notFallback, ValueShouldSyntax} import com.spotify.scio.testing.PipelineSpec +import com.spotify.scio.util.RemoteFileUtil import com.spotify.voyager.jni.Index.{SpaceType, StorageDataType} import com.spotify.voyager.jni.StringIndex +import org.apache.beam.sdk.options.PipelineOptionsFactory import java.io.File import java.nio.file.Files @@ -91,4 +94,11 @@ class VoyagerTest extends PipelineSpec { Files.delete(names) } + "VoyagerUri" should "not use Kryo" in { + val localUri: VoyagerUri = LocalVoyagerUri("gs://this-that") + localUri coderShould notFallback() + + val remoteUri: VoyagerUri = RemoteVoyagerUri("gs//this-that", PipelineOptionsFactory.create()) + remoteUri coderShould notFallback() + } } From ad91a87214a7d41f3229780260e45800108339ef Mon Sep 17 00:00:00 2001 From: Patrick McGee Date: Tue, 26 Sep 2023 15:55:21 -0400 Subject: [PATCH 17/28] Add in the update to rfu to make it work w/ magnolia --- .../java/com/spotify/scio/util/RemoteFileUtil.java | 10 ++++++++++ 1 file changed, 10 insertions(+) diff --git a/scio-core/src/main/java/com/spotify/scio/util/RemoteFileUtil.java b/scio-core/src/main/java/com/spotify/scio/util/RemoteFileUtil.java index e0cdc6f232..303d309128 100644 --- a/scio-core/src/main/java/com/spotify/scio/util/RemoteFileUtil.java +++ b/scio-core/src/main/java/com/spotify/scio/util/RemoteFileUtil.java @@ -252,4 +252,14 @@ private static void copyToRemote(Path src, URI dst, String mimeType) throws IOEx private static Metadata getMetadata(URI src) throws IOException { return FileSystems.matchSingleFileSpec(src.toString()); } + + @Override + public int hashCode() { + return this.getClass().hashCode(); + } + + @Override + public final boolean equals(Object other) { + return other != null && this.getClass().equals(other.getClass()); + } } From 3f73295135b6a22d0221e5529130a046c9b8de5b Mon Sep 17 00:00:00 2001 From: Patrick McGee Date: Wed, 27 Sep 2023 11:01:07 -0400 Subject: [PATCH 18/28] Update scio-extra/src/main/scala/com/spotify/scio/extra/voyager/package.scala Co-authored-by: Claire McGinty --- .../src/main/scala/com/spotify/scio/extra/voyager/package.scala | 1 - 1 file changed, 1 deletion(-) diff --git a/scio-extra/src/main/scala/com/spotify/scio/extra/voyager/package.scala b/scio-extra/src/main/scala/com/spotify/scio/extra/voyager/package.scala index 3f2c8a1c08..96a928acf2 100644 --- a/scio-extra/src/main/scala/com/spotify/scio/extra/voyager/package.scala +++ b/scio-extra/src/main/scala/com/spotify/scio/extra/voyager/package.scala @@ -150,7 +150,6 @@ package object voyager { ): SCollection[VoyagerUri] = { val uri: VoyagerUri = VoyagerUri(path, self.context.options) require(!uri.exists, s"Voyager URI ${uri.path} already exists") - logger.info(s"Voyager URI :${uri.path}") self.transform { in => { in.groupBy(_ => ()) From 08a56fb6a619495c69c8eed6afe14c60e4a9aeae Mon Sep 17 00:00:00 2001 From: Patrick McGee Date: Wed, 27 Sep 2023 12:12:26 -0400 Subject: [PATCH 19/28] Address more feedback --- .../scala/com/spotify/scio/extra/voyager/VoyagerUri.scala | 4 ++-- .../main/scala/com/spotify/scio/extra/voyager/package.scala | 5 +++-- 2 files changed, 5 insertions(+), 4 deletions(-) diff --git a/scio-extra/src/main/scala/com/spotify/scio/extra/voyager/VoyagerUri.scala b/scio-extra/src/main/scala/com/spotify/scio/extra/voyager/VoyagerUri.scala index 0194a8beac..50b786c4e1 100644 --- a/scio-extra/src/main/scala/com/spotify/scio/extra/voyager/VoyagerUri.scala +++ b/scio-extra/src/main/scala/com/spotify/scio/extra/voyager/VoyagerUri.scala @@ -95,14 +95,14 @@ case class RemoteVoyagerUri( } override private[voyager] def saveAndClose(w: VoyagerWriter): Unit = { - val tempPath: Path = Files.createTempDirectory("") - logger.info(s"temp path: $tempPath") + val tempPath: Path = Files.createTempDirectory("voyager-") w.save(tempPath.toString) w.close() VoyagerUri.files.foreach { f => val tf: Path = tempPath.resolve(f) remoteFileUtil.upload(Paths.get(tf.toString), new URI(path + "/" + f)) + logger.info(s"Uploaded Voyager $f file from $tempPath to $path/$f") Files.delete(tf) } } diff --git a/scio-extra/src/main/scala/com/spotify/scio/extra/voyager/package.scala b/scio-extra/src/main/scala/com/spotify/scio/extra/voyager/package.scala index 96a928acf2..74770fd694 100644 --- a/scio-extra/src/main/scala/com/spotify/scio/extra/voyager/package.scala +++ b/scio-extra/src/main/scala/com/spotify/scio/extra/voyager/package.scala @@ -26,6 +26,7 @@ import org.apache.beam.sdk.transforms.{DoFn, View} import org.apache.beam.sdk.values.PCollectionView import org.slf4j.LoggerFactory +import java.nio.file.Paths import java.util.UUID import scala.collection.mutable @@ -195,8 +196,8 @@ package object voyager { ): SCollection[VoyagerUri] = { val uuid: UUID = UUID.randomUUID() val tempLocation: String = self.context.options.getTempLocation - require(tempLocation != null, s"--tempLocation arg is required") - val path = s"$tempLocation/voyager-build-$uuid" + require(tempLocation != null, s"Voyager writes require --tempLocation to be set.") + val path = s"${tempLocation.stripSuffix("/")}/voyager-build-$uuid" this.asVoyager(path, distanceMeasure, storageDataType, dim, ef, m) } From 0694a77e4c73646fd12bcff3a942371407c67f3e Mon Sep 17 00:00:00 2001 From: Claire McGinty Date: Tue, 3 Oct 2023 17:00:20 -0400 Subject: [PATCH 20/28] Apply suggestions from code review --- .../main/scala/com/spotify/scio/extra/voyager/VoyagerUri.scala | 1 - .../src/main/scala/com/spotify/scio/extra/voyager/package.scala | 1 - 2 files changed, 2 deletions(-) diff --git a/scio-extra/src/main/scala/com/spotify/scio/extra/voyager/VoyagerUri.scala b/scio-extra/src/main/scala/com/spotify/scio/extra/voyager/VoyagerUri.scala index 50b786c4e1..1d6a6ae925 100644 --- a/scio-extra/src/main/scala/com/spotify/scio/extra/voyager/VoyagerUri.scala +++ b/scio-extra/src/main/scala/com/spotify/scio/extra/voyager/VoyagerUri.scala @@ -17,7 +17,6 @@ package com.spotify.scio.extra.voyager -import com.spotify.scio.coders.Coder import com.spotify.scio.util.{RemoteFileUtil, ScioUtil} import com.spotify.voyager.jni.Index import com.spotify.voyager.jni.Index.{SpaceType, StorageDataType} diff --git a/scio-extra/src/main/scala/com/spotify/scio/extra/voyager/package.scala b/scio-extra/src/main/scala/com/spotify/scio/extra/voyager/package.scala index 74770fd694..7d46a69770 100644 --- a/scio-extra/src/main/scala/com/spotify/scio/extra/voyager/package.scala +++ b/scio-extra/src/main/scala/com/spotify/scio/extra/voyager/package.scala @@ -26,7 +26,6 @@ import org.apache.beam.sdk.transforms.{DoFn, View} import org.apache.beam.sdk.values.PCollectionView import org.slf4j.LoggerFactory -import java.nio.file.Paths import java.util.UUID import scala.collection.mutable From 5d2b586a1ecb78acc475c3ec62a9e14d28a2156f Mon Sep 17 00:00:00 2001 From: Michel Davit Date: Fri, 6 Oct 2023 15:10:20 +0200 Subject: [PATCH 21/28] Fix integration tests --- .../scio/extra/voyager/VoyagerIT.scala | 112 ++++++++++-------- 1 file changed, 63 insertions(+), 49 deletions(-) diff --git a/scio-extra/src/it/scala/com/spotify/scio/extra/voyager/VoyagerIT.scala b/scio-extra/src/it/scala/com/spotify/scio/extra/voyager/VoyagerIT.scala index 23a9de83e7..0466dc24a1 100644 --- a/scio-extra/src/it/scala/com/spotify/scio/extra/voyager/VoyagerIT.scala +++ b/scio-extra/src/it/scala/com/spotify/scio/extra/voyager/VoyagerIT.scala @@ -20,7 +20,9 @@ import com.spotify.scio.testing.PipelineSpec import com.spotify.scio.testing.util.ItUtils import com.spotify.scio.values.SideInput import com.spotify.voyager.jni.Index.{SpaceType, StorageDataType} +import org.apache.beam.sdk.Pipeline.PipelineExecutionException import org.apache.beam.sdk.io.FileSystems +import org.apache.beam.sdk.options.PipelineOptionsFactory import org.apache.beam.sdk.util.MimeTypes import java.nio.ByteBuffer @@ -30,62 +32,74 @@ class VoyagerIT extends PipelineSpec { val dim: Int = 2 val storageType: StorageDataType = StorageDataType.E4M3 val distanceMeasure: SpaceType = SpaceType.Cosine + val ef = 200 + val m = 16L + + val sideData: Seq[(String, Array[Float])] = Seq( + "1" -> Array(2.5f, 7.2f), + "2" -> Array(1.2f, 2.2f), + "3" -> Array(5.6f, 3.4f) + ) + + FileSystems.setDefaultPipelineOptions(PipelineOptionsFactory.create()) - val sideData: Seq[(String, Array[Float])] = - Seq(("1", Array(2.5f, 7.2f)), ("2", Array(1.2f, 2.2f)), ("3", Array(5.6f, 3.4f))) it should "support .asVoyagerSideInput using GCS tempLocation" in { + val tempLocation = ItUtils.gcpTempLocation("voyager-it") runWithContext { sc => - FileSystems.setDefaultPipelineOptions(sc.options) - - val tempLocation = ItUtils.gcpTempLocation("voyager-it") - - try { - val p1 = sc.parallelize(sideData) - val p2: SideInput[VoyagerReader] = - sc.parallelize(sideData).asVoyagerSideInput(distanceMeasure, storageType, dim) - val s = p1 - .withSideInputs(p2) - .flatMap { (xs, si) => - si(p2).getNearest(xs._2, 1, 100) - } - .toSCollection - - s should containInAnyOrder(sideData.map(_._1)) - } finally { - val files = FileSystems - .`match`(s"$tempLocation") - .metadata() - .asScala - .map(_.resourceId()) - - FileSystems.delete(files.asJava) - } + sc.options.setTempLocation(tempLocation) + val (indexes, vectors) = sideData.unzip + + val voyagerReader = sc + .parallelize(sideData) + .asVoyagerSideInput(distanceMeasure, storageType, dim) + + val result = sc + .parallelize(vectors) + .withSideInputs(voyagerReader) + .flatMap { case (vector, ctx) => + ctx(voyagerReader).getNearest(vector, 1, 100).map(_.value) + } + .toSCollection + + result should containInAnyOrder(indexes) } + + // check files uploaded by voyager + val files = FileSystems + .`match`(s"$tempLocation/voyager-*") + .metadata() + .asScala + .map(_.resourceId()) + + FileSystems.delete(files.asJava) } - it should "support .asVoyagerSideInput using GCS tempLocation" in { - runWithContext { sc => - FileSystems.setDefaultPipelineOptions(sc.options) - - val tempLocation = ItUtils.gcpTempLocation("voyager-it") - val namePath = tempLocation + "/names.json" - val indexPath = tempLocation + "/index.hnsw" - val nameResourceId = FileSystems.matchNewResource(namePath, false) - val indexResourceId = FileSystems.matchNewResource(indexPath, false) - - try { - val f1 = FileSystems.create(nameResourceId, MimeTypes.BINARY) - val f2 = FileSystems.create(indexResourceId, MimeTypes.BINARY) - f1.write(ByteBuffer.wrap("test-data".getBytes())) - f1.close() - f2.write(ByteBuffer.wrap("test-data".getBytes())) - f2.close() - the[IllegalArgumentException] thrownBy { - sc.parallelize(sideData).asVoyager(distanceMeasure, storageType, dim) - } should have message s"" - } finally { - FileSystems.delete(Seq(nameResourceId, indexResourceId).asJava) + it should "throw exception when Voyager file exists" in { + val path = ItUtils.gcpTempLocation("voyager-it") + val namePath = path + "/names.json" + val indexPath = path + "/index.hnsw" + val nameResourceId = FileSystems.matchNewResource(namePath, false) + val indexResourceId = FileSystems.matchNewResource(indexPath, false) + + // write some data in the + val f1 = FileSystems.create(nameResourceId, MimeTypes.BINARY) + val f2 = FileSystems.create(indexResourceId, MimeTypes.BINARY) + try { + f1.write(ByteBuffer.wrap("test-data".getBytes())) + f2.write(ByteBuffer.wrap("test-data".getBytes())) + } finally { + f1.close() + f2.close() + } + + val e = the[IllegalArgumentException] thrownBy { + runWithContext { sc => + sc.parallelize(sideData).asVoyager(path, distanceMeasure, storageType, dim, 200L, 16) } } + + e.getMessage shouldBe s"requirement failed: Voyager URI $path already exists" + + FileSystems.delete(Seq(nameResourceId, indexResourceId).asJava) } } From 85a203ed5e3cbeac0234d33180e41484e9d0aaa4 Mon Sep 17 00:00:00 2001 From: Michel Davit Date: Fri, 6 Oct 2023 15:48:01 +0200 Subject: [PATCH 22/28] Reorganize code to respect scio structure --- .../scio/extra/voyager/VoyagerIT.scala | 2 - .../{VoyagerUri.scala => Voyager.scala} | 94 +++++- .../spotify/scio/extra/voyager/package.scala | 273 +----------------- .../scio/extra/voyager/syntax/AllSyntax.scala | 3 + .../voyager/syntax/SCollectionSyntax.scala | 168 +++++++++++ .../voyager/syntax/ScioContextSyntax.scala | 45 +++ 6 files changed, 310 insertions(+), 275 deletions(-) rename scio-extra/src/main/scala/com/spotify/scio/extra/voyager/{VoyagerUri.scala => Voyager.scala} (63%) create mode 100644 scio-extra/src/main/scala/com/spotify/scio/extra/voyager/syntax/AllSyntax.scala create mode 100644 scio-extra/src/main/scala/com/spotify/scio/extra/voyager/syntax/SCollectionSyntax.scala create mode 100644 scio-extra/src/main/scala/com/spotify/scio/extra/voyager/syntax/ScioContextSyntax.scala diff --git a/scio-extra/src/it/scala/com/spotify/scio/extra/voyager/VoyagerIT.scala b/scio-extra/src/it/scala/com/spotify/scio/extra/voyager/VoyagerIT.scala index 0466dc24a1..2a395e3916 100644 --- a/scio-extra/src/it/scala/com/spotify/scio/extra/voyager/VoyagerIT.scala +++ b/scio-extra/src/it/scala/com/spotify/scio/extra/voyager/VoyagerIT.scala @@ -18,9 +18,7 @@ package com.spotify.scio.extra.voyager import com.spotify.scio.testing.PipelineSpec import com.spotify.scio.testing.util.ItUtils -import com.spotify.scio.values.SideInput import com.spotify.voyager.jni.Index.{SpaceType, StorageDataType} -import org.apache.beam.sdk.Pipeline.PipelineExecutionException import org.apache.beam.sdk.io.FileSystems import org.apache.beam.sdk.options.PipelineOptionsFactory import org.apache.beam.sdk.util.MimeTypes diff --git a/scio-extra/src/main/scala/com/spotify/scio/extra/voyager/VoyagerUri.scala b/scio-extra/src/main/scala/com/spotify/scio/extra/voyager/Voyager.scala similarity index 63% rename from scio-extra/src/main/scala/com/spotify/scio/extra/voyager/VoyagerUri.scala rename to scio-extra/src/main/scala/com/spotify/scio/extra/voyager/Voyager.scala index 1d6a6ae925..af6e5d0919 100644 --- a/scio-extra/src/main/scala/com/spotify/scio/extra/voyager/VoyagerUri.scala +++ b/scio-extra/src/main/scala/com/spotify/scio/extra/voyager/Voyager.scala @@ -18,9 +18,13 @@ package com.spotify.scio.extra.voyager import com.spotify.scio.util.{RemoteFileUtil, ScioUtil} -import com.spotify.voyager.jni.Index +import com.spotify.scio.values.SideInput +import com.spotify.voyager.jni.{Index, StringIndex} import com.spotify.voyager.jni.Index.{SpaceType, StorageDataType} import org.apache.beam.sdk.options.PipelineOptions +import org.apache.beam.sdk.transforms.DoFn +import org.apache.beam.sdk.values.PCollectionView +import org.slf4j.{Logger, LoggerFactory} import java.io.File import java.net.URI @@ -46,6 +50,7 @@ sealed trait VoyagerUri { } private[voyager] object VoyagerUri { + def apply(path: String, opts: PipelineOptions): VoyagerUri = { if (ScioUtil.isLocalUri(new URI(path))) { LocalVoyagerUri(path) @@ -83,6 +88,8 @@ case class RemoteVoyagerUri( path: String, remoteFileUtil: RemoteFileUtil ) extends VoyagerUri { + import RemoteVoyagerUri._ + override private[voyager] def getReader( distanceMeasure: SpaceType, storageType: StorageDataType, @@ -111,6 +118,9 @@ case class RemoteVoyagerUri( } object RemoteVoyagerUri { + + @transient private lazy val logger: Logger = LoggerFactory.getLogger(classOf[RemoteVoyagerUri]) + def apply(path: String, options: PipelineOptions): RemoteVoyagerUri = RemoteVoyagerUri(path, RemoteFileUtil.create(options)) } @@ -122,9 +132,11 @@ private[voyager] class VoyagerWriter( ef: Long = 200L, m: Long = 16L ) { - private[this] val namesOutput = mutable.ListBuffer.empty[String] + import VoyagerWriter._ - private[this] val index: Index = + private val namesOutput = mutable.ListBuffer.empty[String] + + private val index: Index = new Index(spaceType, dim, m, ef, RANDOM_SEED, CHUNK_SIZE, storageDataType) def write(vectors: Iterable[(String, Array[Float])]): Unit = { @@ -163,3 +175,79 @@ private[voyager] class VoyagerWriter( } } + +private object VoyagerWriter { + val RANDOM_SEED: Long = 1L + val CHUNK_SIZE: Int = 32786 // 2^15 +} + +case class VoyagerResult(value: String, distance: Float) + +/** + * Voyager reader class for nearest neighbor lookups. Supports looking up neighbors for a vector and + * returning the string labels and distances associated. + * + * @param indexFileName + * The path to the `index.hnsw` local or remote file. + * @param namesFileName + * The path to the `names.json` local or remote file. + * @param spaceType + * The measurement for computing distance between entities. One of Euclidean, Cosine or Dot (inner + * product). + * @param storageDataType + * The Storage type of the vectors at rest. One of Float8, Float32 or E4M3. + * @param dim + * Number of dimensions in vectors. + */ +class VoyagerReader private[voyager] ( + indexFileName: String, + namesFileName: String, + spaceType: SpaceType, + storageDataType: StorageDataType, + dim: Int +) { + require(dim > 0, "Vector dimension should be > 0") + + private val index: StringIndex = StringIndex + .load(indexFileName, namesFileName, spaceType, dim, storageDataType) + + /** + * Gets maxNumResults nearest neighbors for vector v using ef (where ef is the size of the dynamic + * list for the nearest neighbors during search). + */ + def getNearest(v: Array[Float], maxNumResults: Int, ef: Int): Array[VoyagerResult] = { + val queryResults = index.query(v, maxNumResults, ef) + queryResults.getNames + .zip(queryResults.getDistances) + .map { case (name, distance) => + VoyagerResult(name, distance) + } + } +} + +/** + * Construction for a VoyagerSide input that leverages a synchronized map to ensure that the reader + * is only loaded once per [[VoyagerUri]]. + */ +private[voyager] class VoyagerSideInput( + val view: PCollectionView[VoyagerUri], + distanceMeasure: SpaceType, + storageType: StorageDataType, + dim: Int +) extends SideInput[VoyagerReader] { + + import VoyagerSideInput._ + override def get[I, O](context: DoFn[I, O]#ProcessContext): VoyagerReader = { + val uri = context.sideInput(view) + VOYAGER_URI_MAP.synchronized { + if (!VOYAGER_URI_MAP.contains(uri)) { + VOYAGER_URI_MAP.put(uri, uri.getReader(distanceMeasure, storageType, dim)) + } + VOYAGER_URI_MAP(uri) + } + } +} + +private object VoyagerSideInput { + private val VOYAGER_URI_MAP: mutable.Map[VoyagerUri, VoyagerReader] = mutable.HashMap.empty +} diff --git a/scio-extra/src/main/scala/com/spotify/scio/extra/voyager/package.scala b/scio-extra/src/main/scala/com/spotify/scio/extra/voyager/package.scala index 7d46a69770..6d7c426cb8 100644 --- a/scio-extra/src/main/scala/com/spotify/scio/extra/voyager/package.scala +++ b/scio-extra/src/main/scala/com/spotify/scio/extra/voyager/package.scala @@ -17,274 +17,7 @@ package com.spotify.scio.extra -import com.spotify.scio.ScioContext -import com.spotify.scio.annotations.experimental -import com.spotify.scio.values.{SCollection, SideInput} -import com.spotify.voyager.jni.Index.{SpaceType, StorageDataType} -import com.spotify.voyager.jni.StringIndex -import org.apache.beam.sdk.transforms.{DoFn, View} -import org.apache.beam.sdk.values.PCollectionView -import org.slf4j.LoggerFactory +import com.spotify.scio.extra.voyager.syntax.AllSyntax -import java.util.UUID -import scala.collection.mutable - -/** Main package for Voyager side input APIs. Import all. */ -package object voyager { - @transient lazy val logger = LoggerFactory.getLogger(this.getClass) - - case class VoyagerResult(value: String, distance: Float) - - private val VOYAGER_URI_MAP: mutable.Map[VoyagerUri, VoyagerReader] = mutable.HashMap.empty - private[voyager] val RANDOM_SEED: Long = 1L - private[voyager] val CHUNK_SIZE: Int = 32786 // 2^15 - - /** - * Voyager reader class for nearest neighbor lookups. Supports looking up neighbors for a vector - * and returning the string labels and distances associated. - * - * @param indexFileName - * The path to the `index.hnsw` local or remote file. - * @param namesFileName - * The path to the `names.json` local or remote file. - * @param spaceType - * The measurement for computing distance between entities. One of Euclidean, Cosine or Dot - * (inner product). - * @param storageDataType - * The Storage type of the vectors at rest. One of Float8, Float32 or E4M3. - * @param dim - * Number of dimensions in vectors. - */ - class VoyagerReader private[voyager] ( - indexFileName: String, - namesFileName: String, - spaceType: SpaceType, - storageDataType: StorageDataType, - dim: Int - ) { - require(dim > 0, "Vector dimension should be > 0") - - private val index: StringIndex = - StringIndex.load(indexFileName, namesFileName, spaceType, dim, storageDataType) - - /** - * Gets maxNumResults nearest neighbors for vector v using ef (where ef is the size of the - * dynamic list for the nearest neighbors during search). - */ - def getNearest(v: Array[Float], maxNumResults: Int, ef: Int): Array[VoyagerResult] = { - val queryResults = index.query(v, maxNumResults, ef) - queryResults.getNames - .zip(queryResults.getDistances) - .map { case (name, distance) => - VoyagerResult(name, distance) - } - } - } - - /** Enhanced version of [[ScioContext]] with Voyager methods */ - implicit class VoyagerScioContext(private val self: ScioContext) extends AnyVal { - - /** - * Creates a SideInput of [[VoyagerReader]] from an [[VoyagerUri]] base path. To be used with - * [[com.spotify.scio.values.SCollection.withSideInputs SCollection.withSideInputs]] - * - * @param path - * The directory path to be used for the [[VoyagerUri]]. - * @param spaceType - * The measurement for computing distance between entities. One of Euclidean, Cosine or Dot - * (inner product). - * @param storageDataType - * The Storage type of the vectors at rest. One of Float8, Float32 or E4M3. - * @param dim - * Number of dimensions in vectors. - * @return - * A [[SideInput]] of the [[VoyagerReader]] to be used for querying. - */ - @experimental - def voyagerSideInput( - path: String, - spaceType: SpaceType, - storageDataType: StorageDataType, - dim: Int - ): SideInput[VoyagerReader] = { - val uri = VoyagerUri(path, self.options) - val view = self.parallelize(Seq(uri)).applyInternal(View.asSingleton()) - new VoyagerSideInput(view, spaceType, storageDataType, dim) - } - } - - implicit class VoyagerPairSCollection( - @transient private val self: SCollection[(String, Array[Float])] - ) extends AnyVal { - - /** - * Write the key-value pairs of this SCollection as a Voyager index to a specified location - * using the parameters specified. - * - * @param path - * The directory path to be used for the [[VoyagerUri]]. - * @param spaceType - * The measurement for computing distance between entities. One of Euclidean, Cosine or Dot - * (inner product). - * @param storageDataType - * The storage data type of the vectors at rest. One of Float8, Float32 or E4M3. - * @param dim - * Number of dimensions in vectors. - * @param ef - * The size of the dynamic list of neighbors used during construction time. This parameter - * controls query time/accuracy tradeoff. More information can be found in the hnswlib - * documentation https://github.com/nmslib/hnswlib. - * @param m - * The number of outgoing connections in the graph. - * @return - * A [[VoyagerUri]] representing where the index was written to. - */ - @experimental - def asVoyager( - path: String, - spaceType: SpaceType, - storageDataType: StorageDataType, - dim: Int, - ef: Long, - m: Long - ): SCollection[VoyagerUri] = { - val uri: VoyagerUri = VoyagerUri(path, self.context.options) - require(!uri.exists, s"Voyager URI ${uri.path} already exists") - self.transform { in => - { - in.groupBy(_ => ()) - .map { case (_, xs) => - val voyagerWriter: VoyagerWriter = - new VoyagerWriter(spaceType, storageDataType, dim, ef, m) - - voyagerWriter.write(xs) - uri.saveAndClose(voyagerWriter) - uri - } - } - } - } - - /** - * Write the key-value pairs of this SCollection as a Voyager index to a temporary location and - * building the index using the parameters specified. - * - * @param distanceMeasure - * The measurement for computing distance between entities. One of Euclidean, Cosine or Dot - * (inner product). - * @param storageDataType - * The Storage type of the vectors at rest. One of Float8, Float32 or E4M3. - * @param dim - * Number of dimensions in vectors. - * @param ef - * The size of the dynamic list of neighbors used during construction time. This parameter - * controls query time/accuracy tradeoff. More information can be found in the hnswlib - * documentation https://github.com/nmslib/hnswlib. - * @param m - * The number of outgoing connections in the graph. - * @return - * A [[VoyagerUri]] representing where the index was written to. - */ - @experimental - def asVoyager( - distanceMeasure: SpaceType, - storageDataType: StorageDataType, - dim: Int, - ef: Long = 200L, - m: Long = 16L - ): SCollection[VoyagerUri] = { - val uuid: UUID = UUID.randomUUID() - val tempLocation: String = self.context.options.getTempLocation - require(tempLocation != null, s"Voyager writes require --tempLocation to be set.") - val path = s"${tempLocation.stripSuffix("/")}/voyager-build-$uuid" - this.asVoyager(path, distanceMeasure, storageDataType, dim, ef, m) - } - - /** - * Write the key-value pairs of this SCollection as a Voyager index to a temporary location, - * building the index using the parameters specified and then loading the reader into a side - * input. - * - * @param spaceType - * The measurement for computing distance between entities. One of Euclidean, Cosine or Dot - * (inner product). - * @param storageType - * The Storage type of the vectors at rest. One of Float8, Float32 or E4M3. - * @param dim - * Number of dimensions in vectors. - * @param ef - * The size of the dynamic list of neighbors used during construction time. This parameter - * controls query time/accuracy tradeoff. More information can be found in the hnswlib - * documentation https://github.com/nmslib/hnswlib. - * @param m - * The number of outgoing connections in the graph. - * @return - * A SideInput with a [[VoyagerReader]] - */ - @experimental - def asVoyagerSideInput( - spaceType: SpaceType, - storageType: StorageDataType, - dim: Int, - ef: Long = 200L, - m: Long = 16L - ): SideInput[VoyagerReader] = - self - .asVoyager(spaceType, storageType, dim, ef, m) - .asVoyagerSideInput(spaceType, storageType, dim) - } - - /** - * Enhanced version of [[com.spotify.scio.values.SCollection SCollection]] with Voyager methods. - */ - implicit class AnnoySCollection(@transient private val self: SCollection[VoyagerUri]) - extends AnyVal { - - /** - * Load the Voyager index stored at [[VoyagerUri]] in this - * [[com.spotify.scio.values.SCollection SCollection]]. - * - * @param spaceType - * The measurement for computing distance between entities. One of Euclidean, Cosine or Dot - * (inner product). - * @param storageType - * The Storage type of the vectors at rest. One of Float8, Float32 or E4M3. - * @param dim - * Number of dimensions in vectors. - * @return - * SideInput[VoyagerReader] - */ - @experimental - def asVoyagerSideInput( - spaceType: SpaceType, - storageType: StorageDataType, - dim: Int - ): SideInput[VoyagerReader] = { - val view = self.applyInternal(View.asSingleton()) - new VoyagerSideInput(view, spaceType, storageType, dim) - } - } - - /** - * Construction for a VoyagerSide input that leverages a synchronized map to ensure that the - * reader is only loaded once per [[VoyagerUri]]. - */ - private class VoyagerSideInput( - val view: PCollectionView[VoyagerUri], - distanceMeasure: SpaceType, - storageType: StorageDataType, - dim: Int - ) extends SideInput[VoyagerReader] { - override def get[I, O](context: DoFn[I, O]#ProcessContext): VoyagerReader = { - val uri = context.sideInput(view) - VOYAGER_URI_MAP.synchronized { - if (!VOYAGER_URI_MAP.contains(uri)) { - VOYAGER_URI_MAP.put(uri, uri.getReader(distanceMeasure, storageType, dim)) - } - VOYAGER_URI_MAP(uri) - } - - } - } - -} +/** Main package for Voyager side input APIs. */ +package object voyager extends AllSyntax diff --git a/scio-extra/src/main/scala/com/spotify/scio/extra/voyager/syntax/AllSyntax.scala b/scio-extra/src/main/scala/com/spotify/scio/extra/voyager/syntax/AllSyntax.scala new file mode 100644 index 0000000000..eae76f833a --- /dev/null +++ b/scio-extra/src/main/scala/com/spotify/scio/extra/voyager/syntax/AllSyntax.scala @@ -0,0 +1,3 @@ +package com.spotify.scio.extra.voyager.syntax + +trait AllSyntax extends ScioContextSyntax with SCollectionSyntax diff --git a/scio-extra/src/main/scala/com/spotify/scio/extra/voyager/syntax/SCollectionSyntax.scala b/scio-extra/src/main/scala/com/spotify/scio/extra/voyager/syntax/SCollectionSyntax.scala new file mode 100644 index 0000000000..7b8c665f52 --- /dev/null +++ b/scio-extra/src/main/scala/com/spotify/scio/extra/voyager/syntax/SCollectionSyntax.scala @@ -0,0 +1,168 @@ +package com.spotify.scio.extra.voyager.syntax + +import com.spotify.scio.annotations.experimental +import com.spotify.scio.extra.voyager.{VoyagerReader, VoyagerSideInput, VoyagerUri, VoyagerWriter} +import com.spotify.scio.values.{SCollection, SideInput} +import com.spotify.voyager.jni.Index.{SpaceType, StorageDataType} +import org.apache.beam.sdk.transforms.View + +import java.util.UUID + +class VoyagerSCollectionOps(@transient private val self: SCollection[VoyagerUri]) extends AnyVal { + + /** + * Load the Voyager index stored at [[VoyagerUri]] in this + * [[com.spotify.scio.values.SCollection SCollection]]. + * + * @param spaceType + * The measurement for computing distance between entities. One of Euclidean, Cosine or Dot + * (inner product). + * @param storageType + * The Storage type of the vectors at rest. One of Float8, Float32 or E4M3. + * @param dim + * Number of dimensions in vectors. + * @return + * SideInput[VoyagerReader] + */ + @experimental + def asVoyagerSideInput( + spaceType: SpaceType, + storageType: StorageDataType, + dim: Int + ): SideInput[VoyagerReader] = { + val view = self.applyInternal(View.asSingleton()) + new VoyagerSideInput(view, spaceType, storageType, dim) + } + +} + +class VoyagerPairSCollectionOps( + @transient private val self: SCollection[(String, Array[Float])] +) extends AnyVal { + + /** + * Write the key-value pairs of this SCollection as a Voyager index to a specified location using + * the parameters specified. + * + * @param path + * The directory path to be used for the [[VoyagerUri]]. + * @param spaceType + * The measurement for computing distance between entities. One of Euclidean, Cosine or Dot + * (inner product). + * @param storageDataType + * The storage data type of the vectors at rest. One of Float8, Float32 or E4M3. + * @param dim + * Number of dimensions in vectors. + * @param ef + * The size of the dynamic list of neighbors used during construction time. This parameter + * controls query time/accuracy tradeoff. More information can be found in the hnswlib + * documentation https://github.com/nmslib/hnswlib. + * @param m + * The number of outgoing connections in the graph. + * @return + * A [[VoyagerUri]] representing where the index was written to. + */ + @experimental + def asVoyager( + path: String, + spaceType: SpaceType, + storageDataType: StorageDataType, + dim: Int, + ef: Long, + m: Long + ): SCollection[VoyagerUri] = { + val uri = VoyagerUri(path, self.context.options) + require(!uri.exists, s"Voyager URI ${uri.path} already exists") + self.transform { in => + { + in.groupBy(_ => ()) + .map { case (_, xs) => + val voyagerWriter: VoyagerWriter = + new VoyagerWriter(spaceType, storageDataType, dim, ef, m) + + voyagerWriter.write(xs) + uri.saveAndClose(voyagerWriter) + uri + } + } + } + } + + /** + * Write the key-value pairs of this SCollection as a Voyager index to a temporary location and + * building the index using the parameters specified. + * + * @param distanceMeasure + * The measurement for computing distance between entities. One of Euclidean, Cosine or Dot + * (inner product). + * @param storageDataType + * The Storage type of the vectors at rest. One of Float8, Float32 or E4M3. + * @param dim + * Number of dimensions in vectors. + * @param ef + * The size of the dynamic list of neighbors used during construction time. This parameter + * controls query time/accuracy tradeoff. More information can be found in the hnswlib + * documentation https://github.com/nmslib/hnswlib. + * @param m + * The number of outgoing connections in the graph. + * @return + * A [[VoyagerUri]] representing where the index was written to. + */ + @experimental + def asVoyager( + distanceMeasure: SpaceType, + storageDataType: StorageDataType, + dim: Int, + ef: Long = 200L, + m: Long = 16L + ): SCollection[VoyagerUri] = { + val uuid = UUID.randomUUID() + val tempLocation: String = self.context.options.getTempLocation + require(tempLocation != null, s"Voyager writes require --tempLocation to be set.") + val path = s"${tempLocation.stripSuffix("/")}/voyager-build-$uuid" + this.asVoyager(path, distanceMeasure, storageDataType, dim, ef, m) + } + + /** + * Write the key-value pairs of this SCollection as a Voyager index to a temporary location, + * building the index using the parameters specified and then loading the reader into a side + * input. + * + * @param spaceType + * The measurement for computing distance between entities. One of Euclidean, Cosine or Dot + * (inner product). + * @param storageType + * The Storage type of the vectors at rest. One of Float8, Float32 or E4M3. + * @param dim + * Number of dimensions in vectors. + * @param ef + * The size of the dynamic list of neighbors used during construction time. This parameter + * controls query time/accuracy tradeoff. More information can be found in the hnswlib + * documentation https://github.com/nmslib/hnswlib. + * @param m + * The number of outgoing connections in the graph. + * @return + * A SideInput with a [[VoyagerReader]] + */ + @experimental + def asVoyagerSideInput( + spaceType: SpaceType, + storageType: StorageDataType, + dim: Int, + ef: Long = 200L, + m: Long = 16L + ): SideInput[VoyagerReader] = + new VoyagerSCollectionOps(asVoyager(spaceType, storageType, dim, ef, m)) + .asVoyagerSideInput(spaceType, storageType, dim) +} + +trait SCollectionSyntax { + implicit def voyagerSCollectionOps(coll: SCollection[VoyagerUri]): VoyagerSCollectionOps = + new VoyagerSCollectionOps(coll) + + implicit def VoyagerPairSCollectionOps( + coll: SCollection[(String, Array[Float])] + ): VoyagerPairSCollectionOps = + new VoyagerPairSCollectionOps(coll) + +} diff --git a/scio-extra/src/main/scala/com/spotify/scio/extra/voyager/syntax/ScioContextSyntax.scala b/scio-extra/src/main/scala/com/spotify/scio/extra/voyager/syntax/ScioContextSyntax.scala new file mode 100644 index 0000000000..5cf4796a8b --- /dev/null +++ b/scio-extra/src/main/scala/com/spotify/scio/extra/voyager/syntax/ScioContextSyntax.scala @@ -0,0 +1,45 @@ +package com.spotify.scio.extra.voyager.syntax + +import com.spotify.scio.ScioContext +import com.spotify.scio.annotations.experimental +import com.spotify.scio.extra.voyager.{VoyagerReader, VoyagerSideInput, VoyagerUri} +import com.spotify.scio.values.SideInput +import com.spotify.voyager.jni.Index.{SpaceType, StorageDataType} +import org.apache.beam.sdk.transforms.View + +/** Enhanced version of [[ScioContext]] with Voyager methods */ +class VoyagerScioContextOps(private val self: ScioContext) extends AnyVal { + + /** + * Creates a SideInput of [[VoyagerReader]] from an [[VoyagerUri]] base path. To be used with + * [[com.spotify.scio.values.SCollection.withSideInputs SCollection.withSideInputs]] + * + * @param path + * The directory path to be used for the [[VoyagerUri]]. + * @param spaceType + * The measurement for computing distance between entities. One of Euclidean, Cosine or Dot + * (inner product). + * @param storageDataType + * The Storage type of the vectors at rest. One of Float8, Float32 or E4M3. + * @param dim + * Number of dimensions in vectors. + * @return + * A [[SideInput]] of the [[VoyagerReader]] to be used for querying. + */ + @experimental + def voyagerSideInput( + path: String, + spaceType: SpaceType, + storageDataType: StorageDataType, + dim: Int + ): SideInput[VoyagerReader] = { + val uri = VoyagerUri(path, self.options) + val view = self.parallelize(Seq(uri)).applyInternal(View.asSingleton()) + new VoyagerSideInput(view, spaceType, storageDataType, dim) + } +} + +trait ScioContextSyntax { + implicit def voyagerScioContextOps(sc: ScioContext): VoyagerScioContextOps = + new VoyagerScioContextOps(sc) +} From fc0c8684959d8d64805c9b746f6ac053af660134 Mon Sep 17 00:00:00 2001 From: Claire McGinty Date: Fri, 6 Oct 2023 14:19:58 -0400 Subject: [PATCH 23/28] add headers --- .../scio/extra/voyager/syntax/AllSyntax.scala | 17 +++++++++++++++++ .../voyager/syntax/SCollectionSyntax.scala | 17 +++++++++++++++++ .../voyager/syntax/ScioContextSyntax.scala | 17 +++++++++++++++++ 3 files changed, 51 insertions(+) diff --git a/scio-extra/src/main/scala/com/spotify/scio/extra/voyager/syntax/AllSyntax.scala b/scio-extra/src/main/scala/com/spotify/scio/extra/voyager/syntax/AllSyntax.scala index eae76f833a..b9a3ef5852 100644 --- a/scio-extra/src/main/scala/com/spotify/scio/extra/voyager/syntax/AllSyntax.scala +++ b/scio-extra/src/main/scala/com/spotify/scio/extra/voyager/syntax/AllSyntax.scala @@ -1,3 +1,20 @@ +/* + * 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.extra.voyager.syntax trait AllSyntax extends ScioContextSyntax with SCollectionSyntax diff --git a/scio-extra/src/main/scala/com/spotify/scio/extra/voyager/syntax/SCollectionSyntax.scala b/scio-extra/src/main/scala/com/spotify/scio/extra/voyager/syntax/SCollectionSyntax.scala index 7b8c665f52..efde7282c2 100644 --- a/scio-extra/src/main/scala/com/spotify/scio/extra/voyager/syntax/SCollectionSyntax.scala +++ b/scio-extra/src/main/scala/com/spotify/scio/extra/voyager/syntax/SCollectionSyntax.scala @@ -1,3 +1,20 @@ +/* + * 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.extra.voyager.syntax import com.spotify.scio.annotations.experimental diff --git a/scio-extra/src/main/scala/com/spotify/scio/extra/voyager/syntax/ScioContextSyntax.scala b/scio-extra/src/main/scala/com/spotify/scio/extra/voyager/syntax/ScioContextSyntax.scala index 5cf4796a8b..e5b04a714d 100644 --- a/scio-extra/src/main/scala/com/spotify/scio/extra/voyager/syntax/ScioContextSyntax.scala +++ b/scio-extra/src/main/scala/com/spotify/scio/extra/voyager/syntax/ScioContextSyntax.scala @@ -1,3 +1,20 @@ +/* + * 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.extra.voyager.syntax import com.spotify.scio.ScioContext From ce8c1ce0a8f7a04d2b4f46305c8d94788c8e160d Mon Sep 17 00:00:00 2001 From: Michel Davit Date: Mon, 9 Oct 2023 11:07:42 +0200 Subject: [PATCH 24/28] Remove RemoteFileUtil from model --- .../scio/extra/voyager/VoyagerIT.scala | 23 +- .../spotify/scio/extra/voyager/Voyager.scala | 209 ++++++------------ .../voyager/syntax/SCollectionSyntax.scala | 60 +++-- .../voyager/syntax/ScioContextSyntax.scala | 5 +- .../scio/extra/voyager/VoyagerTest.scala | 51 ++--- 5 files changed, 138 insertions(+), 210 deletions(-) diff --git a/scio-extra/src/it/scala/com/spotify/scio/extra/voyager/VoyagerIT.scala b/scio-extra/src/it/scala/com/spotify/scio/extra/voyager/VoyagerIT.scala index 2a395e3916..7095b9c631 100644 --- a/scio-extra/src/it/scala/com/spotify/scio/extra/voyager/VoyagerIT.scala +++ b/scio-extra/src/it/scala/com/spotify/scio/extra/voyager/VoyagerIT.scala @@ -45,7 +45,7 @@ class VoyagerIT extends PipelineSpec { val tempLocation = ItUtils.gcpTempLocation("voyager-it") runWithContext { sc => sc.options.setTempLocation(tempLocation) - val (indexes, vectors) = sideData.unzip + val (names, vectors) = sideData.unzip val voyagerReader = sc .parallelize(sideData) @@ -54,12 +54,13 @@ class VoyagerIT extends PipelineSpec { val result = sc .parallelize(vectors) .withSideInputs(voyagerReader) - .flatMap { case (vector, ctx) => - ctx(voyagerReader).getNearest(vector, 1, 100).map(_.value) + .flatMap { case (v, ctx) => + ctx(voyagerReader).getNearest(v, 1, 100) } .toSCollection + .map(_.name) - result should containInAnyOrder(indexes) + result should containInAnyOrder(names) } // check files uploaded by voyager @@ -73,11 +74,11 @@ class VoyagerIT extends PipelineSpec { } it should "throw exception when Voyager file exists" in { - val path = ItUtils.gcpTempLocation("voyager-it") - val namePath = path + "/names.json" - val indexPath = path + "/index.hnsw" - val nameResourceId = FileSystems.matchNewResource(namePath, false) - val indexResourceId = FileSystems.matchNewResource(indexPath, false) + val uri = VoyagerUri(ItUtils.gcpTempLocation("voyager-it")) + val indexUri = uri.value.resolve(VoyagerUri.IndexFile) + val nameUri = uri.value.resolve(VoyagerUri.NamesFile) + val indexResourceId = FileSystems.matchNewResource(indexUri.toString, false) + val nameResourceId = FileSystems.matchNewResource(nameUri.toString, false) // write some data in the val f1 = FileSystems.create(nameResourceId, MimeTypes.BINARY) @@ -92,11 +93,11 @@ class VoyagerIT extends PipelineSpec { val e = the[IllegalArgumentException] thrownBy { runWithContext { sc => - sc.parallelize(sideData).asVoyager(path, distanceMeasure, storageType, dim, 200L, 16) + sc.parallelize(sideData).asVoyager(uri, distanceMeasure, storageType, dim, 200L, 16) } } - e.getMessage shouldBe s"requirement failed: Voyager URI $path already exists" + e.getMessage shouldBe s"requirement failed: Voyager URI ${uri.value} already exists" FileSystems.delete(Seq(nameResourceId, indexResourceId).asJava) } diff --git a/scio-extra/src/main/scala/com/spotify/scio/extra/voyager/Voyager.scala b/scio-extra/src/main/scala/com/spotify/scio/extra/voyager/Voyager.scala index af6e5d0919..aa375ceb01 100644 --- a/scio-extra/src/main/scala/com/spotify/scio/extra/voyager/Voyager.scala +++ b/scio-extra/src/main/scala/com/spotify/scio/extra/voyager/Voyager.scala @@ -19,14 +19,11 @@ package com.spotify.scio.extra.voyager import com.spotify.scio.util.{RemoteFileUtil, ScioUtil} import com.spotify.scio.values.SideInput -import com.spotify.voyager.jni.{Index, StringIndex} import com.spotify.voyager.jni.Index.{SpaceType, StorageDataType} -import org.apache.beam.sdk.options.PipelineOptions +import com.spotify.voyager.jni.{Index, StringIndex} import org.apache.beam.sdk.transforms.DoFn import org.apache.beam.sdk.values.PCollectionView -import org.slf4j.{Logger, LoggerFactory} -import java.io.File import java.net.URI import java.nio.charset.StandardCharsets import java.nio.file.{Files, Path, Paths} @@ -35,97 +32,38 @@ import scala.collection.mutable /** * Represents the base URI for a voyager index, either on a local or a remote file system. For * remote file systems, the `path` should be in the form 'scheme:////'. For local - * files, it should be in the form '//'. The `path` specified represents the directory where + * files, it should be in the form '//'. The `uri` specified represents the directory where * the `index.hnsw` and `names.json` are. */ -sealed trait VoyagerUri { - def path: String - private[voyager] def getReader( - distanceMeasure: SpaceType, - storageDataType: StorageDataType, - dim: Int - ): VoyagerReader - private[voyager] def saveAndClose(voyagerWriter: VoyagerWriter): Unit - private[voyager] def exists: Boolean -} +final case class VoyagerUri(value: URI) extends AnyVal { -private[voyager] object VoyagerUri { + import VoyagerUri._ - def apply(path: String, opts: PipelineOptions): VoyagerUri = { - if (ScioUtil.isLocalUri(new URI(path))) { - LocalVoyagerUri(path) + def exists(implicit remoteFileUtil: RemoteFileUtil): Boolean = { + if (ScioUtil.isLocalUri(value)) { + VoyagerFiles.exists(f => Paths.get(value.resolve(f)).toFile.exists()) } else { - val rfu: RemoteFileUtil = RemoteFileUtil.create(opts) - RemoteVoyagerUri(path, rfu) + VoyagerFiles.exists(f => remoteFileUtil.remoteExists(value.resolve(f))) } } - - def files: Seq[String] = Seq("index.hnsw", "names.json") -} - -case class LocalVoyagerUri(path: String) extends VoyagerUri { - override private[voyager] def getReader( - distanceMeasure: SpaceType, - storageType: StorageDataType, - dim: Int - ): VoyagerReader = { - - val indexFileName: String = path + "/index.hnsw" - val namesFileName: String = path + "/names.json" - new VoyagerReader(indexFileName, namesFileName, distanceMeasure, storageType, dim) - } - - override private[voyager] def saveAndClose(w: VoyagerWriter): Unit = { - w.save(path) - w.close() - } - - override private[voyager] def exists: Boolean = - VoyagerUri.files.exists(f => new File(path + "/" + f).exists()) } -case class RemoteVoyagerUri( - path: String, - remoteFileUtil: RemoteFileUtil -) extends VoyagerUri { - import RemoteVoyagerUri._ - - override private[voyager] def getReader( - distanceMeasure: SpaceType, - storageType: StorageDataType, - dim: Int - ): VoyagerReader = { - val indexFileName: String = remoteFileUtil.download(new URI(path + "/index.hnsw")).toString - val namesFileName: String = remoteFileUtil.download(new URI(path + "/names.json")).toString - new VoyagerReader(indexFileName, namesFileName, distanceMeasure, storageType, dim) - } +object VoyagerUri { + def apply(value: String): VoyagerUri = new VoyagerUri(new URI(value)) - override private[voyager] def saveAndClose(w: VoyagerWriter): Unit = { - val tempPath: Path = Files.createTempDirectory("voyager-") - w.save(tempPath.toString) - w.close() + private[voyager] val IndexFile = "index.hnsw" + private[voyager] val NamesFile = "names.json" - VoyagerUri.files.foreach { f => - val tf: Path = tempPath.resolve(f) - remoteFileUtil.upload(Paths.get(tf.toString), new URI(path + "/" + f)) - logger.info(s"Uploaded Voyager $f file from $tempPath to $path/$f") - Files.delete(tf) - } - } + private[voyager] val VoyagerFiles: Seq[String] = Seq(IndexFile, NamesFile) - override private[voyager] def exists: Boolean = - VoyagerUri.files.exists(f => remoteFileUtil.remoteExists(new URI(path + "/" + f))) } -object RemoteVoyagerUri { - - @transient private lazy val logger: Logger = LoggerFactory.getLogger(classOf[RemoteVoyagerUri]) +/** Result of a voyager query */ +final case class VoyagerResult(name: String, distance: Float) - def apply(path: String, options: PipelineOptions): RemoteVoyagerUri = - RemoteVoyagerUri(path, RemoteFileUtil.create(options)) -} - -private[voyager] class VoyagerWriter( +class VoyagerWriter private[voyager] ( + indexFile: Path, + namesFile: Path, spaceType: SpaceType, storageDataType: StorageDataType, dim: Int, @@ -134,63 +72,47 @@ private[voyager] class VoyagerWriter( ) { import VoyagerWriter._ - private val namesOutput = mutable.ListBuffer.empty[String] - - private val index: Index = - new Index(spaceType, dim, m, ef, RANDOM_SEED, CHUNK_SIZE, storageDataType) - def write(vectors: Iterable[(String, Array[Float])]): Unit = { - val nameVectorIndexIterator = vectors.iterator.zipWithIndex - .map { case ((name, vector), idx) => - (name, vector, idx.longValue()) + val indexOutputStream = Files.newOutputStream(indexFile) + val namesOutputStream = Files.newOutputStream(namesFile) + + val names = List.newBuilder[String] + val index = new Index(spaceType, dim, m, ef, RandomSeed, ChunkSize.toLong, storageDataType) + + vectors.zipWithIndex + .map { case ((name, vector), idx) => (name, vector, idx.toLong) } + .grouped(ChunkSize) + .map(_.unzip3) + .foreach { case (ns, vs, is) => + names ++= ns + index.addItems(vs.toArray, is.toArray, -1) } - while (nameVectorIndexIterator.hasNext) { - val (nameArray, vectorArray, indexArray) = nameVectorIndexIterator - .take(CHUNK_SIZE) - .toArray - .unzip3 - - index.addItems(vectorArray, indexArray, -1) - namesOutput ++= nameArray - } - - () - } - - def save(path: String): Unit = { - val indexFileName: String = path + "/index.hnsw" - val namesFileName: String = path + "/names.json" - index.saveIndex(indexFileName) - Files.write( - Paths.get(namesFileName), - namesOutput.mkString("[\"", "\",\"", "\"]").getBytes(StandardCharsets.UTF_8) - ) - () - } - - def close(): Unit = { + // save index + index.saveIndex(indexOutputStream) index.close() - () + // save names + val json = names.result().mkString("[\"", "\",\"", "\"]") + namesOutputStream.write(json.getBytes(StandardCharsets.UTF_8)) + // close + indexOutputStream.close() + namesOutputStream.close() } - } private object VoyagerWriter { - val RANDOM_SEED: Long = 1L - val CHUNK_SIZE: Int = 32786 // 2^15 + private val RandomSeed: Long = 1L + private val ChunkSize: Int = 32786 // 2^15 } -case class VoyagerResult(value: String, distance: Float) - /** * Voyager reader class for nearest neighbor lookups. Supports looking up neighbors for a vector and * returning the string labels and distances associated. * - * @param indexFileName - * The path to the `index.hnsw` local or remote file. - * @param namesFileName - * The path to the `names.json` local or remote file. + * @param indexFile + * The `index.hnsw` file. + * @param namesFile + * The `names.json` file. * @param spaceType * The measurement for computing distance between entities. One of Euclidean, Cosine or Dot (inner * product). @@ -200,16 +122,16 @@ case class VoyagerResult(value: String, distance: Float) * Number of dimensions in vectors. */ class VoyagerReader private[voyager] ( - indexFileName: String, - namesFileName: String, + indexFile: Path, + namesFile: Path, spaceType: SpaceType, storageDataType: StorageDataType, dim: Int ) { require(dim > 0, "Vector dimension should be > 0") - private val index: StringIndex = StringIndex - .load(indexFileName, namesFileName, spaceType, dim, storageDataType) + @transient private lazy val index: StringIndex = + StringIndex.load(indexFile.toString, namesFile.toString, spaceType, dim, storageDataType) /** * Gets maxNumResults nearest neighbors for vector v using ef (where ef is the size of the dynamic @@ -219,9 +141,7 @@ class VoyagerReader private[voyager] ( val queryResults = index.query(v, maxNumResults, ef) queryResults.getNames .zip(queryResults.getDistances) - .map { case (name, distance) => - VoyagerResult(name, distance) - } + .map { case (name, distance) => VoyagerResult(name, distance) } } } @@ -231,23 +151,30 @@ class VoyagerReader private[voyager] ( */ private[voyager] class VoyagerSideInput( val view: PCollectionView[VoyagerUri], + remoteFileUtil: RemoteFileUtil, distanceMeasure: SpaceType, storageType: StorageDataType, dim: Int ) extends SideInput[VoyagerReader] { - import VoyagerSideInput._ - override def get[I, O](context: DoFn[I, O]#ProcessContext): VoyagerReader = { - val uri = context.sideInput(view) - VOYAGER_URI_MAP.synchronized { - if (!VOYAGER_URI_MAP.contains(uri)) { - VOYAGER_URI_MAP.put(uri, uri.getReader(distanceMeasure, storageType, dim)) - } - VOYAGER_URI_MAP(uri) + @transient private lazy val readerCache: mutable.Map[VoyagerUri, VoyagerReader] = mutable.Map() + + private def createReader(uri: VoyagerUri): VoyagerReader = { + val indexUri = uri.value.resolve(VoyagerUri.IndexFile) + val namesUri = uri.value.resolve(VoyagerUri.NamesFile) + + val (localIndex, localNames) = if (ScioUtil.isLocalUri(uri.value)) { + (Paths.get(indexUri), Paths.get(namesUri)) + } else { + val downloadedIndex = remoteFileUtil.download(indexUri) + val downloadedNames = remoteFileUtil.download(namesUri) + (downloadedIndex, downloadedNames) } + new VoyagerReader(localIndex, localNames, distanceMeasure, storageType, dim) } -} -private object VoyagerSideInput { - private val VOYAGER_URI_MAP: mutable.Map[VoyagerUri, VoyagerReader] = mutable.HashMap.empty + override def get[I, O](context: DoFn[I, O]#ProcessContext): VoyagerReader = { + val uri = context.sideInput(view) + readerCache.getOrElseUpdate(uri, createReader(uri)) + } } diff --git a/scio-extra/src/main/scala/com/spotify/scio/extra/voyager/syntax/SCollectionSyntax.scala b/scio-extra/src/main/scala/com/spotify/scio/extra/voyager/syntax/SCollectionSyntax.scala index efde7282c2..804fe99422 100644 --- a/scio-extra/src/main/scala/com/spotify/scio/extra/voyager/syntax/SCollectionSyntax.scala +++ b/scio-extra/src/main/scala/com/spotify/scio/extra/voyager/syntax/SCollectionSyntax.scala @@ -19,10 +19,12 @@ package com.spotify.scio.extra.voyager.syntax import com.spotify.scio.annotations.experimental import com.spotify.scio.extra.voyager.{VoyagerReader, VoyagerSideInput, VoyagerUri, VoyagerWriter} +import com.spotify.scio.util.{RemoteFileUtil, ScioUtil} import com.spotify.scio.values.{SCollection, SideInput} import com.spotify.voyager.jni.Index.{SpaceType, StorageDataType} import org.apache.beam.sdk.transforms.View +import java.nio.file.{Files, Paths} import java.util.UUID class VoyagerSCollectionOps(@transient private val self: SCollection[VoyagerUri]) extends AnyVal { @@ -48,7 +50,13 @@ class VoyagerSCollectionOps(@transient private val self: SCollection[VoyagerUri] dim: Int ): SideInput[VoyagerReader] = { val view = self.applyInternal(View.asSingleton()) - new VoyagerSideInput(view, spaceType, storageType, dim) + new VoyagerSideInput( + view, + RemoteFileUtil.create(self.context.options), + spaceType, + storageType, + dim + ) } } @@ -61,8 +69,8 @@ class VoyagerPairSCollectionOps( * Write the key-value pairs of this SCollection as a Voyager index to a specified location using * the parameters specified. * - * @param path - * The directory path to be used for the [[VoyagerUri]]. + * @param uri + * The [[VoyagerUri]]. * @param spaceType * The measurement for computing distance between entities. One of Euclidean, Cosine or Dot * (inner product). @@ -81,27 +89,43 @@ class VoyagerPairSCollectionOps( */ @experimental def asVoyager( - path: String, + uri: VoyagerUri, spaceType: SpaceType, storageDataType: StorageDataType, dim: Int, ef: Long, m: Long ): SCollection[VoyagerUri] = { - val uri = VoyagerUri(path, self.context.options) - require(!uri.exists, s"Voyager URI ${uri.path} already exists") + implicit val remoteFileUtil: RemoteFileUtil = RemoteFileUtil.create(self.context.options) + require(!uri.exists, s"Voyager URI ${uri.value} already exists") + self.transform { in => - { - in.groupBy(_ => ()) - .map { case (_, xs) => - val voyagerWriter: VoyagerWriter = - new VoyagerWriter(spaceType, storageDataType, dim, ef, m) - - voyagerWriter.write(xs) - uri.saveAndClose(voyagerWriter) - uri + in.reifyAsIterableInGlobalWindow + .map { xs => + val indexUri = uri.value.resolve(VoyagerUri.IndexFile) + val namesUri = uri.value.resolve(VoyagerUri.NamesFile) + val isLocal = ScioUtil.isLocalUri(uri.value) + + val (localIndex, localNames) = if (isLocal) { + (Paths.get(indexUri), Paths.get(namesUri)) + } else { + val tmpDir = Files.createTempDirectory("voyager-") + val tmpIndex = tmpDir.resolve(VoyagerUri.IndexFile) + val tmpNames = tmpDir.resolve(VoyagerUri.NamesFile) + (tmpIndex, tmpNames) + } + + val writer = + new VoyagerWriter(localIndex, localNames, spaceType, storageDataType, dim, ef, m) + writer.write(xs) + + if (!isLocal) { + remoteFileUtil.upload(localIndex, indexUri) + remoteFileUtil.upload(localNames, namesUri) } - } + + uri + } } } @@ -136,8 +160,8 @@ class VoyagerPairSCollectionOps( val uuid = UUID.randomUUID() val tempLocation: String = self.context.options.getTempLocation require(tempLocation != null, s"Voyager writes require --tempLocation to be set.") - val path = s"${tempLocation.stripSuffix("/")}/voyager-build-$uuid" - this.asVoyager(path, distanceMeasure, storageDataType, dim, ef, m) + val uri = VoyagerUri(s"${tempLocation.stripSuffix("/")}/voyager-build-$uuid") + asVoyager(uri, distanceMeasure, storageDataType, dim, ef, m) } /** diff --git a/scio-extra/src/main/scala/com/spotify/scio/extra/voyager/syntax/ScioContextSyntax.scala b/scio-extra/src/main/scala/com/spotify/scio/extra/voyager/syntax/ScioContextSyntax.scala index e5b04a714d..1949ad6b0e 100644 --- a/scio-extra/src/main/scala/com/spotify/scio/extra/voyager/syntax/ScioContextSyntax.scala +++ b/scio-extra/src/main/scala/com/spotify/scio/extra/voyager/syntax/ScioContextSyntax.scala @@ -20,6 +20,7 @@ package com.spotify.scio.extra.voyager.syntax import com.spotify.scio.ScioContext import com.spotify.scio.annotations.experimental import com.spotify.scio.extra.voyager.{VoyagerReader, VoyagerSideInput, VoyagerUri} +import com.spotify.scio.util.RemoteFileUtil import com.spotify.scio.values.SideInput import com.spotify.voyager.jni.Index.{SpaceType, StorageDataType} import org.apache.beam.sdk.transforms.View @@ -50,9 +51,9 @@ class VoyagerScioContextOps(private val self: ScioContext) extends AnyVal { storageDataType: StorageDataType, dim: Int ): SideInput[VoyagerReader] = { - val uri = VoyagerUri(path, self.options) + val uri = VoyagerUri(path) val view = self.parallelize(Seq(uri)).applyInternal(View.asSingleton()) - new VoyagerSideInput(view, spaceType, storageDataType, dim) + new VoyagerSideInput(view, RemoteFileUtil.create(self.options), spaceType, storageDataType, dim) } } diff --git a/scio-extra/src/test/scala/com/spotify/scio/extra/voyager/VoyagerTest.scala b/scio-extra/src/test/scala/com/spotify/scio/extra/voyager/VoyagerTest.scala index c922bcdd33..09f615dd94 100644 --- a/scio-extra/src/test/scala/com/spotify/scio/extra/voyager/VoyagerTest.scala +++ b/scio-extra/src/test/scala/com/spotify/scio/extra/voyager/VoyagerTest.scala @@ -17,16 +17,11 @@ package com.spotify.scio.extra.voyager -import com.spotify.scio.{ScioContext, ScioResult} -import com.spotify.scio.io.ClosedTap import com.spotify.scio.testing.CoderAssertions.{notFallback, ValueShouldSyntax} import com.spotify.scio.testing.PipelineSpec -import com.spotify.scio.util.RemoteFileUtil import com.spotify.voyager.jni.Index.{SpaceType, StorageDataType} import com.spotify.voyager.jni.StringIndex -import org.apache.beam.sdk.options.PipelineOptionsFactory -import java.io.File import java.nio.file.Files class VoyagerTest extends PipelineSpec { @@ -39,19 +34,15 @@ class VoyagerTest extends PipelineSpec { "SCollection" should "support .asVoyager with specified local file" in { val tmpDir = Files.createTempDirectory("voyager-test") - val basePath = tmpDir.toString - val sc = ScioContext() - val p: ClosedTap[VoyagerUri] = - sc.parallelize(sideData) - .asVoyager(basePath, spaceType, storageDataType, dim, 200L, 16L) - .materialize + val uri = VoyagerUri(tmpDir.toUri) - val scioResult: ScioResult = sc.run().waitUntilFinish() - val path: String = scioResult.tap(p).value.next.path + runWithContext { sc => + sc.parallelize(sideData).asVoyager(uri, spaceType, storageDataType, dim, 200L, 16L) + } - val index: StringIndex = StringIndex.load( - path + "/index.hnsw", - path + "/names.json", + val index = StringIndex.load( + tmpDir.resolve(VoyagerUri.IndexFile).toString, + tmpDir.resolve(VoyagerUri.NamesFile).toString, SpaceType.Cosine, dim, StorageDataType.E4M3 @@ -63,14 +54,11 @@ class VoyagerTest extends PipelineSpec { result.getDistances.length shouldEqual 2 result.getNames should contain(data._1) } - - for (file <- Seq("index.hnsw", "names.json")) { - new File(basePath + file).delete() - } } it should "throw exception when the Voyager files already exists" in { val tmpDir = Files.createTempDirectory("voyager-test") + val uri = VoyagerUri(tmpDir.toUri) val index = tmpDir.resolve("index.hnsw") val names = tmpDir.resolve("names.json") @@ -78,27 +66,14 @@ class VoyagerTest extends PipelineSpec { Files.createFile(names) the[IllegalArgumentException] thrownBy { - runWithContext { - _.parallelize(sideData).asVoyager( - tmpDir.toString, - spaceType, - storageDataType, - dim, - 200L, - 16L - ) + runWithContext { sc => + sc.parallelize(sideData).asVoyager(uri, spaceType, storageDataType, dim, 200L, 16L) } - } should have message s"requirement failed: Voyager URI $tmpDir already exists" - - Files.delete(index) - Files.delete(names) + } should have message s"requirement failed: Voyager URI ${uri.value} already exists" } "VoyagerUri" should "not use Kryo" in { - val localUri: VoyagerUri = LocalVoyagerUri("gs://this-that") - localUri coderShould notFallback() - - val remoteUri: VoyagerUri = RemoteVoyagerUri("gs//this-that", PipelineOptionsFactory.create()) - remoteUri coderShould notFallback() + val uri = VoyagerUri("gs://this-that") + uri coderShould notFallback() } } From 72cc4314b39a95ded0a0c6a1fab9172780c734db Mon Sep 17 00:00:00 2001 From: Michel Davit Date: Mon, 9 Oct 2023 11:25:39 +0200 Subject: [PATCH 25/28] Share VoyagerReader side-input cache --- .../com/spotify/scio/extra/voyager/Voyager.scala | 14 +++++++++++--- 1 file changed, 11 insertions(+), 3 deletions(-) diff --git a/scio-extra/src/main/scala/com/spotify/scio/extra/voyager/Voyager.scala b/scio-extra/src/main/scala/com/spotify/scio/extra/voyager/Voyager.scala index aa375ceb01..cf8b101599 100644 --- a/scio-extra/src/main/scala/com/spotify/scio/extra/voyager/Voyager.scala +++ b/scio-extra/src/main/scala/com/spotify/scio/extra/voyager/Voyager.scala @@ -27,7 +27,7 @@ import org.apache.beam.sdk.values.PCollectionView import java.net.URI import java.nio.charset.StandardCharsets import java.nio.file.{Files, Path, Paths} -import scala.collection.mutable +import java.util.concurrent.ConcurrentHashMap /** * Represents the base URI for a voyager index, either on a local or a remote file system. For @@ -157,7 +157,7 @@ private[voyager] class VoyagerSideInput( dim: Int ) extends SideInput[VoyagerReader] { - @transient private lazy val readerCache: mutable.Map[VoyagerUri, VoyagerReader] = mutable.Map() + import VoyagerSideInput._ private def createReader(uri: VoyagerUri): VoyagerReader = { val indexUri = uri.value.resolve(VoyagerUri.IndexFile) @@ -175,6 +175,14 @@ private[voyager] class VoyagerSideInput( override def get[I, O](context: DoFn[I, O]#ProcessContext): VoyagerReader = { val uri = context.sideInput(view) - readerCache.getOrElseUpdate(uri, createReader(uri)) + VoyagerReaderSharedCache.computeIfAbsent(uri, createReader) } } + +private object VoyagerSideInput { + // cache the VoyagerUri to VoyagerReader per JVM so workers with multiple + // voyager side-input steps load the index only once + @transient private lazy val VoyagerReaderSharedCache + : ConcurrentHashMap[VoyagerUri, VoyagerReader] = + new ConcurrentHashMap() +} From 96003e6fea555f2cb016ef2527bc27f2e0f9f2a4 Mon Sep 17 00:00:00 2001 From: Michel Davit Date: Mon, 9 Oct 2023 11:27:20 +0200 Subject: [PATCH 26/28] Add missing dep in compile --- build.sbt | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/build.sbt b/build.sbt index 9e6080e553..b4022d2966 100644 --- a/build.sbt +++ b/build.sbt @@ -896,7 +896,9 @@ lazy val `scio-extra`: Project = project "com.google.zetasketch" % "zetasketch" % zetasketchVersion, "com.nrinaudo" %% "kantan.codecs" % kantanCodecsVersion, "com.nrinaudo" %% "kantan.csv" % kantanCsvVersion, + "com.softwaremill.magnolia1_2" %% "magnolia" % magnoliaVersion, "com.spotify" % "annoy" % annoyVersion, + "com.spotify" % "voyager" % voyagerVersion, "com.spotify.sparkey" % "sparkey" % sparkeyVersion, "com.twitter" %% "algebird-core" % algebirdVersion, "io.circe" %% "circe-core" % circeVersion, @@ -915,7 +917,6 @@ lazy val `scio-extra`: Project = project "org.scalanlp" %% "breeze" % breezeVersion, "org.slf4j" % "slf4j-api" % slf4jVersion, "org.typelevel" %% "algebra" % algebraVersion, - "com.spotify" % "voyager" % voyagerVersion, // test "com.github.ben-manes.caffeine" % "caffeine" % caffeineVersion % "test,it", "org.scalacheck" %% "scalacheck" % scalacheckVersion % "test,it", From ed82684dd24e7bc51d88d3d899ce5dc5bf498b8b Mon Sep 17 00:00:00 2001 From: Michel Davit Date: Mon, 9 Oct 2023 11:32:56 +0200 Subject: [PATCH 27/28] Rollback RemoteFileUtil equality definition --- .../java/com/spotify/scio/util/RemoteFileUtil.java | 10 ---------- 1 file changed, 10 deletions(-) diff --git a/scio-core/src/main/java/com/spotify/scio/util/RemoteFileUtil.java b/scio-core/src/main/java/com/spotify/scio/util/RemoteFileUtil.java index 303d309128..e0cdc6f232 100644 --- a/scio-core/src/main/java/com/spotify/scio/util/RemoteFileUtil.java +++ b/scio-core/src/main/java/com/spotify/scio/util/RemoteFileUtil.java @@ -252,14 +252,4 @@ private static void copyToRemote(Path src, URI dst, String mimeType) throws IOEx private static Metadata getMetadata(URI src) throws IOException { return FileSystems.matchSingleFileSpec(src.toString()); } - - @Override - public int hashCode() { - return this.getClass().hashCode(); - } - - @Override - public final boolean equals(Object other) { - return other != null && this.getClass().equals(other.getClass()); - } } From 8a4d84bac10d81f5f22becef48fd1a658dfd483c Mon Sep 17 00:00:00 2001 From: Michel Davit Date: Mon, 9 Oct 2023 11:59:07 +0200 Subject: [PATCH 28/28] Fix API and doc --- .../extra/voyager/syntax/ScioContextSyntax.scala | 7 +++---- site/src/main/paradox/extras/Voyager.md | 12 +++++++----- site/src/main/paradox/extras/index.md | 1 + 3 files changed, 11 insertions(+), 9 deletions(-) diff --git a/scio-extra/src/main/scala/com/spotify/scio/extra/voyager/syntax/ScioContextSyntax.scala b/scio-extra/src/main/scala/com/spotify/scio/extra/voyager/syntax/ScioContextSyntax.scala index 1949ad6b0e..cff6f37512 100644 --- a/scio-extra/src/main/scala/com/spotify/scio/extra/voyager/syntax/ScioContextSyntax.scala +++ b/scio-extra/src/main/scala/com/spotify/scio/extra/voyager/syntax/ScioContextSyntax.scala @@ -32,8 +32,8 @@ class VoyagerScioContextOps(private val self: ScioContext) extends AnyVal { * Creates a SideInput of [[VoyagerReader]] from an [[VoyagerUri]] base path. To be used with * [[com.spotify.scio.values.SCollection.withSideInputs SCollection.withSideInputs]] * - * @param path - * The directory path to be used for the [[VoyagerUri]]. + * @param uri + * The [[VoyagerUri]]. * @param spaceType * The measurement for computing distance between entities. One of Euclidean, Cosine or Dot * (inner product). @@ -46,12 +46,11 @@ class VoyagerScioContextOps(private val self: ScioContext) extends AnyVal { */ @experimental def voyagerSideInput( - path: String, + uri: VoyagerUri, spaceType: SpaceType, storageDataType: StorageDataType, dim: Int ): SideInput[VoyagerReader] = { - val uri = VoyagerUri(path) val view = self.parallelize(Seq(uri)).applyInternal(View.asSingleton()) new VoyagerSideInput(view, RemoteFileUtil.create(self.options), spaceType, storageDataType, dim) } diff --git a/site/src/main/paradox/extras/Voyager.md b/site/src/main/paradox/extras/Voyager.md index 4a962c7c31..485e89ffa9 100644 --- a/site/src/main/paradox/extras/Voyager.md +++ b/site/src/main/paradox/extras/Voyager.md @@ -5,25 +5,26 @@ works in python and java. ## Write -A keyed `SCollection` with `String` keys and `Array[Float]` vector values can be saved with @scaladoc[asVoyager](com.spotify.scio.extra.voyager.VoyagerPairSCollection#asVoyager(path:String,spaceType:com.spotify.voyager.jni.Index.SpaceType,storageDataType:com.spotify.voyager.jni.Index.StorageDataType,dim:Int,ef:Long,m:Long)): +A keyed `SCollection` with `String` keys and `Array[Float]` vector values can be saved with @scaladoc[asVoyager](com.spotify.scio.extra.voyager.syntax.VoyagerPairSCollectionOps#asVoyager(uri:com.spotify.scio.extra.voyager.VoyagerUri,spaceType:com.spotify.voyager.jni.Index.SpaceType,storageDataType:com.spotify.voyager.jni.Index.StorageDataType,dim:Int,ef:Long,m:Long):com.spotify.scio.values.SCollection[com.spotify.scio.extra.voyager.VoyagerUri]): ```scala import com.spotify.scio.values.SCollection import com.spotify.scio.extra.voyager._ import com.spotify.voyager.jni.Index.{SpaceType, StorageDataType} +val voyagerUri = VoyagerUri("gs://output-path") val dim: Int = ??? val ef: Long = ??? val m: Int = ??? val storageType: StorageDataType = ??? val spaceType: SpaceType = ??? val itemVectors: SCollection[(String, Array[Float])] = ??? -itemVectors.asVoyager("gs://output-path", spaceType, storageType, dim, ef, m) +itemVectors.asVoyager(voyagerUri, spaceType, storageType, dim, ef, m) ``` ## Side Input -A Voyager index can be read directly as a `SideInput` with @scaladoc[voyagerSideInput](com.spotify.scio.extra.voyager.VoyagerScioContext#voyagerSideInput(com.spotify.voyager.jni.Index.SpaceType,storageDataType:com.spotify.voyager.jni.Index.StorageDataType,dim:Int)): +A Voyager index can be read directly as a `SideInput` with @scaladoc[asVoyagerSideInput](com.spotify.scio.extra.voyager.syntax.VoyagerScioContextOps#voyagerSideInput(uri:com.spotify.scio.extra.voyager.VoyagerUri,spaceType:com.spotify.voyager.jni.Index.SpaceType,storageDataType:com.spotify.voyager.jni.Index.StorageDataType,dim:Int):com.spotify.scio.values.SideInput[com.spotify.scio.extra.voyager.VoyagerReader]): ```scala import com.spotify.scio._ @@ -33,14 +34,15 @@ import com.spotify.voyager.jni.Index.{SpaceType, StorageDataType} val sc: ScioContext = ??? +val voyagerUri = VoyagerUri("gs://output-path") val dim: Int = ??? val storageType: StorageDataType = ??? val spaceType: SpaceType = ??? val itemVectors: SCollection[(String, Array[Float])] = ??? -val voyagerSI: SideInput[VoyagerReader] = sc.voyagerSideInput("gs://input-path", spaceType, storageType, dim) +val voyagerSI: SideInput[VoyagerReader] = sc.voyagerSideInput(voyagerUri, spaceType, storageType, dim) ``` -Alternatively, an `SCollection` can be converted directly to a `SideInput` with @scaladoc[asVoyagerSideInput](com.spotify.scio.extra.voyager.VoyagerPairSCollection#asVoyagerSideInput(spaceType:com.spotify.voyager.jni.Index.SpaceType,storageDataType:com.spotify.voyager.jni.Index.StorageDataType,dim:Int,ef:Long,m:Long)): +Alternatively, an `SCollection` can be converted directly to a `SideInput` with @scaladoc[asVoyagerSideInput](com.spotify.scio.extra.voyager.syntax.VoyagerSCollectionOps#asVoyagerSideInput(spaceType:com.spotify.voyager.jni.Index.SpaceType,storageType:com.spotify.voyager.jni.Index.StorageDataType,dim:Int):com.spotify.scio.values.SideInput[com.spotify.scio.extra.voyager.VoyagerReader]): ```scala import com.spotify.scio.values.SCollection import com.spotify.scio.extra.voyager._ diff --git a/site/src/main/paradox/extras/index.md b/site/src/main/paradox/extras/index.md index dba8a6656d..85a597c1d3 100644 --- a/site/src/main/paradox/extras/index.md +++ b/site/src/main/paradox/extras/index.md @@ -17,5 +17,6 @@ * @ref:[Sparkey](Sparkey.md) * @ref:[REPL](Scio-REPL.md) * @ref:[Transforms](Transforms.md) +* @ref:[Voyager](Voyager.md) @@@