diff --git a/buildSrc/src/main/kotlin/Dependencies.kt b/buildSrc/src/main/kotlin/Dependencies.kt
index 4779a1648..611bb978b 100644
--- a/buildSrc/src/main/kotlin/Dependencies.kt
+++ b/buildSrc/src/main/kotlin/Dependencies.kt
@@ -6,6 +6,7 @@ object Dependencies {
val aws2DynamodbEnhanced = "software.amazon.awssdk:dynamodb-enhanced:2.16.99"
val dokkaGradlePlugin = "org.jetbrains.dokka:dokka-gradle-plugin:1.5.0"
val flywayGradleBuildscriptDep = "gradle.plugin.com.boxfuse.client:flyway-release:5.0.2"
+ val gcpSpanner = "com.google.cloud:google-cloud-spanner:6.13.0"
val guava = "com.google.guava:guava:31.0.1-jre"
val guice = "com.google.inject:guice:5.1.0"
val jCommander = "com.beust:jcommander:1.72"
@@ -41,6 +42,8 @@ object Dependencies {
val miskAws2Dynamodb = "com.squareup.misk:misk-aws2-dynamodb:${Versions.misk}"
val miskAws2DynamodbTesting = "com.squareup.misk:misk-aws2-dynamodb-testing:${Versions.misk}"
val miskCore = "com.squareup.misk:misk-core:${Versions.misk}"
+ val miskGcp = "com.squareup.misk:misk-gcp:${Versions.misk}"
+ val miskGcpTesting = "com.squareup.misk:misk-gcp-testing:${Versions.misk}"
val miskHibernate = "com.squareup.misk:misk-hibernate:${Versions.misk}"
val miskHibernateTesting = "com.squareup.misk:misk-hibernate-testing:${Versions.misk}"
val miskJdbc = "com.squareup.misk:misk-jdbc:${Versions.misk}"
diff --git a/client-misk-spanner/build.gradle.kts b/client-misk-spanner/build.gradle.kts
new file mode 100644
index 000000000..5a969383c
--- /dev/null
+++ b/client-misk-spanner/build.gradle.kts
@@ -0,0 +1,45 @@
+apply(plugin = "kotlin")
+
+dependencies {
+ implementation(Dependencies.guava)
+ implementation(Dependencies.moshiCore)
+ implementation(Dependencies.moshiKotlin)
+ implementation(Dependencies.wireRuntime)
+ implementation(Dependencies.gcpSpanner)
+ implementation(Dependencies.guice)
+ implementation(Dependencies.okio)
+ implementation(Dependencies.kotlinStdLib)
+ implementation(Dependencies.loggingApi)
+ implementation(Dependencies.wireMoshiAdapter)
+
+ api(project(":client"))
+ // We do not want to leak client-base implementation details to customers.
+ implementation(project(":client-base"))
+
+ implementation(Dependencies.misk)
+ implementation(Dependencies.miskActions)
+ implementation(Dependencies.miskCore)
+ implementation(Dependencies.miskGcp)
+ implementation(Dependencies.miskInject)
+
+ testImplementation(Dependencies.assertj)
+ testImplementation(Dependencies.miskTesting)
+ testImplementation(Dependencies.miskGcpTesting)
+ testImplementation(project(":client-misk"))
+ testImplementation(Dependencies.kotlinTest)
+ testImplementation(Dependencies.junitEngine)
+ testImplementation(Dependencies.okHttp)
+
+ testImplementation(project(":backfila-embedded"))
+ testImplementation(project(":client-testing"))
+}
+
+val jar by tasks.getting(Jar::class) {
+ baseName = "backfila-client-misk-spanner"
+}
+
+if (rootProject.file("hooks.gradle").exists()) {
+ apply(from = rootProject.file("hooks.gradle"))
+}
+
+apply(from = "$rootDir/gradle-mvn-publish.gradle")
\ No newline at end of file
diff --git a/client-misk-spanner/gradle.properties b/client-misk-spanner/gradle.properties
new file mode 100644
index 000000000..a880185a7
--- /dev/null
+++ b/client-misk-spanner/gradle.properties
@@ -0,0 +1,4 @@
+POM_ARTIFACT_ID=client-misk-spanner
+POM_NAME=client-misk-spanner
+POM_DESCRIPTION=Misk spanner backfila client backend implementation library
+POM_PACKAGING=jar
diff --git a/client-misk-spanner/src/main/kotlin/app/cash/backfila/client/misk/spanner/SpannerBackfill.kt b/client-misk-spanner/src/main/kotlin/app/cash/backfila/client/misk/spanner/SpannerBackfill.kt
new file mode 100644
index 000000000..aca69b663
--- /dev/null
+++ b/client-misk-spanner/src/main/kotlin/app/cash/backfila/client/misk/spanner/SpannerBackfill.kt
@@ -0,0 +1,35 @@
+package app.cash.backfila.client.misk.spanner
+
+import app.cash.backfila.client.Backfill
+import app.cash.backfila.client.BackfillConfig
+import com.google.cloud.spanner.DatabaseClient
+import com.google.cloud.spanner.KeyRange
+
+abstract class SpannerBackfill : Backfill {
+ /**
+ * A previously established connection to the DB that owns the table.
+ */
+ abstract val dbClient: DatabaseClient
+
+ /**
+ * The name of the table to be used as the source of the backfill.
+ */
+ abstract val tableName: String
+
+ /**
+ * A list of names of columns that make up the table's primary keys.
+ * Only primary key columns that are strings are supported.
+ */
+ abstract val primaryKeyColumns: List
+
+ /**
+ * Override this and throw an exception to prevent the backfill from being created.
+ * This is also a good place to do any prep work before batches are run.
+ */
+ open fun validate(config: BackfillConfig) {}
+
+ /**
+ * Run a backfill operation based on the provided range of primary keys from `tableName`.
+ */
+ abstract fun runBatch(range: KeyRange, config: BackfillConfig)
+}
diff --git a/client-misk-spanner/src/main/kotlin/app/cash/backfila/client/misk/spanner/SpannerBackfillModule.kt b/client-misk-spanner/src/main/kotlin/app/cash/backfila/client/misk/spanner/SpannerBackfillModule.kt
new file mode 100644
index 000000000..158fee62d
--- /dev/null
+++ b/client-misk-spanner/src/main/kotlin/app/cash/backfila/client/misk/spanner/SpannerBackfillModule.kt
@@ -0,0 +1,70 @@
+package app.cash.backfila.client.misk.spanner
+
+import app.cash.backfila.client.spi.BackfillBackend
+import app.cash.backfila.client.misk.spanner.internal.SpannerBackend
+import com.google.inject.Binder
+import com.google.inject.Provides
+import com.google.inject.Singleton
+import com.google.inject.TypeLiteral
+import com.google.inject.multibindings.MapBinder
+import com.google.inject.multibindings.Multibinder
+import com.squareup.moshi.Moshi
+import com.squareup.moshi.kotlin.reflect.KotlinJsonAdapterFactory
+import javax.inject.Qualifier
+import kotlin.reflect.KClass
+import misk.inject.KAbstractModule
+import kotlin.reflect.jvm.jvmName
+
+class SpannerBackfillModule> private constructor(
+ private val backfillClass: KClass
+) : KAbstractModule() {
+ override fun configure() {
+ install(SpannerBackfillBackendModule)
+ // Ensures that the backfill class is injectable. If you are failing this check you probably
+ // want to add an @Inject annotation to your class or check that all of your dependencies are provided.
+ binder().getProvider(backfillClass.java)
+ mapBinder(binder()).addBinding(backfillClass.jvmName).toInstance(backfillClass)
+ }
+
+ companion object {
+ inline fun > create(): SpannerBackfillModule =
+ create(T::class)
+
+ @JvmStatic
+ fun > create(backfillClass: KClass): SpannerBackfillModule {
+ return SpannerBackfillModule(backfillClass)
+ }
+
+ @JvmStatic
+ fun > create(backfillClass: Class): SpannerBackfillModule {
+ return SpannerBackfillModule(backfillClass.kotlin)
+ }
+ }
+}
+
+/**
+ * This is a kotlin object so these dependencies are only installed once.
+ */
+private object SpannerBackfillBackendModule : KAbstractModule() {
+ override fun configure() {
+ Multibinder.newSetBinder(binder(), BackfillBackend::class.java).addBinding()
+ .to(SpannerBackend::class.java)
+ }
+
+ @Provides @Singleton @ForSpannerBackend
+ fun provideSpannerMoshi(): Moshi {
+ return Moshi.Builder()
+ .add(KotlinJsonAdapterFactory())
+ .build()
+ }
+}
+
+private fun mapBinder(binder: Binder) = MapBinder.newMapBinder(
+ binder,
+ object : TypeLiteral() {},
+ object : TypeLiteral>>() {},
+ ForSpannerBackend::class.java
+)
+
+/** Annotation for specifying dependencies specifically for this Backend. */
+@Qualifier annotation class ForSpannerBackend
diff --git a/client-misk-spanner/src/main/kotlin/app/cash/backfila/client/misk/spanner/internal/SpannerBackend.kt b/client-misk-spanner/src/main/kotlin/app/cash/backfila/client/misk/spanner/internal/SpannerBackend.kt
new file mode 100644
index 000000000..a1bb4945c
--- /dev/null
+++ b/client-misk-spanner/src/main/kotlin/app/cash/backfila/client/misk/spanner/internal/SpannerBackend.kt
@@ -0,0 +1,76 @@
+package app.cash.backfila.client.misk.spanner.internal
+
+import app.cash.backfila.client.DeleteBy
+import app.cash.backfila.client.Description
+import app.cash.backfila.client.misk.spanner.ForSpannerBackend
+import app.cash.backfila.client.misk.spanner.SpannerBackfill
+import app.cash.backfila.client.parseDeleteByDate
+import app.cash.backfila.client.spi.BackfillBackend
+import app.cash.backfila.client.spi.BackfillOperator
+import app.cash.backfila.client.spi.BackfilaParametersOperator
+import app.cash.backfila.client.spi.BackfillRegistration
+import com.google.cloud.spanner.Spanner
+import com.google.inject.Injector
+import com.google.inject.TypeLiteral
+import com.squareup.moshi.Moshi
+import com.squareup.moshi.Types
+import java.lang.reflect.ParameterizedType
+import javax.inject.Inject
+import javax.inject.Singleton
+import kotlin.reflect.KClass
+import kotlin.reflect.full.findAnnotation
+
+@Singleton
+internal class SpannerBackend @Inject constructor(
+ private val injector: Injector,
+ @ForSpannerBackend private val backfills: MutableMap>>,
+ @ForSpannerBackend internal val moshi: Moshi,
+ internal val spanner: Spanner,
+) : BackfillBackend {
+
+ private fun getBackfill(name: String): SpannerBackfill<*>? {
+ val backfillClass = backfills[name] ?: return null
+ return injector.getInstance(backfillClass.java) as SpannerBackfill<*>
+ }
+
+ private fun createSpannerOperator(
+ backfill: SpannerBackfill
+ ) = SpannerBackfillOperator(
+ backfill,
+ BackfilaParametersOperator(parametersClass(backfill::class)),
+ this,
+ )
+
+ override fun create(backfillName: String, backfillId: String): BackfillOperator? {
+ val backfill = getBackfill(backfillName)
+
+ if (backfill != null) {
+ @Suppress("UNCHECKED_CAST") // We don't know the types statically, so fake them.
+ return createSpannerOperator(backfill as SpannerBackfill)
+ }
+
+ return null
+ }
+
+ override fun backfills(): Set {
+ return backfills.map {
+ BackfillRegistration(
+ name = it.key,
+ description = it.value.findAnnotation()?.text,
+ parametersClass = parametersClass(it.value as KClass>),
+ deleteBy = it.value.findAnnotation()?.parseDeleteByDate(),
+ )
+ }.toSet()
+ }
+
+ private fun parametersClass(backfillClass: KClass>): KClass {
+ // Like MyBackfill.
+ val thisType = TypeLiteral.get(backfillClass.java)
+
+ // Like Backfill.
+ val supertype = thisType.getSupertype(SpannerBackfill::class.java).type as ParameterizedType
+
+ // Like MyDataClass
+ return (Types.getRawType(supertype.actualTypeArguments[0]) as Class).kotlin
+ }
+}
diff --git a/client-misk-spanner/src/main/kotlin/app/cash/backfila/client/misk/spanner/internal/SpannerBackfillOperator.kt b/client-misk-spanner/src/main/kotlin/app/cash/backfila/client/misk/spanner/internal/SpannerBackfillOperator.kt
new file mode 100644
index 000000000..4457f6abc
--- /dev/null
+++ b/client-misk-spanner/src/main/kotlin/app/cash/backfila/client/misk/spanner/internal/SpannerBackfillOperator.kt
@@ -0,0 +1,117 @@
+package app.cash.backfila.client.misk.spanner.internal
+
+import app.cash.backfila.client.misk.spanner.SpannerBackfill
+import app.cash.backfila.client.spi.BackfillOperator
+import app.cash.backfila.client.spi.BackfilaParametersOperator
+import app.cash.backfila.protos.clientservice.GetNextBatchRangeRequest
+import app.cash.backfila.protos.clientservice.GetNextBatchRangeResponse
+import app.cash.backfila.protos.clientservice.PrepareBackfillRequest
+import app.cash.backfila.protos.clientservice.PrepareBackfillResponse
+import app.cash.backfila.protos.clientservice.RunBatchRequest
+import app.cash.backfila.protos.clientservice.RunBatchResponse
+import com.google.cloud.spanner.Key
+import com.google.cloud.spanner.KeyRange
+import com.google.cloud.spanner.KeySet
+import com.squareup.moshi.Moshi
+import misk.moshi.adapter
+import okio.ByteString
+import okio.ByteString.Companion.encodeUtf8
+
+class SpannerBackfillOperator internal constructor(
+ override val backfill: SpannerBackfill,
+ private val parametersOperator: BackfilaParametersOperator,
+ backend: SpannerBackend,
+) : BackfillOperator {
+ private var moshi: Moshi = backend.moshi
+ private val adapter = moshi.adapter>()
+
+ override fun name() = backfill.javaClass.toString()
+
+ override fun prepareBackfill(request: PrepareBackfillRequest): PrepareBackfillResponse {
+ val config =
+ parametersOperator.constructBackfillConfig(request.parameters, request.dry_run)
+ backfill.validate(config)
+
+ val partitions = listOf(
+ PrepareBackfillResponse.Partition.Builder()
+ .backfill_range(request.range)
+ .partition_name("partition")
+ .build()
+ )
+
+ return PrepareBackfillResponse.Builder()
+ .partitions(partitions)
+ .build()
+ }
+
+ override fun getNextBatchRange(request: GetNextBatchRangeRequest): GetNextBatchRangeResponse {
+ // Establish a range to scane - either we want to start at the first key,
+ // or start from (and exclude) the last key that was scanned.
+ val range = if (request.previous_end_key == null) {
+ KeySet.all()
+ } else {
+ val previousEndKey = adapter.fromJson(request.previous_end_key.utf8())!!
+ KeySet.range(
+ KeyRange.openClosed(
+ Key.of(*previousEndKey.toTypedArray()),
+ Key.of(),
+ )
+ )
+ }
+
+ // Query the table with the desired range, only fetching the components of the primary key.
+ val query = backfill.dbClient.singleUseReadOnlyTransaction()
+ .read(backfill.tableName, range, backfill.primaryKeyColumns)
+
+ val keys = mutableListOf()
+
+ // For each result, until we reach the maximum scan size, create a key representation that
+ // can be used to uniquely identify a result row.
+ var numberToScan = request.scan_size
+ while (numberToScan > 0 && query.next()) {
+ val newKey = adapter.toJson(
+ backfill.primaryKeyColumns.map { query.getString(it) }
+ ).encodeUtf8()
+ keys.add(newKey)
+ numberToScan -= 1
+ }
+ query.close()
+
+ // Return the starting and ending keys obtained from the scan.
+ val batches = keys.chunked(request.batch_size.toInt()).map {
+ GetNextBatchRangeResponse.Batch.Builder()
+ .batch_range(
+ app.cash.backfila.protos.clientservice.KeyRange.Builder()
+ .start(it.first())
+ .end(it.last())
+ .build()
+ )
+ .scanned_record_count(it.size.toLong())
+ .matching_record_count(it.size.toLong())
+ .build()
+ }
+
+ return GetNextBatchRangeResponse.Builder()
+ .batches(batches)
+ .build()
+ }
+
+ override fun runBatch(request: RunBatchRequest): RunBatchResponse {
+ val config =
+ parametersOperator.constructBackfillConfig(request.parameters, request.dry_run)
+
+ // Create a range that encompasses the batch's starting and ending keys.
+ val startKey = adapter.fromJson(request.batch_range.start.utf8())!!.toTypedArray()
+ val endKey = adapter.fromJson(request.batch_range.end.utf8())!!.toTypedArray()
+ val keyRange = KeyRange.closedClosed(
+ Key.of(*startKey),
+ Key.of(*endKey),
+ )
+
+ // Let the backfill do whatever it wants for the given batch.
+ backfill.runBatch(keyRange, config)
+
+ return RunBatchResponse.Builder()
+ .build()
+ }
+}
diff --git a/client-misk-spanner/src/test/kotlin/app/cash/backfila/client/misk/spanner/BackfillsModule.kt b/client-misk-spanner/src/test/kotlin/app/cash/backfila/client/misk/spanner/BackfillsModule.kt
new file mode 100644
index 000000000..28d0751b0
--- /dev/null
+++ b/client-misk-spanner/src/test/kotlin/app/cash/backfila/client/misk/spanner/BackfillsModule.kt
@@ -0,0 +1,19 @@
+package app.cash.backfila.client.misk.spanner
+
+import app.cash.backfila.client.BackfilaHttpClientConfig
+import app.cash.backfila.client.misk.MiskBackfillModule
+import misk.inject.KAbstractModule
+
+class BackfillsModule : KAbstractModule() {
+ override fun configure() {
+ install(
+ MiskBackfillModule(
+ BackfilaHttpClientConfig(
+ url = "test.url", slack_channel = "#test"
+ ),
+ dependsOn = listOf()
+ )
+ )
+ install(SpannerBackfillModule.create())
+ }
+}
diff --git a/client-misk-spanner/src/test/kotlin/app/cash/backfila/client/misk/spanner/SpannerBackfillTest.kt b/client-misk-spanner/src/test/kotlin/app/cash/backfila/client/misk/spanner/SpannerBackfillTest.kt
new file mode 100644
index 000000000..3f7d54df7
--- /dev/null
+++ b/client-misk-spanner/src/test/kotlin/app/cash/backfila/client/misk/spanner/SpannerBackfillTest.kt
@@ -0,0 +1,223 @@
+package app.cash.backfila.client.misk.spanner
+
+import app.cash.backfila.client.BackfillConfig
+import app.cash.backfila.embedded.Backfila
+import app.cash.backfila.embedded.createDryRun
+import app.cash.backfila.embedded.createWetRun
+import com.google.cloud.spanner.DatabaseClient
+import com.google.cloud.spanner.DatabaseId
+import com.google.cloud.spanner.InstanceId
+import com.google.cloud.spanner.KeyRange
+import com.google.cloud.spanner.KeySet
+import com.google.cloud.spanner.Mutation
+import com.google.cloud.spanner.Mutation.WriteBuilder
+import com.google.cloud.spanner.Spanner
+import com.google.cloud.spanner.Statement
+import misk.testing.MiskTest
+import misk.testing.MiskTestModule
+import org.assertj.core.api.Assertions.assertThat
+import org.junit.jupiter.api.AfterEach
+import org.junit.jupiter.api.BeforeEach
+import org.junit.jupiter.api.Test
+import org.junit.jupiter.api.assertThrows
+import java.lang.IllegalArgumentException
+import java.util.UUID
+import javax.inject.Inject
+
+@MiskTest(startService = true)
+class SpannerBackfillTest {
+ @Suppress("unused")
+ @MiskTestModule
+ val module = TestingModule()
+
+ @Inject
+ lateinit var backfila: Backfila
+ @Inject lateinit var spanner: Spanner
+ private lateinit var dbClient: DatabaseClient
+
+ @BeforeEach
+ fun setup() {
+ dbClient = createDbClient(
+ spanner, project = TestingModule.PROJECT_ID, instance = TestingModule.INSTANCE_ID, db = TestingModule.DB_ID,
+ )
+ val adminDbClient = spanner.databaseAdminClient
+ val adminClient = adminDbClient.getDatabase(TestingModule.INSTANCE_ID, TestingModule.DB_ID)
+ val columns = TrackData.COLUMNS.joinToString(",") {
+ "${it.name} STRING(128)"
+ }
+
+ val statement = "CREATE TABLE ${TrackData.TABLE_NAME}($columns) PRIMARY KEY (id)"
+ adminClient.updateDdl(listOf(statement), null).get()
+
+ dbClient.readWriteTransaction().run {
+ repeat(10) { n ->
+ val uuid = UUID.randomUUID().toString()
+ val mutation = Mutation.newInsertBuilder(TrackData.TABLE_NAME)
+ .setColumnTo(TrackData.Column.id.name, uuid)
+ .setColumnTo(TrackData.Column.album_title.name, "album title $n")
+ .setColumnTo(TrackData.Column.album_token.name, "token $n")
+ .setColumnTo(TrackData.Column.track_title.name, "track title $n")
+ .setColumnTo(TrackData.Column.artist_name.name, "artist $n")
+ .build()
+ it.buffer(mutation)
+ }
+ }
+ }
+
+ @AfterEach
+ fun cleanup() {
+ val tableNameQuery = dbClient
+ .singleUseReadOnlyTransaction()
+ .executeQuery(
+ Statement.of(
+ """
+ SELECT
+ table_name
+ FROM
+ information_schema.tables
+ WHERE
+ table_catalog = '' and table_schema = ''
+ """.trimIndent()
+ )
+ )
+
+ val tableNames: MutableList = mutableListOf()
+ while (tableNameQuery.next()) {
+ tableNames.add(tableNameQuery.getString(0))
+ }
+
+ if (tableNames.size == 0) return
+ dbClient.readWriteTransaction().run {
+ it.batchUpdate(
+ tableNames.map {
+ tableName ->
+ Statement.of("DELETE FROM $tableName WHERE true")
+ }
+ )
+ }
+
+ val adminDbClient = spanner.databaseAdminClient
+ val adminClient = adminDbClient.getDatabase(TestingModule.INSTANCE_ID, TestingModule.DB_ID)
+ val statement = "DROP TABLE ${TrackData.TABLE_NAME}"
+ adminClient.updateDdl(listOf(statement), null).get()
+ }
+
+ @Test
+ fun `happy path`() {
+ val run = backfila.createWetRun(Param())
+ run.execute()
+
+ verify(isExplicit = true)
+ }
+
+ @Test
+ fun `happy path dry run`() {
+ val run = backfila.createDryRun(Param())
+ run.execute()
+
+ verify(isExplicit = false)
+ }
+
+ @Test
+ fun `validation stops wet run`() {
+ assertThrows {
+ backfila.createWetRun(Param(failValidation = true))
+ }
+ }
+
+ @Test
+ fun `small scan size scans everything`() {
+ val run = backfila.createWetRun(Param())
+ run.scanSize = 2
+
+ run.precomputeRemaining()
+ assertThat(run.precomputeScannedCount).isEqualTo(10)
+ assertThat(run.precomputeMatchingCount).isEqualTo(10)
+ run.scanRemaining()
+ assertThat(run.batchesToRunSnapshot.size).isEqualTo(5)
+ run.runAllScanned()
+ assertThat(run.complete()).isTrue
+
+ verify(isExplicit = true)
+ }
+
+ @Test
+ fun `small batch size processes everything`() {
+ val run = backfila.createWetRun(Param())
+ run.batchSize = 2
+
+ run.precomputeRemaining()
+ assertThat(run.precomputeScannedCount).isEqualTo(10)
+ assertThat(run.precomputeMatchingCount).isEqualTo(10)
+ run.scanRemaining()
+ assertThat(run.batchesToRunSnapshot.size).isEqualTo(5)
+ run.runAllScanned()
+ assertThat(run.complete()).isTrue
+
+ verify(isExplicit = true)
+ }
+
+ private fun verify(isExplicit: Boolean) {
+ dbClient.singleUseReadOnlyTransaction().run {
+ val result = this.read(
+ TrackData.TABLE_NAME,
+ KeySet.all(),
+ listOf(TrackData.Column.track_title.name)
+ )
+ result.use {
+ while (result.next()) {
+ val isExplicitResult = result.getString(TrackData.Column.track_title.name).endsWith("(EXPLICIT)")
+ assertThat(isExplicit).isEqualTo(isExplicitResult)
+ }
+ }
+ }
+ }
+
+ data class Param(
+ val failValidation: Boolean = false,
+ )
+
+ class MakeTracksExplicitBackfill @Inject constructor(
+ spanner: Spanner,
+ ) : SpannerBackfill() {
+ override val dbClient: DatabaseClient = createDbClient(
+ spanner, project = TestingModule.PROJECT_ID, instance = TestingModule.INSTANCE_ID, db = TestingModule.DB_ID
+ )
+ override val primaryKeyColumns: List = listOf(TrackData.Column.id.name)
+ override val tableName: String = TrackData.TABLE_NAME
+
+ override fun runBatch(range: KeyRange, config: BackfillConfig) {
+ val transaction = dbClient.readWriteTransaction()
+ transaction.run { transaction ->
+ val result = transaction.read(
+ tableName,
+ KeySet.range(range),
+ listOf(TrackData.Column.id.name, TrackData.Column.track_title.name),
+ )
+ result.use {
+ while (result.next()) {
+ val title = result.getString(TrackData.Column.track_title.name)
+ if (!title.endsWith("(EXPLICIT)") && !config.dryRun) {
+ val id = result.getString(TrackData.Column.id.name)
+ val mutation = Mutation.newUpdateBuilder(tableName)
+ val change = mutation.setColumnTo(TrackData.Column.id.name, id)
+ .setColumnTo(TrackData.Column.track_title.name, "$title (EXPLICIT)")
+ .build()
+ transaction.buffer(change)
+ }
+ }
+ }
+ }
+ }
+
+ override fun validate(config: BackfillConfig) {
+ super.validate(config)
+ require(!config.parameters.failValidation)
+ }
+ }
+}
+
+fun WriteBuilder.setColumnTo(column: String, value: String): WriteBuilder = set(column).to(value)
+
+fun createDbClient(spanner: Spanner, project: String, instance: String, db: String) =
+ spanner.getDatabaseClient(DatabaseId.of(InstanceId.of(project, instance), db))
diff --git a/client-misk-spanner/src/test/kotlin/app/cash/backfila/client/misk/spanner/TestingModule.kt b/client-misk-spanner/src/test/kotlin/app/cash/backfila/client/misk/spanner/TestingModule.kt
new file mode 100644
index 000000000..f8e58822f
--- /dev/null
+++ b/client-misk-spanner/src/test/kotlin/app/cash/backfila/client/misk/spanner/TestingModule.kt
@@ -0,0 +1,43 @@
+package app.cash.backfila.client.misk.spanner
+
+import app.cash.backfila.embedded.EmbeddedBackfilaModule
+import misk.MiskTestingServiceModule
+import misk.cloud.gcp.spanner.GoogleSpannerEmulatorModule
+import misk.cloud.gcp.spanner.GoogleSpannerModule
+import misk.cloud.gcp.spanner.SpannerConfig
+import misk.cloud.gcp.spanner.SpannerEmulatorConfig
+import misk.environment.DeploymentModule
+import misk.inject.KAbstractModule
+import misk.logging.LogCollectorModule
+import wisp.deployment.TESTING
+
+/**
+ * Simulates a specific service implementation module
+ */
+class TestingModule : KAbstractModule() {
+ companion object {
+ val DB_ID = "test-db"
+ val INSTANCE_ID = "test"
+ val PROJECT_ID = "test"
+ }
+
+ override fun configure() {
+ install(DeploymentModule(TESTING))
+ install(LogCollectorModule())
+ install(MiskTestingServiceModule())
+ install(BackfillsModule())
+
+ install(EmbeddedBackfilaModule())
+
+ val spannerConfig = SpannerConfig(
+ database = DB_ID,
+ emulator = SpannerEmulatorConfig(
+ enabled = true,
+ ),
+ instance_id = INSTANCE_ID,
+ project_id = PROJECT_ID,
+ )
+ install(GoogleSpannerModule(spannerConfig))
+ install(GoogleSpannerEmulatorModule(spannerConfig))
+ }
+}
diff --git a/client-misk-spanner/src/test/kotlin/app/cash/backfila/client/misk/spanner/TrackData.kt b/client-misk-spanner/src/test/kotlin/app/cash/backfila/client/misk/spanner/TrackData.kt
new file mode 100644
index 000000000..9e6074313
--- /dev/null
+++ b/client-misk-spanner/src/test/kotlin/app/cash/backfila/client/misk/spanner/TrackData.kt
@@ -0,0 +1,16 @@
+package app.cash.backfila.client.misk.spanner
+
+class TrackData {
+ companion object {
+ val TABLE_NAME = "track_data"
+ val COLUMNS = Column.values()
+ }
+
+ enum class Column {
+ id,
+ album_token,
+ track_title,
+ album_title,
+ artist_name,
+ }
+}
diff --git a/client-misk-spanner/src/test/kotlin/app/cash/backfila/client/misk/spanner/TrackItem.kt b/client-misk-spanner/src/test/kotlin/app/cash/backfila/client/misk/spanner/TrackItem.kt
new file mode 100644
index 000000000..034b33c7e
--- /dev/null
+++ b/client-misk-spanner/src/test/kotlin/app/cash/backfila/client/misk/spanner/TrackItem.kt
@@ -0,0 +1,9 @@
+package app.cash.backfila.client.misk.spanner
+
+class TrackItem {
+ var id: String? = null
+ var album_token: String? = null
+ var track_title: String? = null
+ var album_title: String? = null
+ var artist_name: String? = null
+}
diff --git a/polyrepo-settings.gradle.kts b/polyrepo-settings.gradle.kts
index 32f193a65..52cd3cae3 100644
--- a/polyrepo-settings.gradle.kts
+++ b/polyrepo-settings.gradle.kts
@@ -5,6 +5,7 @@ include("client-dynamodb")
include("client-jooq")
include("client-misk")
include("client-misk-hibernate")
+include("client-misk-spanner")
include("client-static")
include("client-testing")
include("service")
diff --git a/settings.gradle.kts b/settings.gradle.kts
index 9f7efd55d..f8a769c49 100644
--- a/settings.gradle.kts
+++ b/settings.gradle.kts
@@ -6,6 +6,7 @@ include("client-dynamodb-v2")
include("client-jooq")
include("client-misk")
include("client-misk-hibernate")
+include("client-misk-spanner")
include("client-static")
include("client-testing")
include("service")