Skip to content
New issue

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

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

Already on GitHub? Sign in to your account

Voyager support in Scio #4996

Merged
merged 28 commits into from
Oct 10, 2023
Merged
Show file tree
Hide file tree
Changes from 17 commits
Commits
Show all changes
28 commits
Select commit Hold shift + click to select a range
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
2 changes: 2 additions & 0 deletions build.sbt
Original file line number Diff line number Diff line change
Expand Up @@ -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.6"
val zoltarVersion = "0.6.0"
// dependent versions
val scalatestplusVersion = s"$scalatestVersion.0"
Expand Down Expand Up @@ -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,
RustedBones marked this conversation as resolved.
Show resolved Hide resolved
// test
"com.github.ben-manes.caffeine" % "caffeine" % caffeineVersion % "test,it",
"org.scalacheck" %% "scalacheck" % scalacheckVersion % "test,it",
Expand Down
10 changes: 10 additions & 0 deletions scio-core/src/main/java/com/spotify/scio/util/RemoteFileUtil.java
Original file line number Diff line number Diff line change
Expand Up @@ -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
RustedBones marked this conversation as resolved.
Show resolved Hide resolved
public int hashCode() {
return this.getClass().hashCode();
}

@Override
public final boolean equals(Object other) {
return other != null && this.getClass().equals(other.getClass());
}
}
Original file line number Diff line number Diff line change
@@ -0,0 +1,91 @@
/*
* 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.SideInput
import com.spotify.voyager.jni.Index.{SpaceType, StorageDataType}
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 {
val dim: Int = 2
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)))
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)
}
}
}

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)
}
}
}
}
Original file line number Diff line number Diff line change
@@ -0,0 +1,166 @@
/*
* 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
clairemcginty marked this conversation as resolved.
Show resolved Hide resolved
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 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://<bucket>/<path>/'. For local
* files, it should be in the form '/<path>/'. The `path` 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
}

private[voyager] object VoyagerUri {
def apply(path: String, opts: PipelineOptions): VoyagerUri = {
if (ScioUtil.isLocalUri(new URI(path))) {
LocalVoyagerUri(path)
} else {
val rfu: RemoteFileUtil = RemoteFileUtil.create(opts)
RemoteVoyagerUri(path, rfu)
}
}

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 {
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)
}

override private[voyager] def saveAndClose(w: VoyagerWriter): Unit = {
val tempPath: Path = Files.createTempDirectory("")
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Suggested change
val tempPath: Path = Files.createTempDirectory("")
val tempPath: Path = Files.createTempDirectory("voyager-")

logger.info(s"temp path: $tempPath")
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Suggested change
logger.info(s"temp path: $tempPath")

I would move this to line 106, so we can log it as "Uploaded Voyager file from $tempPath to $path/$f" 👍

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))
Files.delete(tf)
}
}

override private[voyager] def exists: Boolean =
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(
spaceType: SpaceType,
storageDataType: StorageDataType,
dim: Int,
ef: Long = 200L,
m: Long = 16L
) {
private[this] val namesOutput = mutable.ListBuffer.empty[String]

private[this] 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())
}

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 = {
index.close()
()
}

}
Loading