From 4b12540102a516b8de68a1d9f503a40dd8c9784e Mon Sep 17 00:00:00 2001 From: Michel Davit Date: Fri, 1 Sep 2023 16:04:58 +0200 Subject: [PATCH 1/7] Move to hbase bigtable --- build.sbt | 6 +- .../scala/com/spotify/scio/io/ScioIO.scala | 4 +- .../scio/examples/extra/BigtableExample.scala | 46 +-- .../extra/MagnolifyBigtableExample.scala | 224 +++++------ .../examples/extra/BigtableExampleTest.scala | 37 +- .../spotify/scio/bigtable/BigtableIT.scala | 202 ++++------ .../scio/bigtable/BigtableBulkWriter.java | 229 ----------- .../spotify/scio/bigtable/BigtableDoFn.java | 109 ----- .../spotify/scio/bigtable/BigtableUtil.java | 166 -------- .../scio/bigtable/ChannelPoolCreator.java | 59 --- .../gcp/bigtable/BigtableServiceHelper.java | 41 -- .../spotify/scio/bigtable/BigTableIO.scala | 146 +++---- .../scio/bigtable/BigtableCoders.scala | 29 ++ .../spotify/scio/bigtable/InstanceAdmin.scala | 88 ++++ .../com/spotify/scio/bigtable/Mutations.scala | 102 ----- .../com/spotify/scio/bigtable/Rows.scala | 75 ---- .../spotify/scio/bigtable/TableAdmin.scala | 289 +++---------- .../scio/bigtable/syntax/AllSyntax.scala | 4 +- .../scio/bigtable/syntax/RowSyntax.scala | 85 ---- .../bigtable/syntax/SCollectionSyntax.scala | 38 +- .../bigtable/syntax/ScioContextSyntax.scala | 379 ++++++------------ .../scio/coders/GcpKryoRegistrar.scala | 6 +- .../coders/instances/kryo/GcpSerializer.scala | 43 -- .../scio/bigtable/BigtableBulkWriterTest.java | 200 --------- .../scio/bigtable/BigtableUtilTest.java | 33 -- .../scio/bigtable/BigtableDoFnTest.scala | 100 ----- .../scio/bigtable/BigtableIOTest.scala | 40 +- .../spotify/scio/bigtable/RichRowTest.scala | 101 ----- .../instance/kryo/GcpSerializerTest.scala | 68 ---- .../com/spotify/scio/testing/ScioIOSpec.scala | 57 ++- 30 files changed, 656 insertions(+), 2350 deletions(-) delete mode 100644 scio-google-cloud-platform/src/main/java/com/spotify/scio/bigtable/BigtableBulkWriter.java delete mode 100644 scio-google-cloud-platform/src/main/java/com/spotify/scio/bigtable/BigtableDoFn.java delete mode 100644 scio-google-cloud-platform/src/main/java/com/spotify/scio/bigtable/BigtableUtil.java delete mode 100644 scio-google-cloud-platform/src/main/java/com/spotify/scio/bigtable/ChannelPoolCreator.java delete mode 100644 scio-google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableServiceHelper.java create mode 100644 scio-google-cloud-platform/src/main/scala/com/spotify/scio/bigtable/BigtableCoders.scala create mode 100644 scio-google-cloud-platform/src/main/scala/com/spotify/scio/bigtable/InstanceAdmin.scala delete mode 100644 scio-google-cloud-platform/src/main/scala/com/spotify/scio/bigtable/Mutations.scala delete mode 100644 scio-google-cloud-platform/src/main/scala/com/spotify/scio/bigtable/Rows.scala delete mode 100644 scio-google-cloud-platform/src/main/scala/com/spotify/scio/bigtable/syntax/RowSyntax.scala delete mode 100644 scio-google-cloud-platform/src/main/scala/com/spotify/scio/coders/instances/kryo/GcpSerializer.scala delete mode 100644 scio-google-cloud-platform/src/test/java/com/spotify/scio/bigtable/BigtableBulkWriterTest.java delete mode 100644 scio-google-cloud-platform/src/test/java/com/spotify/scio/bigtable/BigtableUtilTest.java delete mode 100644 scio-google-cloud-platform/src/test/scala/com/spotify/scio/bigtable/BigtableDoFnTest.scala delete mode 100644 scio-google-cloud-platform/src/test/scala/com/spotify/scio/bigtable/RichRowTest.scala delete mode 100644 scio-google-cloud-platform/src/test/scala/com/spotify/scio/coders/instance/kryo/GcpSerializerTest.scala diff --git a/build.sbt b/build.sbt index d647465614..2ce5346db0 100644 --- a/build.sbt +++ b/build.sbt @@ -37,7 +37,6 @@ val autoServiceVersion = "1.0.1" val autoValueVersion = "1.9" val avroVersion = "1.8.2" val bigdataossVersion = "2.2.16" -val bigtableClientVersion = "1.28.0" val commonsCodecVersion = "1.15" val commonsCompressVersion = "1.21" val commonsIoVersion = "2.13.0" @@ -99,6 +98,7 @@ val algebraVersion = "2.9.0" val annoy4sVersion = "0.10.0" val annoyVersion = "0.2.6" val breezeVersion = "2.1.0" +val bigtableHbaseBeamVersion = "2.11.0" val caffeineVersion = "2.9.3" val cassandraDriverVersion = "3.11.4" val cassandraVersion = "3.11.16" @@ -733,8 +733,7 @@ lazy val `scio-google-cloud-platform`: Project = project "com.google.cloud" % "google-cloud-core" % googleCloudCoreVersion, "com.google.cloud" % "google-cloud-spanner" % googleCloudSpannerVersion, "com.google.cloud.bigdataoss" % "util" % bigdataossVersion, - "com.google.cloud.bigtable" % "bigtable-client-core" % bigtableClientVersion, - "com.google.cloud.bigtable" % "bigtable-client-core-config" % bigtableClientVersion, + "com.google.cloud.bigtable" % "bigtable-hbase-beam" % bigtableHbaseBeamVersion, "com.google.guava" % "guava" % guavaVersion, "com.google.http-client" % "google-http-client" % googleHttpClientsVersion, "com.google.http-client" % "google-http-client-gson" % googleHttpClientsVersion, @@ -753,6 +752,7 @@ lazy val `scio-google-cloud-platform`: Project = project "org.apache.beam" % "beam-sdks-java-core" % beamVersion, "org.apache.beam" % "beam-sdks-java-extensions-google-cloud-platform-core" % beamVersion, "org.apache.beam" % "beam-sdks-java-io-google-cloud-platform" % beamVersion, + "org.apache.beam" % "beam-sdks-java-io-hbase" % beamVersion, "org.apache.beam" % "beam-vendor-guava-26_0-jre" % beamVendorVersion, "org.slf4j" % "slf4j-api" % slf4jVersion, // test diff --git a/scio-core/src/main/scala/com/spotify/scio/io/ScioIO.scala b/scio-core/src/main/scala/com/spotify/scio/io/ScioIO.scala index e91104cdc5..3f5ecd21c2 100644 --- a/scio-core/src/main/scala/com/spotify/scio/io/ScioIO.scala +++ b/scio-core/src/main/scala/com/spotify/scio/io/ScioIO.scala @@ -148,8 +148,8 @@ object ScioIO { /** Base trait for [[ScioIO]] without business logic, for stubbing mock data with `JobTest`. */ trait TestIO[T] extends ScioIO[T] { - override type ReadP = Nothing - override type WriteP = Nothing + override type ReadP = Unit + override type WriteP = Unit override protected def read(sc: ScioContext, params: ReadP): SCollection[T] = throw new UnsupportedOperationException(s"$this is for testing purpose only") diff --git a/scio-examples/src/main/scala/com/spotify/scio/examples/extra/BigtableExample.scala b/scio-examples/src/main/scala/com/spotify/scio/examples/extra/BigtableExample.scala index 1ef57e301b..7129131922 100644 --- a/scio-examples/src/main/scala/com/spotify/scio/examples/extra/BigtableExample.scala +++ b/scio-examples/src/main/scala/com/spotify/scio/examples/extra/BigtableExample.scala @@ -18,35 +18,31 @@ // Example: Bigtable Input and Output package com.spotify.scio.examples.extra -import com.google.bigtable.v2.{Mutation, Row} -import com.google.protobuf.ByteString import com.spotify.scio._ import com.spotify.scio.bigtable._ import com.spotify.scio.examples.common.ExampleData +import org.apache.hadoop.hbase.client.{Mutation, Put, Result} +import org.apache.hadoop.hbase.{HColumnDescriptor, HTableDescriptor, TableName} import org.joda.time.Duration +import java.nio.charset.StandardCharsets + // This depends on APIs from `scio-bigtable` and imports from `com.spotify.scio.bigtable._`. object BigtableExample { - val FAMILY_NAME: String = "count" - val COLUMN_QUALIFIER: ByteString = ByteString.copyFromUtf8("long") + val FAMILY_NAME: Array[Byte] = "count".getBytes(StandardCharsets.UTF_8) + val COLUMN_QUALIFIER: Array[Byte] = "long".getBytes(StandardCharsets.UTF_8) // Convert a key-value pair to a Bigtable `Mutation` for writing - def toMutation(key: String, value: Long): (ByteString, Iterable[Mutation]) = { - val m = Mutations.newSetCell( - FAMILY_NAME, - COLUMN_QUALIFIER, - ByteString.copyFromUtf8(value.toString), - 0L - ) - (ByteString.copyFromUtf8(key), Iterable(m)) + def toPutMutation(key: String, value: Long): Mutation = + new Put(key.getBytes(StandardCharsets.UTF_8)) + .addColumn(FAMILY_NAME, COLUMN_QUALIFIER, 0L, BigInt(value).toByteArray) + + // Convert a Bigtable `Result` from reading to a formatted key-value string + def fromRow(r: Result): String = { + val key = new String(r.getRow, StandardCharsets.UTF_8) + val value = BigInt(r.getValue(FAMILY_NAME, COLUMN_QUALIFIER)).toLong + s"$key:$value" } - - // Convert a Bigtable `Row` from reading to a formatted key-value string - def fromRow(r: Row): String = - r.getKey.toStringUtf8 + ": " + r - .getValue(FAMILY_NAME, COLUMN_QUALIFIER) - .get - .toStringUtf8 } // ## Bigtable Write example @@ -70,28 +66,28 @@ object BigtableWriteExample { // Bump up the number of bigtable nodes before writing so that the extra traffic does not // affect production service. A sleep period is inserted to ensure all new nodes are online // before the ingestion starts. - sc.updateNumberOfBigtableNodes(btProjectId, btInstanceId, 15) + sc.resizeClusters(btProjectId, btInstanceId, 15) // Ensure that destination tables and column families exist + val table = new HTableDescriptor(TableName.valueOf(btTableId)) + .addFamily(new HColumnDescriptor(BigtableExample.FAMILY_NAME)) sc.ensureTables( btProjectId, btInstanceId, - Map( - btTableId -> List(BigtableExample.FAMILY_NAME) - ) + List(table) ) sc.textFile(args.getOrElse("input", ExampleData.KING_LEAR)) .flatMap(_.split("[^a-zA-Z']+").filter(_.nonEmpty)) .countByValue - .map(kv => BigtableExample.toMutation(kv._1, kv._2)) + .map { case (word, count) => BigtableExample.toPutMutation(word, count) } .saveAsBigtable(btProjectId, btInstanceId, btTableId) sc.run().waitUntilDone() // Bring down the number of nodes after the job ends to save cost. There is no need to wait // after bumping the nodes down. - sc.updateNumberOfBigtableNodes(btProjectId, btInstanceId, 3, Duration.ZERO) + sc.resizeClusters(btProjectId, btInstanceId, 3, Duration.ZERO) } } diff --git a/scio-examples/src/main/scala/com/spotify/scio/examples/extra/MagnolifyBigtableExample.scala b/scio-examples/src/main/scala/com/spotify/scio/examples/extra/MagnolifyBigtableExample.scala index fd52360ea7..253cec5f48 100644 --- a/scio-examples/src/main/scala/com/spotify/scio/examples/extra/MagnolifyBigtableExample.scala +++ b/scio-examples/src/main/scala/com/spotify/scio/examples/extra/MagnolifyBigtableExample.scala @@ -1,112 +1,112 @@ -/* - * Copyright 2020 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. - */ - -// Example: Handling Bigtable Mutations and Rows with Magnolify - -// Bigtable `Mutation` and `Row` classes are Bigtable low level API tightly -// coupled types. Writing/reading a pair of values to/from a Bigtable table -// requires to produce redundant non-reusable and error-prone code. By using -// [Magnolify](https://github.com/spotify/magnolify), one can seamlessly -// convert a case classes to `Seq[Mutation]` for writing and a `Row` to -// a case class when reading data back from a Bigtable table. -package com.spotify.scio.examples.extra - -import com.google.protobuf.ByteString -import com.spotify.scio._ -import com.spotify.scio.bigtable._ -import com.spotify.scio.examples.common.ExampleData -import magnolify.bigtable._ - -import scala.collection.compat._ - -object MagnolifyBigtableExample { - // Define case class representation of TensorFlow `Example` - case class WordCount(cnt: Long) - // `BigtableType` provides mapping between case classes and `Seq[Mutation]`/`Row` - // for writing/reading. - val WordCountType: BigtableType[WordCount] = BigtableType[WordCount] -} - -// ## Magnolify Bigtable Write Example -// Count words and save result to a Bigtable table - -// Usage: - -// `sbt "runMain com.spotify.scio.examples.extra.MagnolifyBigtableWriteExample -// --project=[PROJECT] --runner=DataflowRunner --region=[REGION NAME] -// --input=gs://apache-beam-samples/shakespeare/kinglear.txt -// --bigtableProjectId=[BIG_TABLE_PROJECT_ID] -// --bigtableInstanceId=[BIG_TABLE_INSTANCE_ID] -// --bigtableTableId=[BIG_TABLE_TABLE_ID] -object MagnolifyBigtableWriteExample { - def main(cmdlineArgs: Array[String]): Unit = { - import MagnolifyBigtableExample._ - - val (sc, args) = ContextAndArgs(cmdlineArgs) - - val btProjectId = args("bigtableProjectId") - val btInstanceId = args("bigtableInstanceId") - val btTableId = args("bigtableTableId") - - sc.textFile(args.getOrElse("input", ExampleData.KING_LEAR)) - .flatMap(_.split("[^a-zA-Z']+").filter(_.nonEmpty)) - .countByValue - // Convert case class to `Seq[Mutation]` and lift it into a key-value pair - // for saving to Bigtable table. - .map { case (word, count) => - val mutations = - WordCountType(WordCount(count), columnFamily = "counts").iterator.to(Iterable) - ByteString.copyFromUtf8(word) -> mutations - } - .saveAsBigtable(btProjectId, btInstanceId, btTableId) - - sc.run() - () - } -} - -// ## Magnolify Bigtable Read example -// Read word count result back from Bigtable - -// Usage: - -// `sbt "runMain com.spotify.scio.examples.extra.MagnolifyBigtableReadExample -// --project=[PROJECT] --runner=DataflowRunner --region=[REGION NAME] -// --bigtableProjectId=[BIG_TABLE_PROJECT_ID] -// --bigtableInstanceId=[BIG_TABLE_INSTANCE_ID] -// --bigtableTableId=[BIG_TABLE_TABLE_ID] -// --output=gs://[BUCKET]/[PATH]/wordcount"` -object MagnolifyBigtableReadExample { - def main(cmdlineArgs: Array[String]): Unit = { - import MagnolifyBigtableExample._ - - val (sc, args) = ContextAndArgs(cmdlineArgs) - val btProjectId = args("bigtableProjectId") - val btInstanceId = args("bigtableInstanceId") - val btTableId = args("bigtableTableId") - - sc.bigtable(btProjectId, btInstanceId, btTableId) - .map { row => - // Convert Bigtable `Row` to the case class and lift it into a key-value pair. - row.getKey.toStringUtf8 -> WordCountType(row, columnFamily = "counts").cnt - } - .saveAsTextFile(args("output")) - - sc.run() - () - } -} +///* +// * Copyright 2020 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. +// */ +// +//// Example: Handling Bigtable Mutations and Rows with Magnolify +// +//// Bigtable `Mutation` and `Row` classes are Bigtable low level API tightly +//// coupled types. Writing/reading a pair of values to/from a Bigtable table +//// requires to produce redundant non-reusable and error-prone code. By using +//// [Magnolify](https://github.com/spotify/magnolify), one can seamlessly +//// convert a case classes to `Seq[Mutation]` for writing and a `Row` to +//// a case class when reading data back from a Bigtable table. +//package com.spotify.scio.examples.extra +// +//import com.google.protobuf.ByteString +//import com.spotify.scio._ +//import com.spotify.scio.bigtable._ +//import com.spotify.scio.examples.common.ExampleData +//import magnolify.bigtable._ +// +//import scala.collection.compat._ +// +//object MagnolifyBigtableExample { +// // Define case class representation of TensorFlow `Example` +// case class WordCount(cnt: Long) +// // `BigtableType` provides mapping between case classes and `Seq[Mutation]`/`Row` +// // for writing/reading. +// val WordCountType: BigtableType[WordCount] = BigtableType[WordCount] +//} +// +//// ## Magnolify Bigtable Write Example +//// Count words and save result to a Bigtable table +// +//// Usage: +// +//// `sbt "runMain com.spotify.scio.examples.extra.MagnolifyBigtableWriteExample +//// --project=[PROJECT] --runner=DataflowRunner --region=[REGION NAME] +//// --input=gs://apache-beam-samples/shakespeare/kinglear.txt +//// --bigtableProjectId=[BIG_TABLE_PROJECT_ID] +//// --bigtableInstanceId=[BIG_TABLE_INSTANCE_ID] +//// --bigtableTableId=[BIG_TABLE_TABLE_ID] +//object MagnolifyBigtableWriteExample { +// def main(cmdlineArgs: Array[String]): Unit = { +// import MagnolifyBigtableExample._ +// +// val (sc, args) = ContextAndArgs(cmdlineArgs) +// +// val btProjectId = args("bigtableProjectId") +// val btInstanceId = args("bigtableInstanceId") +// val btTableId = args("bigtableTableId") +// +// sc.textFile(args.getOrElse("input", ExampleData.KING_LEAR)) +// .flatMap(_.split("[^a-zA-Z']+").filter(_.nonEmpty)) +// .countByValue +// // Convert case class to `Seq[Mutation]` and lift it into a key-value pair +// // for saving to Bigtable table. +// .map { case (word, count) => +// val mutations = +// WordCountType(WordCount(count), columnFamily = "counts").iterator.to(Iterable) +// ByteString.copyFromUtf8(word) -> mutations +// } +// .saveAsBigtable(btProjectId, btInstanceId, btTableId) +// +// sc.run() +// () +// } +//} +// +//// ## Magnolify Bigtable Read example +//// Read word count result back from Bigtable +// +//// Usage: +// +//// `sbt "runMain com.spotify.scio.examples.extra.MagnolifyBigtableReadExample +//// --project=[PROJECT] --runner=DataflowRunner --region=[REGION NAME] +//// --bigtableProjectId=[BIG_TABLE_PROJECT_ID] +//// --bigtableInstanceId=[BIG_TABLE_INSTANCE_ID] +//// --bigtableTableId=[BIG_TABLE_TABLE_ID] +//// --output=gs://[BUCKET]/[PATH]/wordcount"` +//object MagnolifyBigtableReadExample { +// def main(cmdlineArgs: Array[String]): Unit = { +// import MagnolifyBigtableExample._ +// +// val (sc, args) = ContextAndArgs(cmdlineArgs) +// val btProjectId = args("bigtableProjectId") +// val btInstanceId = args("bigtableInstanceId") +// val btTableId = args("bigtableTableId") +// +// sc.bigtable(btProjectId, btInstanceId, btTableId) +// .map { row => +// // Convert Bigtable `Row` to the case class and lift it into a key-value pair. +// row.getKey.toStringUtf8 -> WordCountType(row, columnFamily = "counts").cnt +// } +// .saveAsTextFile(args("output")) +// +// sc.run() +// () +// } +//} diff --git a/scio-examples/src/test/scala/com/spotify/scio/examples/extra/BigtableExampleTest.scala b/scio-examples/src/test/scala/com/spotify/scio/examples/extra/BigtableExampleTest.scala index 95ce4be055..9a46b4a9a7 100644 --- a/scio-examples/src/test/scala/com/spotify/scio/examples/extra/BigtableExampleTest.scala +++ b/scio-examples/src/test/scala/com/spotify/scio/examples/extra/BigtableExampleTest.scala @@ -17,11 +17,15 @@ package com.spotify.scio.examples.extra -import com.google.bigtable.v2.{Mutation, Row} -import com.google.protobuf.ByteString +import com.google.cloud.bigtable.hbase.adapters.read.RowCell import com.spotify.scio.bigtable._ import com.spotify.scio.io._ import com.spotify.scio.testing._ +import org.apache.hadoop.hbase.Cell +import org.apache.hadoop.hbase.client.{Mutation, Result} + +import java.nio.charset.StandardCharsets +import java.util.Collections class BigtableExampleTest extends PipelineSpec { import BigtableExample._ @@ -33,37 +37,38 @@ class BigtableExampleTest extends PipelineSpec { ) val textIn: Seq[String] = Seq("a b c d e", "a b a b") - val wordCount: Seq[(String, Long)] = Seq(("a", 3L), ("b", 3L), ("c", 1L), ("d", 1L), ("e", 1L)) - val expectedMutations: Seq[(ByteString, Iterable[Mutation])] = - wordCount.map(kv => BigtableExample.toMutation(kv._1, kv._2)) + val wordCount: Seq[(String, Long)] = Seq("a" -> 3L, "b" -> 3L, "c" -> 1L, "d" -> 1L, "e" -> 1L) + val expectedMutations: Seq[Mutation] = + wordCount.map { case (word, count) => BigtableExample.toPutMutation(word, count) } - "BigtableV1WriteExample" should "work" in { + "BigtableWriteExample" should "work" in { JobTest[com.spotify.scio.examples.extra.BigtableWriteExample.type] .args(bigtableOptions :+ "--input=in.txt": _*) .input(TextIO("in.txt"), textIn) - .output( - BigtableIO[(ByteString, Iterable[Mutation])]("my-project", "my-instance", "my-table") - ) { + .output(BigtableIO[Mutation]("my-project", "my-instance", "my-table")) { _ should containInAnyOrder(expectedMutations) } .run() } - def toRow(key: String, value: Long): Row = - Rows.newRow( - ByteString.copyFromUtf8(key), + def toResult(key: String, value: Long): Result = { + val cell = new RowCell( + key.getBytes(StandardCharsets.UTF_8), FAMILY_NAME, COLUMN_QUALIFIER, - ByteString.copyFromUtf8(value.toString) + 0L, + BigInt(value).toByteArray ) + Result.create(Collections.singletonList[Cell](cell)) + } - val rowsIn: Seq[Row] = wordCount.map(kv => toRow(kv._1, kv._2)) - val expectedText: Seq[String] = wordCount.map(kv => kv._1 + ": " + kv._2) + val results: Seq[Result] = wordCount.map { case (word, count) => toResult(word, count) } + val expectedText: Seq[String] = wordCount.map { case (word, count) => s"$word:$count" } "BigtableReadExample" should "work" in { JobTest[com.spotify.scio.examples.extra.BigtableReadExample.type] .args(bigtableOptions :+ "--output=out.txt": _*) - .input(BigtableIO("my-project", "my-instance", "my-table"), rowsIn) + .input(BigtableIO("my-project", "my-instance", "my-table"), results) .output(TextIO("out.txt"))(coll => coll should containInAnyOrder(expectedText)) .run() } diff --git a/scio-google-cloud-platform/src/it/scala/com/spotify/scio/bigtable/BigtableIT.scala b/scio-google-cloud-platform/src/it/scala/com/spotify/scio/bigtable/BigtableIT.scala index 80761f7fee..a85faacef5 100644 --- a/scio-google-cloud-platform/src/it/scala/com/spotify/scio/bigtable/BigtableIT.scala +++ b/scio-google-cloud-platform/src/it/scala/com/spotify/scio/bigtable/BigtableIT.scala @@ -17,152 +17,128 @@ package com.spotify.scio.bigtable -import java.util.UUID - -import com.google.bigtable.admin.v2.{DeleteTableRequest, GetTableRequest, ListTablesRequest} -import com.google.bigtable.v2.{Mutation, Row, RowFilter} -import com.google.cloud.bigtable.config.BigtableOptions -import com.google.cloud.bigtable.grpc._ -import com.google.protobuf.ByteString +import com.google.cloud.bigtable.beam.CloudBigtableConfiguration +import com.google.cloud.bigtable.hbase.BigtableConfiguration import com.spotify.scio._ import com.spotify.scio.testing._ +import org.apache.hadoop.hbase.client.{Delete, Put, Result} +import org.apache.hadoop.hbase.filter.PrefixFilter +import org.apache.hadoop.hbase.{HColumnDescriptor, HTableDescriptor, TableName} import org.joda.time.Duration -import scala.jdk.CollectionConverters._ +import java.nio.charset.StandardCharsets +import java.util.UUID object BigtableIT { - val projectId = "data-integration-test" - val instanceId = "scio-bigtable-it" - val clusterId = "scio-bigtable-it-cluster" - val zoneId = "us-central1-f" - val tableId = "scio-bigtable-it-counts" - val uuid: String = UUID.randomUUID().toString.substring(0, 8) - val testData: Seq[(String, Long)] = - Seq((s"$uuid-key1", 1L), (s"$uuid-key2", 2L), (s"$uuid-key3", 3L)) - - val bigtableOptions: BigtableOptions = BigtableOptions - .builder() - .setProjectId(projectId) - .setInstanceId(instanceId) - .build - - val FAMILY_NAME: String = "count" - val COLUMN_QUALIFIER: ByteString = ByteString.copyFromUtf8("long") - - def toWriteMutation(key: String, value: Long): (ByteString, Iterable[Mutation]) = { - val m = Mutations.newSetCell( - FAMILY_NAME, - COLUMN_QUALIFIER, - ByteString.copyFromUtf8(value.toString), - 0L - ) - (ByteString.copyFromUtf8(key), Iterable(m)) - } - - def toDeleteMutation(key: String): (ByteString, Iterable[Mutation]) = { - val m = Mutations.newDeleteFromRow - (ByteString.copyFromUtf8(key), Iterable(m)) - } - - def fromRow(r: Row): (String, Long) = - (r.getKey.toStringUtf8, r.getValue(FAMILY_NAME, COLUMN_QUALIFIER).get.toStringUtf8.toLong) - - def listTables(client: BigtableTableAdminGrpcClient): Set[String] = { - val instancePath = s"projects/$projectId/instances/$instanceId" - val tables = client.listTables(ListTablesRequest.newBuilder().setParent(instancePath).build) - tables.getTablesList.asScala.map(t => new BigtableTableName(t.getName).getTableId).toSet + val ProjectId = "data-integration-test" + val InstanceId = "scio-bigtable-it" + val ClusterId = "scio-bigtable-it-cluster" + val TableId = "scio-bigtable-it-counts" + val ZoneId = "us-central1-f" + + val Config: CloudBigtableConfiguration = new CloudBigtableConfiguration.Builder() + .withProjectId(ProjectId) + .withInstanceId(InstanceId) + .build() + + val TableFamily = new HColumnDescriptor("count") + val TableColumnQualifier: Array[Byte] = "long".getBytes(StandardCharsets.UTF_8) + val Table: HTableDescriptor = new HTableDescriptor(TableName.valueOf(TableId)) + .addFamily(TableFamily) + + val TestId: String = UUID.randomUUID().toString.substring(0, 8) + val TestData: Seq[(String, Long)] = Seq( + s"$TestId-key1" -> 1L, + s"$TestId-key2" -> 2L, + s"$TestId-key3" -> 3L + ) + def toPutMutation(key: String, value: Long): Put = + new Put(key.getBytes(StandardCharsets.UTF_8)) + .addColumn(TableFamily.getName, TableColumnQualifier, BigInt(value).toByteArray) + + def toDeleteMutation(key: String): Delete = + new Delete(key.getBytes(StandardCharsets.UTF_8)) + .addColumns(TableFamily.getName, TableColumnQualifier) + + def fromResult(r: Result): (String, Long) = { + val key = new String(r.getRow, StandardCharsets.UTF_8) + val value = BigInt(r.getValue(TableFamily.getName, TableColumnQualifier)).toLong + key -> value } } class BigtableIT extends PipelineSpec { import BigtableIT._ - // "Update number of bigtable nodes" should "work" in { - ignore should "update number of bigtable nodes" in { - val bt = new BigtableClusterUtilities(bigtableOptions) - val sc = ScioContext() - sc.updateNumberOfBigtableNodes(projectId, instanceId, 4, Duration.standardSeconds(10)) - sc.getBigtableClusterSizes(projectId, instanceId)(clusterId) shouldBe 4 - bt.getClusterNodeCount(clusterId, zoneId) shouldBe 4 - sc.updateNumberOfBigtableNodes(projectId, instanceId, 3, Duration.standardSeconds(10)) - sc.getBigtableClusterSizes(projectId, instanceId)(clusterId) shouldBe 3 - bt.getClusterNodeCount(clusterId, zoneId) shouldBe 3 - } - "BigtableIO" should "work" in { - TableAdmin.ensureTables(bigtableOptions, Map(tableId -> List(FAMILY_NAME))) + TableAdmin.ensureTables(Config, Seq(Table)) try { // Write rows to table val sc1 = ScioContext() sc1 - .parallelize(testData.map(kv => toWriteMutation(kv._1, kv._2))) - .saveAsBigtable(projectId, instanceId, tableId) + .parallelize(TestData) + .map { case (key, value) => toPutMutation(key, value) } + .saveAsBigtable(ProjectId, InstanceId, TableId) sc1.run().waitUntilFinish() // Read rows back val sc2 = ScioContext() // Filter rows in case there are other keys in the table - val rowFilter = RowFilter - .newBuilder() - .setRowKeyRegexFilter(ByteString.copyFromUtf8(s"$uuid-.*")) - .build() + val filter = new PrefixFilter(TestId.getBytes(StandardCharsets.UTF_8)) sc2 - .bigtable(projectId, instanceId, tableId, rowFilter = rowFilter) - .map(fromRow) should containInAnyOrder(testData) + .bigtable(ProjectId, InstanceId, TableId, filter = filter) + .map(fromResult) should containInAnyOrder(TestData) sc2.run().waitUntilFinish() - } catch { - case e: Throwable => throw e } finally { // Delete rows afterwards val sc = ScioContext() - sc.parallelize(testData.map(kv => toDeleteMutation(kv._1))) - .saveAsBigtable(projectId, instanceId, tableId) + sc.parallelize(TestData) + .keys + .map(toDeleteMutation) + .saveAsBigtable(ProjectId, InstanceId, TableId) sc.run().waitUntilFinish() - () } } + behavior of "InstanceAdmin" + ignore should "work" in { + InstanceAdmin.resizeClusters(Config, 3, Duration.standardSeconds(10)) + InstanceAdmin.getCluster(Config, ClusterId) shouldBe 2 + + InstanceAdmin.resizeClusters(Config, Set(ClusterId), 1, Duration.standardSeconds(10)) + InstanceAdmin.getCluster(Config, ClusterId) shouldBe 1 + } + "TableAdmin" should "work" in { - val tables = Map( - s"scio-bigtable-empty-table-$uuid" -> List(), - s"scio-bigtable-one-cf-table-$uuid" -> List("colfam1"), - s"scio-bigtable-two-cf-table-$uuid" -> List("colfam1", "colfam2") + val tables = Seq( + new HTableDescriptor(TableName.valueOf(s"scio-bigtable-empty-table-$TestId")), + new HTableDescriptor(TableName.valueOf(s"scio-bigtable-one-cf-table-$TestId")) + .addFamily(new HColumnDescriptor("colfam1")), + new HTableDescriptor(TableName.valueOf(s"scio-bigtable-two-cf-table-$TestId")) + .addFamily(new HColumnDescriptor("colfam1")) + .addFamily(new HColumnDescriptor("colfam2")) ) - val channel = ChannelPoolCreator.createPool(bigtableOptions) - val executorService = BigtableSessionSharedThreadPools.getInstance().getRetryExecutor - val client = new BigtableTableAdminGrpcClient(channel, executorService, bigtableOptions) - val instancePath = s"projects/$projectId/instances/$instanceId" - val tableIds = tables.keys.toSet - def tablePath(table: String): String = s"$instancePath/tables/$table" - def deleteTable(table: String): Unit = - client.deleteTable(DeleteTableRequest.newBuilder().setName(tablePath(table)).build) - - // Delete any tables that could be left around from previous IT run. - val oldTables = listTables(client).intersect(tableIds) - oldTables.foreach(deleteTable) - - // Ensure that the tables don't exist now - listTables(client).intersect(tableIds) shouldBe empty - - // Run UUT - TableAdmin.ensureTables(bigtableOptions, tables) - - // Tables must exist - listTables(client).intersect(tableIds) shouldEqual tableIds - - // Assert Column families exist - for ((table, columnFamilies) <- tables) { - val tableInfo = client.getTable( - GetTableRequest - .newBuilder() - .setName(tablePath(table)) - .build - ) - val actualColumnFamilies = tableInfo.getColumnFamiliesMap.asScala.keys - actualColumnFamilies should contain theSameElementsAs columnFamilies - } - // Clean up and delete - tables.keys.foreach(deleteTable) + val connection = BigtableConfiguration.connect(Config.toHBaseConfig) + val client = connection.getAdmin + try { + // Delete any tables that could be left around from previous IT run. + client + .listTableNames(s".*$TestId".r.pattern) + .foreach(client.deleteTable) + + // Ensure that the tables don't exist now + client.listTableNames(s".*$TestId".r.pattern) shouldBe empty + + // Run UT + TableAdmin.ensureTables(Config, tables) + + // Tables must exist with exact settings + val actualTables = client.listTables(s".*$TestId".r.pattern) + actualTables should contain theSameElementsAs tables + } finally { + tables.foreach(t => client.deleteTable(t.getTableName)) + connection.close() + } } } diff --git a/scio-google-cloud-platform/src/main/java/com/spotify/scio/bigtable/BigtableBulkWriter.java b/scio-google-cloud-platform/src/main/java/com/spotify/scio/bigtable/BigtableBulkWriter.java deleted file mode 100644 index 7b13b87a51..0000000000 --- a/scio-google-cloud-platform/src/main/java/com/spotify/scio/bigtable/BigtableBulkWriter.java +++ /dev/null @@ -1,229 +0,0 @@ -/* - * Copyright 2018 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.bigtable; - -import com.google.bigtable.v2.Mutation; -import com.google.cloud.bigtable.config.BigtableOptions; -import com.google.protobuf.ByteString; -import java.io.IOException; -import java.util.ArrayList; -import java.util.List; -import java.util.concurrent.ConcurrentLinkedQueue; -import java.util.concurrent.ThreadLocalRandom; -import org.apache.beam.sdk.io.gcp.bigtable.BigtableServiceHelper; -import org.apache.beam.sdk.transforms.DoFn; -import org.apache.beam.sdk.transforms.GroupByKey; -import org.apache.beam.sdk.transforms.PTransform; -import org.apache.beam.sdk.transforms.ParDo; -import org.apache.beam.sdk.transforms.display.DisplayData; -import org.apache.beam.sdk.transforms.windowing.AfterProcessingTime; -import org.apache.beam.sdk.transforms.windowing.GlobalWindows; -import org.apache.beam.sdk.transforms.windowing.Repeatedly; -import org.apache.beam.sdk.transforms.windowing.Window; -import org.apache.beam.sdk.values.KV; -import org.apache.beam.sdk.values.PCollection; -import org.apache.beam.sdk.values.PDone; -import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.annotations.VisibleForTesting; -import org.joda.time.Duration; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -public class BigtableBulkWriter - extends PTransform>>, PDone> { - - private static final Logger LOG = LoggerFactory.getLogger(BigtableBulkWriter.class); - - private final BigtableOptions bigtableOptions; - private final String tableName; - private final int numOfShards; - private final Duration flushInterval; - - public BigtableBulkWriter( - final String tableName, - final BigtableOptions bigtableOptions, - final int numOfShards, - final Duration flushInterval) { - this.bigtableOptions = bigtableOptions; - this.tableName = tableName; - this.numOfShards = numOfShards; - this.flushInterval = flushInterval; - } - - @Override - public PDone expand(PCollection>> input) { - createBulkShards(input, numOfShards, flushInterval) - .apply("Bigtable BulkWrite", ParDo.of(new BigtableBulkWriterFn())); - return PDone.in(input.getPipeline()); - } - - @VisibleForTesting - static PCollection>>> createBulkShards( - final PCollection>> input, - final int numOfShards, - final Duration flushInterval) { - return input - .apply("Assign To Shard", ParDo.of(new AssignToShard(numOfShards))) - .apply( - "Window", - Window.>>>into(new GlobalWindows()) - .triggering( - Repeatedly.forever( - AfterProcessingTime.pastFirstElementInPane().plusDelayOf(flushInterval))) - .discardingFiredPanes() - .withAllowedLateness(Duration.ZERO)) - .apply("Group By Shard", GroupByKey.create()) - .apply( - "Gets Mutations", - ParDo.of( - new DoFn< - KV>>>, - Iterable>>>() { - @ProcessElement - public void process( - @Element KV>>> element, - OutputReceiver>>> out) { - out.output(element.getValue()); - } - })); - } - - private class BigtableBulkWriterFn - extends DoFn>>, Void> { - - private BigtableServiceHelper.Writer bigtableWriter; - private long recordsWritten; - private final ConcurrentLinkedQueue failures; - - public BigtableBulkWriterFn() { - this.failures = new ConcurrentLinkedQueue<>(); - } - - @StartBundle - public void startBundle(StartBundleContext c) throws IOException { - if (bigtableWriter == null) { - bigtableWriter = - new BigtableServiceHelper(bigtableOptions, c.getPipelineOptions()) - .openForWriting(tableName); - } - recordsWritten = 0; - } - - @ProcessElement - public void processElement(@Element Iterable>> element) - throws Exception { - checkForFailures(failures); - for (KV> r : element) { - bigtableWriter - .writeRecord(r) - .whenComplete( - (mutationResult, exception) -> { - if (exception != null) { - failures.add(new BigtableWriteException(r, exception)); - } - }); - ++recordsWritten; - } - } - - @FinishBundle - public void finishBundle() throws Exception { - bigtableWriter.flush(); - checkForFailures(failures); - LOG.debug("Wrote {} records", recordsWritten); - } - - @Teardown - public void tearDown() throws Exception { - if (bigtableWriter != null) { - bigtableWriter.close(); - bigtableWriter = null; - } - } - - @Override - public void populateDisplayData(DisplayData.Builder builder) { - super.populateDisplayData(builder); - builder.add(DisplayData.item("Records Written", recordsWritten)); - } - - /** If any write has asynchronously failed, fail the bundle with a useful error. */ - private void checkForFailures(final ConcurrentLinkedQueue failures) - throws IOException { - // Note that this function is never called by multiple threads and is the only place that - // we remove from failures, so this code is safe. - if (failures.isEmpty()) { - return; - } - - StringBuilder logEntry = new StringBuilder(); - int i = 0; - List suppressed = new ArrayList<>(); - for (; i < 10 && !failures.isEmpty(); ++i) { - BigtableWriteException exc = failures.remove(); - logEntry.append("\n").append(exc.getMessage()); - if (exc.getCause() != null) { - logEntry.append(": ").append(exc.getCause().getMessage()); - } - suppressed.add(exc); - } - String message = - String.format( - "At least %d errors occurred writing to Bigtable. First %d errors: %s", - i + failures.size(), i, logEntry.toString()); - LOG.error(message); - IOException exception = new IOException(message); - for (BigtableWriteException e : suppressed) { - exception.addSuppressed(e); - } - throw exception; - } - - /** An exception that puts information about the failed record being written in its message. */ - class BigtableWriteException extends IOException { - - public BigtableWriteException( - final KV> record, Throwable cause) { - super( - String.format( - "Error mutating row %s with mutations %s", - record.getKey().toStringUtf8(), record.getValue()), - cause); - } - } - } - - static class AssignToShard - extends DoFn< - KV>, KV>>> { - - private final int numOfShards; - - AssignToShard(final int numOfShards) { - this.numOfShards = numOfShards; - } - - @ProcessElement - public void processElement( - @Element KV> element, - OutputReceiver>>> out) { - // assign this element to a random shard - final long shard = ThreadLocalRandom.current().nextLong(numOfShards); - out.output(KV.of(shard, element)); - } - } -} diff --git a/scio-google-cloud-platform/src/main/java/com/spotify/scio/bigtable/BigtableDoFn.java b/scio-google-cloud-platform/src/main/java/com/spotify/scio/bigtable/BigtableDoFn.java deleted file mode 100644 index 0248693fc8..0000000000 --- a/scio-google-cloud-platform/src/main/java/com/spotify/scio/bigtable/BigtableDoFn.java +++ /dev/null @@ -1,109 +0,0 @@ -/* - * Copyright 2017 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.bigtable; - -import com.google.cloud.bigtable.config.BigtableOptions; -import com.google.cloud.bigtable.grpc.BigtableSession; -import com.google.common.util.concurrent.ListenableFuture; -import com.spotify.scio.transforms.BaseAsyncLookupDoFn; -import com.spotify.scio.transforms.GuavaAsyncLookupDoFn; -import java.io.IOException; -import org.apache.beam.sdk.transforms.DoFn; - -/** - * A {@link DoFn} that performs asynchronous lookup using Google Cloud Bigtable. - * - * @param input element type. - * @param Bigtable lookup value type. - */ -public abstract class BigtableDoFn extends GuavaAsyncLookupDoFn { - - private final BigtableOptions options; - - /** Perform asynchronous Bigtable lookup. */ - public abstract ListenableFuture asyncLookup(BigtableSession session, A input); - - /** - * Create a {@link BigtableDoFn} instance. - * - * @param options Bigtable options. - */ - public BigtableDoFn(BigtableOptions options) { - this(options, 1000); - } - - /** - * Create a {@link BigtableDoFn} instance. - * - * @param options Bigtable options. - * @param maxPendingRequests maximum number of pending requests on every cloned DoFn. This - * prevents runner from timing out and retrying bundles. - */ - public BigtableDoFn(BigtableOptions options, int maxPendingRequests) { - this(options, maxPendingRequests, new BaseAsyncLookupDoFn.NoOpCacheSupplier<>()); - } - - /** - * Create a {@link BigtableDoFn} instance. - * - * @param options Bigtable options. - * @param maxPendingRequests maximum number of pending requests on every cloned DoFn. This - * prevents runner from timing out and retrying bundles. - * @param cacheSupplier supplier for lookup cache. - */ - public BigtableDoFn( - BigtableOptions options, - int maxPendingRequests, - BaseAsyncLookupDoFn.CacheSupplier cacheSupplier) { - super(maxPendingRequests, cacheSupplier); - this.options = options; - } - - /** - * Create a {@link BigtableDoFn} instance. - * - * @param options Bigtable options. - * @param maxPendingRequests maximum number of pending requests on every cloned DoFn. This - * prevents runner from timing out and retrying bundles. - * @param deduplicate if an attempt should be made to de-duplicate simultaneous requests for the - * same input - * @param cacheSupplier supplier for lookup cache. - */ - public BigtableDoFn( - BigtableOptions options, - int maxPendingRequests, - boolean deduplicate, - BaseAsyncLookupDoFn.CacheSupplier cacheSupplier) { - super(maxPendingRequests, deduplicate, cacheSupplier); - this.options = options; - } - - @Override - public ResourceType getResourceType() { - // BigtableSession is backed by a gRPC thread safe client - return ResourceType.PER_INSTANCE; - } - - protected BigtableSession newClient() { - try { - return new BigtableSession(options); - } catch (IOException e) { - throw new RuntimeException(e); - } - } -} diff --git a/scio-google-cloud-platform/src/main/java/com/spotify/scio/bigtable/BigtableUtil.java b/scio-google-cloud-platform/src/main/java/com/spotify/scio/bigtable/BigtableUtil.java deleted file mode 100644 index 752dc41976..0000000000 --- a/scio-google-cloud-platform/src/main/java/com/spotify/scio/bigtable/BigtableUtil.java +++ /dev/null @@ -1,166 +0,0 @@ -/* - * Copyright 2016 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.bigtable; - -import com.google.bigtable.admin.v2.Cluster; -import com.google.bigtable.admin.v2.ListClustersRequest; -import com.google.bigtable.admin.v2.ListClustersResponse; -import com.google.cloud.bigtable.config.BigtableOptions; -import com.google.cloud.bigtable.grpc.BigtableClusterUtilities; -import com.google.cloud.bigtable.grpc.BigtableInstanceClient; -import com.google.cloud.bigtable.grpc.BigtableInstanceGrpcClient; -import com.google.cloud.bigtable.grpc.io.ChannelPool; -import java.io.IOException; -import java.security.GeneralSecurityException; -import java.util.Collections; -import java.util.List; -import java.util.Map; -import java.util.Set; -import java.util.stream.Collectors; -import org.joda.time.Duration; -import org.joda.time.format.PeriodFormatter; -import org.joda.time.format.PeriodFormatterBuilder; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -/** Utilities to deal with Bigtable. */ -public final class BigtableUtil { - - private BigtableUtil() {} - - private static final Logger LOG = LoggerFactory.getLogger(BigtableUtil.class); - - private static final PeriodFormatter formatter = - new PeriodFormatterBuilder() - .appendDays() - .appendSuffix("d") - .appendHours() - .appendSuffix("h") - .appendMinutes() - .appendSuffix("m") - .appendSeconds() - .appendSuffix("s") - .toFormatter(); - - /** - * Updates all clusters within the specified Bigtable instance to a specified number of nodes. - * Useful for increasing the number of nodes at the beginning of a job and decreasing it at the - * end to lower costs yet still get high throughput during bulk ingests/dumps. - * - * @param bigtableOptions Bigtable Options - * @param numberOfNodes New number of nodes in the cluster - * @param sleepDuration How long to sleep after updating the number of nodes. Google recommends at - * least 20 minutes before the new nodes are fully functional - * @throws IOException If setting up channel pool fails - * @throws InterruptedException If sleep fails - */ - public static void updateNumberOfBigtableNodes( - final BigtableOptions bigtableOptions, final int numberOfNodes, final Duration sleepDuration) - throws IOException, InterruptedException { - updateNumberOfBigtableNodes( - bigtableOptions, numberOfNodes, sleepDuration, Collections.emptySet()); - } - - /** - * Updates all clusters within the specified Bigtable instance to a specified number of nodes. - * Useful for increasing the number of nodes at the beginning of a job and decreasing it at the - * end to lower costs yet still get high throughput during bulk ingests/dumps. - * - * @param bigtableOptions Bigtable Options - * @param numberOfNodes New number of nodes in the cluster - * @param sleepDuration How long to sleep after updating the number of nodes. Google recommends at - * least 20 minutes before the new nodes are fully functional - * @param clusterNames Names of clusters to be updated, all if empty - * @throws IOException If setting up channel pool fails - * @throws InterruptedException If sleep fails - */ - public static void updateNumberOfBigtableNodes( - final BigtableOptions bigtableOptions, - final int numberOfNodes, - final Duration sleepDuration, - final Set clusterNames) - throws IOException, InterruptedException { - final ChannelPool channelPool = ChannelPoolCreator.createPool(bigtableOptions); - - try { - final BigtableInstanceClient bigtableInstanceClient = - new BigtableInstanceGrpcClient(channelPool); - - final String instanceName = bigtableOptions.getInstanceName().toString(); - - // Fetch clusters in Bigtable instance - final ListClustersRequest clustersRequest = - ListClustersRequest.newBuilder().setParent(instanceName).build(); - final ListClustersResponse clustersResponse = - bigtableInstanceClient.listCluster(clustersRequest); - final List clustersToUpdate = - clusterNames.isEmpty() - ? clustersResponse.getClustersList() - : clustersResponse.getClustersList().stream() - .filter(c -> clusterNames.contains(shorterName(c.getName()))) - .collect(Collectors.toList()); - - // For each cluster update the number of nodes - for (Cluster cluster : clustersToUpdate) { - final Cluster updatedCluster = - Cluster.newBuilder().setName(cluster.getName()).setServeNodes(numberOfNodes).build(); - LOG.info("Updating number of nodes to {} for cluster {}", numberOfNodes, cluster.getName()); - bigtableInstanceClient.updateCluster(updatedCluster); - } - - // Wait for the new nodes to be provisioned - if (sleepDuration.getMillis() > 0) { - LOG.info("Sleeping for {} after update", formatter.print(sleepDuration.toPeriod())); - Thread.sleep(sleepDuration.getMillis()); - } - } finally { - channelPool.shutdownNow(); - } - } - - /** - * Get size of all clusters for specified Bigtable instance. - * - * @param projectId GCP projectId - * @param instanceId Bigtable instanceId - * @return map of clusterId to its number of nodes - * @throws IOException If setting up channel pool fails - * @throws GeneralSecurityException If security-related exceptions occurs - */ - public static Map getClusterSizes( - final String projectId, final String instanceId) - throws IOException, GeneralSecurityException { - try (BigtableClusterUtilities clusterUtil = - BigtableClusterUtilities.forInstance(projectId, instanceId)) { - return Collections.unmodifiableMap( - clusterUtil.getClusters().getClustersList().stream() - .collect( - Collectors.toMap( - cn -> cn.getName().substring(cn.getName().indexOf("/clusters/") + 10), - Cluster::getServeNodes))); - } - } - - static String shorterName(String name) { - if (name.lastIndexOf('/') != -1) { - return name.substring(name.lastIndexOf('/') + 1, name.length()); - } else { - return name; - } - } -} diff --git a/scio-google-cloud-platform/src/main/java/com/spotify/scio/bigtable/ChannelPoolCreator.java b/scio-google-cloud-platform/src/main/java/com/spotify/scio/bigtable/ChannelPoolCreator.java deleted file mode 100644 index 3303c1d6eb..0000000000 --- a/scio-google-cloud-platform/src/main/java/com/spotify/scio/bigtable/ChannelPoolCreator.java +++ /dev/null @@ -1,59 +0,0 @@ -/* - * Copyright 2016 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.bigtable; - -import com.google.cloud.bigtable.config.BigtableOptions; -import com.google.cloud.bigtable.grpc.BigtableSession; -import com.google.cloud.bigtable.grpc.io.ChannelPool; -import com.google.cloud.bigtable.grpc.io.CredentialInterceptorCache; -import io.grpc.ClientInterceptor; -import java.io.IOException; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -public class ChannelPoolCreator { - private static final Logger LOG = LoggerFactory.getLogger(ChannelPoolCreator.class); - - private static ClientInterceptor[] getClientInterceptors(final BigtableOptions options) { - try { - final ClientInterceptor interceptor = - CredentialInterceptorCache.getInstance() - .getCredentialsInterceptor(options.getCredentialOptions()); - // If credentials are unset (i.e. via local emulator), CredentialsInterceptor will return null - if (interceptor == null) { - return new ClientInterceptor[] {}; - } else { - return new ClientInterceptor[] {interceptor}; - } - } catch (Exception e) { - LOG.error( - "Failed to get credentials interceptor. No interceptor will be used for the channel.", e); - return new ClientInterceptor[] {}; - } - } - - public static ChannelPool createPool(final BigtableOptions options) throws IOException { - final ClientInterceptor[] interceptors = getClientInterceptors(options); - - return new ChannelPool( - () -> - BigtableSession.createNettyChannel( - options.getAdminHost(), options, false, interceptors), - 1); - } -} diff --git a/scio-google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableServiceHelper.java b/scio-google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableServiceHelper.java deleted file mode 100644 index 9cc9b5dcc6..0000000000 --- a/scio-google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableServiceHelper.java +++ /dev/null @@ -1,41 +0,0 @@ -/* - * Copyright 2018 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 org.apache.beam.sdk.io.gcp.bigtable; - -import com.google.cloud.bigtable.config.BigtableOptions; -import com.google.cloud.bigtable.data.v2.BigtableDataSettings; -import java.io.IOException; -import org.apache.beam.sdk.options.PipelineOptions; - -/** Wrap {@link BigtableServiceImpl} and expose package private methods. */ -public class BigtableServiceHelper extends BigtableServiceImpl { - - private static final BigtableConfig EMPTY_CONFIG = BigtableConfig.builder().build(); - - public BigtableServiceHelper(BigtableOptions bigtableOptions, PipelineOptions pipelineOptions) - throws IOException { - super(translateToVeneerSettings(bigtableOptions, pipelineOptions)); - } - - private static BigtableDataSettings translateToVeneerSettings( - BigtableOptions bigtableOptions, PipelineOptions pipelineOptions) throws IOException { - final BigtableConfig config = - BigtableConfigTranslator.translateToBigtableConfig(EMPTY_CONFIG, bigtableOptions); - return BigtableConfigTranslator.translateToVeneerSettings(config, pipelineOptions); - } -} diff --git a/scio-google-cloud-platform/src/main/scala/com/spotify/scio/bigtable/BigTableIO.scala b/scio-google-cloud-platform/src/main/scala/com/spotify/scio/bigtable/BigTableIO.scala index 2c654e2100..ff731a4203 100644 --- a/scio-google-cloud-platform/src/main/scala/com/spotify/scio/bigtable/BigTableIO.scala +++ b/scio-google-cloud-platform/src/main/scala/com/spotify/scio/bigtable/BigTableIO.scala @@ -17,21 +17,21 @@ package com.spotify.scio.bigtable -import com.google.bigtable.v2._ -import com.google.cloud.bigtable.config.BigtableOptions -import com.google.protobuf.ByteString +import com.google.cloud.bigtable.beam.{ + CloudBigtableIO, + CloudBigtableScanConfiguration, + CloudBigtableTableConfiguration +} import com.spotify.scio.ScioContext -import com.spotify.scio.coders.{Coder, CoderMaterializer} -import com.spotify.scio.io.{EmptyTap, EmptyTapOf, ScioIO, Tap, TestIO} +import com.spotify.scio.coders.CoderMaterializer +import com.spotify.scio.io._ import com.spotify.scio.values.SCollection -import org.apache.beam.sdk.io.gcp.{bigtable => beam} +import org.apache.beam.sdk.io.Read import org.apache.beam.sdk.io.range.ByteKeyRange -import org.apache.beam.sdk.transforms.SerializableFunction -import org.apache.beam.sdk.values.KV -import org.joda.time.Duration +import org.apache.hadoop.hbase.client.{Mutation, Result, Scan} +import org.apache.hadoop.hbase.filter.Filter -import scala.jdk.CollectionConverters._ -import com.spotify.scio.io.TapT +import scala.util.chaining._ sealed trait BigtableIO[T] extends ScioIO[T] { final override val tapT: TapT.Aux[T, Nothing] = EmptyTapOf[T] @@ -45,38 +45,30 @@ object BigtableIO { } } -final case class BigtableRead(bigtableOptions: BigtableOptions, tableId: String) - extends BigtableIO[Row] { +final case class BigtableRead(config: CloudBigtableTableConfiguration) extends BigtableIO[Result] { override type ReadP = BigtableRead.ReadParam override type WriteP = Nothing override def testId: String = - s"BigtableIO(${bigtableOptions.getProjectId}\t${bigtableOptions.getInstanceId}\t$tableId)" - - override protected def read(sc: ScioContext, params: ReadP): SCollection[Row] = { - val coder = CoderMaterializer.beam(sc, Coder.protoMessageCoder[Row]) - val opts = bigtableOptions // defeat closure - var read = beam.BigtableIO - .read() - .withProjectId(bigtableOptions.getProjectId) - .withInstanceId(bigtableOptions.getInstanceId) - .withTableId(tableId) - .withBigtableOptionsConfigurator( - new SerializableFunction[BigtableOptions.Builder, BigtableOptions.Builder] { - override def apply(input: BigtableOptions.Builder): BigtableOptions.Builder = - opts.toBuilder + s"BigtableIO(${config.getProjectId}\t${config.getInstanceId}\t${config.getTableId})" + + override protected def read(sc: ScioContext, params: ReadP): SCollection[Result] = { + val coder = CoderMaterializer.beam(sc, BigtableCoders.resultCoder) + val scan = new Scan() + .pipe(s => + Option(params.keyRange).fold(s) { kr => + s.withStartRow(kr.getStartKey.getBytes).withStopRow(kr.getEndKey.getBytes) } ) - if (!params.keyRanges.isEmpty) { - read = read.withKeyRanges(params.keyRanges.asJava) - } - if (params.rowFilter != null) { - read = read.withRowFilter(params.rowFilter) - } + .pipe(s => Option(params.filter).fold(s)(s.setFilter)) + + val scanConfig = CloudBigtableScanConfiguration.fromConfig(config, scan) + val read = Read.from(CloudBigtableIO.read(scanConfig)) + sc.applyTransform(read).setCoder(coder) } - override protected def write(data: SCollection[Row], params: WriteP): Tap[Nothing] = + override protected def write(data: SCollection[Result], params: WriteP): Tap[Nothing] = throw new UnsupportedOperationException( "BigtableRead is read-only, use Mutation to write to Bigtable" ) @@ -87,75 +79,50 @@ final case class BigtableRead(bigtableOptions: BigtableOptions, tableId: String) object BigtableRead { object ReadParam { - val DefaultKeyRanges: Seq[ByteKeyRange] = Seq.empty[ByteKeyRange] - val DefaultRowFilter: RowFilter = null - - def apply(keyRange: ByteKeyRange) = new ReadParam(Seq(keyRange)) - - def apply(keyRange: ByteKeyRange, rowFilter: RowFilter): ReadParam = - new ReadParam(Seq(keyRange), rowFilter) + val DefaultKeyRange: ByteKeyRange = null + val DefaultFilter: Filter = null } final case class ReadParam private ( - keyRanges: Seq[ByteKeyRange] = ReadParam.DefaultKeyRanges, - rowFilter: RowFilter = ReadParam.DefaultRowFilter + keyRange: ByteKeyRange = ReadParam.DefaultKeyRange, + filter: Filter = ReadParam.DefaultFilter ) final def apply(projectId: String, instanceId: String, tableId: String): BigtableRead = { - val bigtableOptions = BigtableOptions - .builder() - .setProjectId(projectId) - .setInstanceId(instanceId) + val config = new CloudBigtableTableConfiguration.Builder() + .withProjectId(projectId) + .withInstanceId(instanceId) + .withTableId(tableId) .build - BigtableRead(bigtableOptions, tableId) + BigtableRead(config) } } -final case class BigtableWrite[T <: Mutation](bigtableOptions: BigtableOptions, tableId: String) - extends BigtableIO[(ByteString, Iterable[T])] { +final case class BigtableWrite[T <: Mutation](config: CloudBigtableTableConfiguration) + extends BigtableIO[T] { override type ReadP = Nothing - override type WriteP = BigtableWrite.WriteParam + override type WriteP = Unit override def testId: String = - s"BigtableIO(${bigtableOptions.getProjectId}\t${bigtableOptions.getInstanceId}\t$tableId)" + s"BigtableIO(${config.getProjectId}\t${config.getInstanceId}\t${config.getTableId})" override protected def read( sc: ScioContext, params: ReadP - ): SCollection[(ByteString, Iterable[T])] = + ): SCollection[T] = throw new UnsupportedOperationException( "BigtableWrite is write-only, use Row to read from Bigtable" ) override protected def write( - data: SCollection[(ByteString, Iterable[T])], + data: SCollection[T], params: WriteP ): Tap[Nothing] = { - val sink = - params match { - case BigtableWrite.Default => - val opts = bigtableOptions // defeat closure - beam.BigtableIO - .write() - .withProjectId(bigtableOptions.getProjectId) - .withInstanceId(bigtableOptions.getInstanceId) - .withTableId(tableId) - .withBigtableOptionsConfigurator( - new SerializableFunction[BigtableOptions.Builder, BigtableOptions.Builder] { - override def apply(input: BigtableOptions.Builder): BigtableOptions.Builder = - opts.toBuilder - } - ) - case BigtableWrite.Bulk(numOfShards, flushInterval) => - new BigtableBulkWriter(tableId, bigtableOptions, numOfShards, flushInterval) - } - data.transform_("Bigtable write") { coll => - coll - .map { case (key, value) => - KV.of(key, value.asJava.asInstanceOf[java.lang.Iterable[Mutation]]) - } - .applyInternal(sink) - } + CloudBigtableIO.writeToTable(config) + + data + .covary[Mutation] + .applyInternal("Bigtable write", CloudBigtableIO.writeToTable(config)) EmptyTap } @@ -164,28 +131,17 @@ final case class BigtableWrite[T <: Mutation](bigtableOptions: BigtableOptions, } object BigtableWrite { - sealed trait WriteParam - object Default extends WriteParam - - object Bulk { - private[bigtable] val DefaultFlushInterval = Duration.standardSeconds(1) - } - - final case class Bulk private ( - numOfShards: Int, - flushInterval: Duration = Bulk.DefaultFlushInterval - ) extends WriteParam final def apply[T <: Mutation]( projectId: String, instanceId: String, tableId: String ): BigtableWrite[T] = { - val bigtableOptions = BigtableOptions - .builder() - .setProjectId(projectId) - .setInstanceId(instanceId) + val config = new CloudBigtableTableConfiguration.Builder() + .withProjectId(projectId) + .withInstanceId(instanceId) + .withTableId(tableId) .build - BigtableWrite[T](bigtableOptions, tableId) + BigtableWrite[T](config) } } diff --git a/scio-google-cloud-platform/src/main/scala/com/spotify/scio/bigtable/BigtableCoders.scala b/scio-google-cloud-platform/src/main/scala/com/spotify/scio/bigtable/BigtableCoders.scala new file mode 100644 index 0000000000..8984493a1c --- /dev/null +++ b/scio-google-cloud-platform/src/main/scala/com/spotify/scio/bigtable/BigtableCoders.scala @@ -0,0 +1,29 @@ +package com.spotify.scio.bigtable + +import com.spotify.scio.coders.Coder +import com.spotify.scio.util.ScioUtil +import org.apache.beam.sdk.io.hbase.HBaseCoderProviderRegistrar +import org.apache.beam.sdk.values.TypeDescriptor +import org.apache.hadoop.hbase.client.{Mutation, Result} + +import java.util.Collections +import scala.jdk.CollectionConverters._ +import scala.reflect.ClassTag + +trait BigtableCoders { + + // FIXME hbase coders are protected. This access is risky + private val mutationProvider :: _ :: resultProvider :: Nil = + new HBaseCoderProviderRegistrar().getCoderProviders.asScala.toList + + implicit def mutationCoder[T <: Mutation: ClassTag]: Coder[T] = { + val td = TypeDescriptor.of(ScioUtil.classOf[T]) + Coder.beam(mutationProvider.coderFor(td, Collections.emptyList())) + } + + implicit val resultCoder: Coder[Result] = + Coder.beam(resultProvider.coderFor(TypeDescriptor.of(classOf[Result]), Collections.emptyList())) + +} + +object BigtableCoders extends BigtableCoders diff --git a/scio-google-cloud-platform/src/main/scala/com/spotify/scio/bigtable/InstanceAdmin.scala b/scio-google-cloud-platform/src/main/scala/com/spotify/scio/bigtable/InstanceAdmin.scala new file mode 100644 index 0000000000..67791f11d1 --- /dev/null +++ b/scio-google-cloud-platform/src/main/scala/com/spotify/scio/bigtable/InstanceAdmin.scala @@ -0,0 +1,88 @@ +package com.spotify.scio.bigtable + +import com.google.bigtable.repackaged.com.google.cloud.bigtable.admin.v2.BigtableInstanceAdminClient +import com.google.bigtable.repackaged.com.google.cloud.bigtable.admin.v2.models.Cluster +import com.google.cloud.bigtable.beam.CloudBigtableConfiguration +import com.google.cloud.bigtable.hbase.BigtableConfiguration +import com.google.cloud.bigtable.hbase.wrappers.veneer.BigtableHBaseVeneerSettings +import org.apache.hadoop.hbase.client.AbstractBigtableConnection +import org.joda.time.Duration +import org.joda.time.format.PeriodFormatterBuilder +import org.slf4j.{Logger, LoggerFactory} + +import scala.jdk.CollectionConverters._ +import scala.util.chaining._ +object InstanceAdmin { + + private val Formatter = new PeriodFormatterBuilder().appendDays + .appendSuffix("d") + .appendHours + .appendSuffix("h") + .appendMinutes + .appendSuffix("m") + .appendSeconds + .appendSuffix("s") + .toFormatter + + private val log: Logger = LoggerFactory.getLogger(TableAdmin.getClass) + + private def execute[A]( + config: CloudBigtableConfiguration + )(f: BigtableInstanceAdminClient => A): A = { + val connection = BigtableConfiguration.connect(config.toHBaseConfig) + try { + val settings = connection + .asInstanceOf[AbstractBigtableConnection] + .getBigtableSettings + .asInstanceOf[BigtableHBaseVeneerSettings] + .getInstanceAdminSettings + + f(BigtableInstanceAdminClient.create(settings)) + } finally { + connection.close() + } + } + + def getCluster(config: CloudBigtableConfiguration, clusterId: String): Cluster = + execute(config)(_.getCluster(config.getInstanceId, clusterId)) + + def listClusters(config: CloudBigtableConfiguration): Iterable[Cluster] = + execute(config)(_.listClusters(config.getInstanceId).asScala) + + def resizeClusters( + config: CloudBigtableConfiguration, + numServeNodes: Int, + sleepDuration: Duration + ): Unit = resizeClusters(config, None, numServeNodes, sleepDuration) + + def resizeClusters( + config: CloudBigtableConfiguration, + clusterIds: Set[String], + numServeNodes: Int, + sleepDuration: Duration + ): Unit = resizeClusters(config, Some(clusterIds), numServeNodes, sleepDuration) + + private def resizeClusters( + config: CloudBigtableConfiguration, + clusterIds: Option[Set[String]], + numServeNodes: Int, + sleepDuration: Duration + ): Unit = + execute(config) { client => + client + .listClusters(config.getInstanceId) + .asScala + .pipe(cs => clusterIds.fold(cs)(ids => cs.filter(c => ids.contains(c.getId)))) + .foreach { c => + // For each cluster update the number of nodes + log.info("Updating number of nodes to {} for cluster {}", numServeNodes, c.getId); + client.resizeCluster(c.getInstanceId, c.getId, numServeNodes) + } + + // Wait for the new nodes to be provisioned + if (sleepDuration.isLongerThan(Duration.ZERO)) { + log.info("Sleeping for {} after update", Formatter.print(sleepDuration.toPeriod())); + Thread.sleep(sleepDuration.getMillis); + } + } +} diff --git a/scio-google-cloud-platform/src/main/scala/com/spotify/scio/bigtable/Mutations.scala b/scio-google-cloud-platform/src/main/scala/com/spotify/scio/bigtable/Mutations.scala deleted file mode 100644 index d66ccdcec9..0000000000 --- a/scio-google-cloud-platform/src/main/scala/com/spotify/scio/bigtable/Mutations.scala +++ /dev/null @@ -1,102 +0,0 @@ -/* - * Copyright 2019 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.bigtable - -import java.util.concurrent.TimeUnit - -import com.google.bigtable.v2.Mutation.{DeleteFromColumn, DeleteFromFamily, DeleteFromRow, SetCell} -import com.google.bigtable.v2._ -import com.google.protobuf.ByteString - -/** Helper methods for `Mutation`. */ -object Mutations { - - /** New `SetCell` mutation using the current timestamp. */ - def newSetCell(familyName: String, columnQualifier: ByteString, value: ByteString): Mutation = - Mutation - .newBuilder() - .setSetCell( - SetCell - .newBuilder() - .setFamilyName(familyName) - .setColumnQualifier(columnQualifier) - .setValue(value) - .setTimestampMicros(TimeUnit.MILLISECONDS.toMicros(System.currentTimeMillis())) - ) - .build() - - /** New `SetCell` mutation. */ - def newSetCell( - familyName: String, - columnQualifier: ByteString, - value: ByteString, - timestampMicros: Long - ): Mutation = - Mutation - .newBuilder() - .setSetCell( - SetCell - .newBuilder() - .setFamilyName(familyName) - .setColumnQualifier(columnQualifier) - .setValue(value) - .setTimestampMicros(timestampMicros) - ) - .build() - - /** New `DeleteFromColumn` mutation. */ - def newDeleteFromColumn( - familyName: String, - columnQualifier: ByteString, - startTimestampMicros: Long, - endTimestampMicros: Long - ): Mutation = - Mutation - .newBuilder() - .setDeleteFromColumn( - DeleteFromColumn - .newBuilder() - .setFamilyName(familyName) - .setColumnQualifier(columnQualifier) - .setTimeRange( - TimestampRange - .newBuilder() - .setStartTimestampMicros(startTimestampMicros) - .setEndTimestampMicros(endTimestampMicros) - ) - ) - .build() - - /** New `DeleteFromFamily` mutation. */ - def newDeleteFromFamily(familyName: String): Mutation = - Mutation - .newBuilder() - .setDeleteFromFamily( - DeleteFromFamily - .newBuilder() - .setFamilyName(familyName) - ) - .build() - - /** New `DeleteFromRow` mutation. */ - def newDeleteFromRow: Mutation = - Mutation - .newBuilder() - .setDeleteFromRow(DeleteFromRow.getDefaultInstance) - .build() -} diff --git a/scio-google-cloud-platform/src/main/scala/com/spotify/scio/bigtable/Rows.scala b/scio-google-cloud-platform/src/main/scala/com/spotify/scio/bigtable/Rows.scala deleted file mode 100644 index e9f26cc441..0000000000 --- a/scio-google-cloud-platform/src/main/scala/com/spotify/scio/bigtable/Rows.scala +++ /dev/null @@ -1,75 +0,0 @@ -/* - * Copyright 2019 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.bigtable - -import com.google.bigtable.v2.{Cell, Column, Family, Row} -import com.google.protobuf.ByteString - -/** Helper methods for `Row`. */ -object Rows { - private def newCell(value: ByteString): Cell = - Cell.newBuilder().setValue(value).build() - - private def newCell(value: ByteString, timestampMicros: Long): Cell = - Cell - .newBuilder() - .setValue(value) - .setTimestampMicros(timestampMicros) - .build() - - private def newRow( - key: ByteString, - familyName: String, - columnQualifier: ByteString, - cell: Cell - ): Row = - Row - .newBuilder() - .setKey(key) - .addFamilies( - Family - .newBuilder() - .setName(familyName) - .addColumns( - Column - .newBuilder() - .setQualifier(columnQualifier) - .addCells(cell) - ) - ) - .build() - - /** New `Row` with timestamp default to 0. */ - def newRow( - key: ByteString, - familyName: String, - columnQualifier: ByteString, - value: ByteString - ): Row = - newRow(key, familyName, columnQualifier, newCell(value)) - - /** New `Row`. */ - def newRow( - key: ByteString, - familyName: String, - columnQualifier: ByteString, - value: ByteString, - timestampMicros: Long - ): Row = - newRow(key, familyName, columnQualifier, newCell(value, timestampMicros)) -} diff --git a/scio-google-cloud-platform/src/main/scala/com/spotify/scio/bigtable/TableAdmin.scala b/scio-google-cloud-platform/src/main/scala/com/spotify/scio/bigtable/TableAdmin.scala index 1a316382e2..19fd217bec 100644 --- a/scio-google-cloud-platform/src/main/scala/com/spotify/scio/bigtable/TableAdmin.scala +++ b/scio-google-cloud-platform/src/main/scala/com/spotify/scio/bigtable/TableAdmin.scala @@ -17,18 +17,13 @@ package com.spotify.scio.bigtable -import java.nio.charset.Charset - -import com.google.bigtable.admin.v2._ -import com.google.bigtable.admin.v2.ModifyColumnFamiliesRequest.Modification -import com.google.cloud.bigtable.config.BigtableOptions -import com.google.cloud.bigtable.grpc._ -import com.google.protobuf.{ByteString, Duration => ProtoDuration} -import org.joda.time.Duration +import com.google.cloud.bigtable.beam.{CloudBigtableConfiguration, CloudBigtableTableConfiguration} +import com.google.cloud.bigtable.hbase.BigtableConfiguration +import org.apache.hadoop.hbase.client.AbstractBigtableAdmin +import org.apache.hadoop.hbase.{HTableDescriptor, TableName} import org.slf4j.{Logger, LoggerFactory} -import scala.jdk.CollectionConverters._ -import scala.util.Try +import java.nio.charset.StandardCharsets /** Bigtable Table Admin API helper commands. */ object TableAdmin { @@ -37,261 +32,77 @@ object TableAdmin { object CreateDisposition { case object Never extends CreateDisposition case object CreateIfNeeded extends CreateDisposition - val default = CreateIfNeeded + def default: CreateDisposition = CreateIfNeeded } private val log: Logger = LoggerFactory.getLogger(TableAdmin.getClass) - private def adminClient[A]( - bigtableOptions: BigtableOptions - )(f: BigtableTableAdminClient => A): Try[A] = { - val channel = - ChannelPoolCreator.createPool(bigtableOptions) - val executorService = - BigtableSessionSharedThreadPools.getInstance().getRetryExecutor - val client = new BigtableTableAdminGrpcClient(channel, executorService, bigtableOptions) - - val result = Try(f(client)) - channel.shutdownNow() - result + private def execute[A]( + config: CloudBigtableConfiguration + )(f: AbstractBigtableAdmin => A): A = { + val connection = BigtableConfiguration.connect(config.toHBaseConfig) + try { + f(connection.getAdmin.asInstanceOf[AbstractBigtableAdmin]) + } finally { + connection.close() + } } - /** - * Retrieves a set of tables from the given instancePath. - * - * @param client - * Client for calling Bigtable. - * @param instancePath - * String of the form "projects/$project/instances/$instance". - * @return - */ - private def fetchTables(client: BigtableTableAdminClient, instancePath: String): Set[String] = - client - .listTables( - ListTablesRequest - .newBuilder() - .setParent(instancePath) - .build() - ) - .getTablesList - .asScala - .map(_.getName) - .toSet - /** * Ensure that tables and column families exist. Checks for existence of tables or creates them if * they do not exist. Also checks for existence of column families within each table and creates * them if they do not exist. * - * @param tablesAndColumnFamilies - * A map of tables and column families. Keys are table names. Values are a list of column family - * names. + * @param config + * Bigtable configuration + * @param tables + * List of tables to check existence + * @param createDisposition + * Create disposition One of [CreateIfNeeded, Never] */ def ensureTables( - bigtableOptions: BigtableOptions, - tablesAndColumnFamilies: Map[String, Iterable[String]], - createDisposition: CreateDisposition = CreateDisposition.default - ): Unit = { - val tcf = tablesAndColumnFamilies.iterator.map { case (k, l) => - k -> l.map(_ -> None) - }.toMap - ensureTablesImpl(bigtableOptions, tcf, createDisposition).get - } - - /** - * Ensure that tables and column families exist. Checks for existence of tables or creates them if - * they do not exist. Also checks for existence of column families within each table and creates - * them if they do not exist. - * - * @param tablesAndColumnFamilies - * A map of tables and column families. Keys are table names. Values are a list of column family - * names along with the desired cell expiration. Cell expiration is the duration before which - * garbage collection of a cell may occur. Note: minimum granularity is one second. - */ - def ensureTablesWithExpiration( - bigtableOptions: BigtableOptions, - tablesAndColumnFamilies: Map[String, Iterable[(String, Option[Duration])]], + config: CloudBigtableConfiguration, + tables: Iterable[HTableDescriptor], createDisposition: CreateDisposition = CreateDisposition.default ): Unit = { - // Convert Duration to GcRule - val x = tablesAndColumnFamilies.iterator.map { case (k, v) => - k -> v.map { case (columnFamily, duration) => - (columnFamily, duration.map(gcRuleFromDuration)) - } - }.toMap - - ensureTablesImpl(bigtableOptions, x, createDisposition).get - } - - /** - * Ensure that tables and column families exist. Checks for existence of tables or creates them if - * they do not exist. Also checks for existence of column families within each table and creates - * them if they do not exist. - * - * @param tablesAndColumnFamilies - * A map of tables and column families. Keys are table names. Values are a list of column family - * names along with the desired GcRule. - */ - def ensureTablesWithGcRules( - bigtableOptions: BigtableOptions, - tablesAndColumnFamilies: Map[String, Iterable[(String, Option[GcRule])]], - createDisposition: CreateDisposition = CreateDisposition.default - ): Unit = - ensureTablesImpl(bigtableOptions, tablesAndColumnFamilies, createDisposition).get - - /** - * Ensure that tables and column families exist. Checks for existence of tables or creates them if - * they do not exist. Also checks for existence of column families within each table and creates - * them if they do not exist. - * - * @param tablesAndColumnFamilies - * A map of tables and column families. Keys are table names. Values are a list of column family - * names. - */ - private def ensureTablesImpl( - bigtableOptions: BigtableOptions, - tablesAndColumnFamilies: Map[String, Iterable[(String, Option[GcRule])]], - createDisposition: CreateDisposition - ): Try[Unit] = { - val project = bigtableOptions.getProjectId - val instance = bigtableOptions.getInstanceId - val instancePath = s"projects/$project/instances/$instance" - - log.info("Ensuring tables and column families exist in instance {}", instance) - - adminClient(bigtableOptions) { client => - val existingTables = fetchTables(client, instancePath) - - tablesAndColumnFamilies.foreach { case (table, columnFamilies) => - val tablePath = s"$instancePath/tables/$table" - - val exists = existingTables.contains(tablePath) + log.info("Ensuring tables and column families exist in instance {}", config.getInstanceId) + execute(config) { client => + val existingTables = client.listTableNames().toSet + tables.foreach { table => + val tableName = table.getTableName + val exists = existingTables.contains(tableName) createDisposition match { case _ if exists => - log.info("Table {} exists", table) + log.info("Table {} exists", tableName) + val existingTable = client.getTableDescriptor(tableName) + if (existingTable != table) { + log.info("Modifying table {}", tableName) + client.modifyTable(tableName, table) + } case CreateDisposition.CreateIfNeeded => - log.info("Creating table {}", table) - client.createTable( - CreateTableRequest - .newBuilder() - .setParent(instancePath) - .setTableId(table) - .build() - ) + log.info("Creating table {}", tableName) + client.createTable(table) case CreateDisposition.Never => - throw new IllegalStateException(s"Table $table does not exist") + throw new IllegalStateException(s"Table $tableName does not exist") } - - ensureColumnFamilies(client, tablePath, columnFamilies, createDisposition) } } } - /** - * Ensure that column families exist. Checks for existence of column families and creates them if - * they don't exist. - * - * @param tablePath - * A full table path that the bigtable API expects, in the form of - * `projects/projectId/instances/instanceId/tables/tableId` - * @param columnFamilies - * A list of column family names. - */ - private def ensureColumnFamilies( - client: BigtableTableAdminClient, - tablePath: String, - columnFamilies: Iterable[(String, Option[GcRule])], - createDisposition: CreateDisposition - ): Unit = - createDisposition match { - case CreateDisposition.CreateIfNeeded => - val tableInfo = - client.getTable(GetTableRequest.newBuilder().setName(tablePath).build) - - val cfList = columnFamilies - .map { case (n, gcRule) => - val cf = tableInfo - .getColumnFamiliesOrDefault(n, ColumnFamily.newBuilder().build()) - .toBuilder - .setGcRule(gcRule.getOrElse(GcRule.getDefaultInstance)) - .build() - - (n, cf) - } - val modifications = - cfList.map { case (n, cf) => - val mod = Modification.newBuilder().setId(n) - if (tableInfo.containsColumnFamilies(n)) { - mod.setUpdate(cf) - } else { - mod.setCreate(cf) - } - mod.build() - } - - log.info( - "Modifying or updating {} column families for table {}", - modifications.size, - tablePath - ) - - if (modifications.nonEmpty) { - client.modifyColumnFamily( - ModifyColumnFamiliesRequest - .newBuilder() - .setName(tablePath) - .addAllModifications(modifications.asJava) - .build - ) - } - () - case CreateDisposition.Never => - () - } - - private def gcRuleFromDuration(duration: Duration): GcRule = { - val protoDuration = ProtoDuration.newBuilder.setSeconds(duration.getStandardSeconds) - GcRule.newBuilder.setMaxAge(protoDuration).build - } - - /** - * Permanently deletes a row range from the specified table that match a particular prefix. - * - * @param table - * table name - * @param rowPrefix - * row key prefix - */ - def dropRowRange(bigtableOptions: BigtableOptions, table: String, rowPrefix: String): Try[Unit] = - adminClient(bigtableOptions) { client => - val project = bigtableOptions.getProjectId - val instance = bigtableOptions.getInstanceId - val instancePath = s"projects/$project/instances/$instance" - val tablePath = s"$instancePath/tables/$table" - - dropRowRange(tablePath, rowPrefix, client) - } - /** * Permanently deletes a row range from the specified table that match a particular prefix. * - * @param tablePath - * A full table path that the bigtable API expects, in the form of - * `projects/projectId/instances/instanceId/tables/tableId` - * @param rowPrefix - * row key prefix + * @param config + * Bigtable configuration + * @param tableName + * Table name + * @param prefix + * Row key prefix */ - private def dropRowRange( - tablePath: String, - rowPrefix: String, - client: BigtableTableAdminClient - ): Unit = { - val request = DropRowRangeRequest - .newBuilder() - .setName(tablePath) - .setRowKeyPrefix(ByteString.copyFrom(rowPrefix, Charset.forName("UTF-8"))) - .build() - - client.dropRowRange(request) - } + def dropRowRange( + config: CloudBigtableTableConfiguration, + tableName: TableName, + prefix: String + ): Unit = + execute(config)(_.deleteRowRangeByPrefix(tableName, prefix.getBytes(StandardCharsets.UTF_8))) } diff --git a/scio-google-cloud-platform/src/main/scala/com/spotify/scio/bigtable/syntax/AllSyntax.scala b/scio-google-cloud-platform/src/main/scala/com/spotify/scio/bigtable/syntax/AllSyntax.scala index 76d7f01fd2..42898c1db0 100644 --- a/scio-google-cloud-platform/src/main/scala/com/spotify/scio/bigtable/syntax/AllSyntax.scala +++ b/scio-google-cloud-platform/src/main/scala/com/spotify/scio/bigtable/syntax/AllSyntax.scala @@ -17,4 +17,6 @@ package com.spotify.scio.bigtable.syntax -trait AllSyntax extends SCollectionSyntax with ScioContextSyntax with RowSyntax +import com.spotify.scio.bigtable.BigtableCoders + +trait AllSyntax extends SCollectionSyntax with ScioContextSyntax with BigtableCoders diff --git a/scio-google-cloud-platform/src/main/scala/com/spotify/scio/bigtable/syntax/RowSyntax.scala b/scio-google-cloud-platform/src/main/scala/com/spotify/scio/bigtable/syntax/RowSyntax.scala deleted file mode 100644 index d3e160f22d..0000000000 --- a/scio-google-cloud-platform/src/main/scala/com/spotify/scio/bigtable/syntax/RowSyntax.scala +++ /dev/null @@ -1,85 +0,0 @@ -/* - * Copyright 2020 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.bigtable.syntax - -import com.google.bigtable.v2.Row -import com.google.protobuf.ByteString - -import scala.jdk.CollectionConverters._ -import com.google.bigtable.v2.Cell - -/** Enhanced version of `Row` with convenience methods. */ -final class RowOps(private val self: Row) extends AnyVal { - - /** Return the `Cell`s for the specific column. */ - def getColumnCells(familyName: String, columnQualifier: ByteString): List[Cell] = - (for { - f <- self.getFamiliesList.asScala.find(_.getName == familyName) - c <- f.getColumnsList.asScala.find(_.getQualifier == columnQualifier) - } yield c.getCellsList.asScala).toList.flatten - - /** The `Cell` for the most recent timestamp for a given column. */ - def getColumnLatestCell(familyName: String, columnQualifier: ByteString): Option[Cell] = - getColumnCells(familyName, columnQualifier).headOption - - /** Map of qualifiers to values. */ - def getFamilyMap(familyName: String): Map[ByteString, ByteString] = - self.getFamiliesList.asScala.find(_.getName == familyName) match { - case None => Map.empty - case Some(f) => - if (f.getColumnsCount > 0) { - f.getColumnsList.asScala - .map(c => c.getQualifier -> c.getCells(0).getValue) - .toMap - } else { - Map.empty - } - } - - /** Map of families to all versions of its qualifiers and values. */ - def getMap: Map[String, Map[ByteString, Map[Long, ByteString]]] = { - val m = Map.newBuilder[String, Map[ByteString, Map[Long, ByteString]]] - for (family <- self.getFamiliesList.asScala) { - val columnMap = Map.newBuilder[ByteString, Map[Long, ByteString]] - for (column <- family.getColumnsList.asScala) { - val cellMap = column.getCellsList.asScala - .map(x => x.getTimestampMicros -> x.getValue) - .toMap - columnMap += column.getQualifier -> cellMap - } - m += family.getName -> columnMap.result() - } - m.result() - } - - /** Map of families to their most recent qualifiers and values. */ - def getNoVersionMap: Map[String, Map[ByteString, ByteString]] = - self.getFamiliesList.asScala - .map(f => f.getName -> getFamilyMap(f.getName)) - .toMap - - /** Get the latest version of the specified column. */ - def getValue(familyName: String, columnQualifier: ByteString): Option[ByteString] = - for { - f <- self.getFamiliesList.asScala.find(_.getName == familyName) - c <- f.getColumnsList.asScala.find(_.getQualifier == columnQualifier) - } yield c.getCells(0).getValue -} - -trait RowSyntax { - implicit def rowOps(row: Row): RowOps = new RowOps(row) -} diff --git a/scio-google-cloud-platform/src/main/scala/com/spotify/scio/bigtable/syntax/SCollectionSyntax.scala b/scio-google-cloud-platform/src/main/scala/com/spotify/scio/bigtable/syntax/SCollectionSyntax.scala index f918df27bb..daab3b44e5 100644 --- a/scio-google-cloud-platform/src/main/scala/com/spotify/scio/bigtable/syntax/SCollectionSyntax.scala +++ b/scio-google-cloud-platform/src/main/scala/com/spotify/scio/bigtable/syntax/SCollectionSyntax.scala @@ -17,47 +17,27 @@ package com.spotify.scio.bigtable.syntax -import com.google.bigtable.v2._ -import com.google.cloud.bigtable.config.BigtableOptions -import com.google.protobuf.ByteString +import com.google.cloud.bigtable.beam.CloudBigtableTableConfiguration +import com.spotify.scio.bigtable.BigtableWrite import com.spotify.scio.io.ClosedTap import com.spotify.scio.values.SCollection -import org.joda.time.Duration - -import com.spotify.scio.bigtable.BigtableWrite +import org.apache.hadoop.hbase.client.Mutation /** * Enhanced version of [[com.spotify.scio.values.SCollection SCollection]] with Bigtable methods. */ -final class SCollectionMutationOps[T <: Mutation]( - private val self: SCollection[(ByteString, Iterable[T])] -) { +final class SCollectionMutationOps[T <: Mutation](private val self: SCollection[T]) { /** Save this SCollection as a Bigtable table. Note that elements must be of type `Mutation`. */ def saveAsBigtable(projectId: String, instanceId: String, tableId: String): ClosedTap[Nothing] = - self.write(BigtableWrite[T](projectId, instanceId, tableId))(BigtableWrite.Default) + self.write(BigtableWrite[T](projectId, instanceId, tableId)) /** Save this SCollection as a Bigtable table. Note that elements must be of type `Mutation`. */ - def saveAsBigtable(bigtableOptions: BigtableOptions, tableId: String): ClosedTap[Nothing] = - self.write(BigtableWrite[T](bigtableOptions, tableId))(BigtableWrite.Default) - - /** - * Save this SCollection as a Bigtable table. This version supports batching. Note that elements - * must be of type `Mutation`. - */ - def saveAsBigtable( - bigtableOptions: BigtableOptions, - tableId: String, - numOfShards: Int, - flushInterval: Duration = BigtableWrite.Bulk.DefaultFlushInterval - ): ClosedTap[Nothing] = - self.write(BigtableWrite[T](bigtableOptions, tableId))( - BigtableWrite.Bulk(numOfShards, flushInterval) - ) + def saveAsBigtable(config: CloudBigtableTableConfiguration): ClosedTap[Nothing] = + self.write(BigtableWrite[T](config)) } trait SCollectionSyntax { - implicit def bigtableMutationOps[T <: Mutation]( - sc: SCollection[(ByteString, Iterable[T])] - ): SCollectionMutationOps[T] = new SCollectionMutationOps[T](sc) + implicit def bigtableMutationOps[T <: Mutation](sc: SCollection[T]): SCollectionMutationOps[T] = + new SCollectionMutationOps[T](sc) } diff --git a/scio-google-cloud-platform/src/main/scala/com/spotify/scio/bigtable/syntax/ScioContextSyntax.scala b/scio-google-cloud-platform/src/main/scala/com/spotify/scio/bigtable/syntax/ScioContextSyntax.scala index a84b1f0e47..aae1053c82 100644 --- a/scio-google-cloud-platform/src/main/scala/com/spotify/scio/bigtable/syntax/ScioContextSyntax.scala +++ b/scio-google-cloud-platform/src/main/scala/com/spotify/scio/bigtable/syntax/ScioContextSyntax.scala @@ -17,19 +17,17 @@ package com.spotify.scio.bigtable.syntax -import com.google.bigtable.admin.v2.GcRule -import com.google.bigtable.v2._ +import com.google.cloud.bigtable.beam.{CloudBigtableConfiguration, CloudBigtableTableConfiguration} import com.google.cloud.bigtable.config.BigtableOptions import com.spotify.scio.ScioContext -import com.spotify.scio.bigtable.BigtableRead -import com.spotify.scio.bigtable.BigtableUtil -import com.spotify.scio.bigtable.TableAdmin +import com.spotify.scio.bigtable.{BigtableRead, InstanceAdmin, TableAdmin} import com.spotify.scio.values.SCollection import org.apache.beam.sdk.io.range.ByteKeyRange +import org.apache.hadoop.hbase.HTableDescriptor +import org.apache.hadoop.hbase.client.Result +import org.apache.hadoop.hbase.filter.Filter import org.joda.time.Duration -import scala.jdk.CollectionConverters._ - object ScioContextOps { private val DefaultSleepDuration = Duration.standardMinutes(20) private val DefaultClusterNames: Set[String] = Set.empty @@ -44,41 +42,38 @@ final class ScioContextOps(private val self: ScioContext) extends AnyVal { projectId: String, instanceId: String, tableId: String, - keyRange: ByteKeyRange, - rowFilter: RowFilter - ): SCollection[Row] = - bigtable(projectId, instanceId, tableId, Seq(keyRange), rowFilter) + keyRange: ByteKeyRange = BigtableRead.ReadParam.DefaultKeyRange, + filter: Filter = BigtableRead.ReadParam.DefaultFilter + ): SCollection[Result] = { + val params = BigtableRead.ReadParam(keyRange, filter) + self.read(BigtableRead(projectId, instanceId, tableId))(params) + } /** Get an SCollection for a Bigtable table. */ def bigtable( - projectId: String, - instanceId: String, - tableId: String, - keyRanges: Seq[ByteKeyRange] = BigtableRead.ReadParam.DefaultKeyRanges, - rowFilter: RowFilter = BigtableRead.ReadParam.DefaultRowFilter - ): SCollection[Row] = { - val parameters = BigtableRead.ReadParam(keyRanges, rowFilter) - self.read(BigtableRead(projectId, instanceId, tableId))(parameters) + config: CloudBigtableTableConfiguration + ): SCollection[Result] = { + val parameters = BigtableRead.ReadParam() + self.read(BigtableRead(config))(parameters) } /** Get an SCollection for a Bigtable table. */ def bigtable( - bigtableOptions: BigtableOptions, - tableId: String, - keyRange: ByteKeyRange, - rowFilter: RowFilter - ): SCollection[Row] = - bigtable(bigtableOptions, tableId, Seq(keyRange), rowFilter) + config: CloudBigtableTableConfiguration, + keyRange: ByteKeyRange + ): SCollection[Result] = { + val parameters = BigtableRead.ReadParam(keyRange) + self.read(BigtableRead(config))(parameters) + } /** Get an SCollection for a Bigtable table. */ def bigtable( - bigtableOptions: BigtableOptions, - tableId: String, - keyRanges: Seq[ByteKeyRange], - rowFilter: RowFilter - ): SCollection[Row] = { - val parameters = BigtableRead.ReadParam(keyRanges, rowFilter) - self.read(BigtableRead(bigtableOptions, tableId))(parameters) + config: CloudBigtableTableConfiguration, + keyRange: ByteKeyRange, + filter: Filter + ): SCollection[Result] = { + val parameters = BigtableRead.ReadParam(keyRange, filter) + self.read(BigtableRead(config))(parameters) } /** @@ -90,92 +85,102 @@ final class ScioContextOps(private val self: ScioContext) extends AnyVal { * How long to sleep after updating the number of nodes. Google recommends at least 20 minutes * before the new nodes are fully functional */ - def updateNumberOfBigtableNodes( + def resizeClusters( projectId: String, instanceId: String, - numberOfNodes: Int, + numServeNodes: Int, sleepDuration: Duration = DefaultSleepDuration - ): Unit = - updateNumberOfBigtableNodes( - projectId, - instanceId, - numberOfNodes, - DefaultClusterNames, - sleepDuration - ) + ): Unit = { + val config = new CloudBigtableConfiguration.Builder() + .withProjectId(projectId) + .withInstanceId(instanceId) + .build() + resizeClusters(config, numServeNodes, sleepDuration) + } /** - * Updates given clusters within the specified Bigtable instance to a specified number of nodes. + * Updates all clusters within the specified Bigtable instance to a specified number of nodes. * Useful for increasing the number of nodes at the beginning of a job and decreasing it at the * end to lower costs yet still get high throughput during bulk ingests/dumps. * * @param sleepDuration * How long to sleep after updating the number of nodes. Google recommends at least 20 minutes * before the new nodes are fully functional - * @param clusterNames - * Names of clusters to be updated, all if empty */ - def updateNumberOfBigtableNodes( + def resizeClusters( + config: CloudBigtableConfiguration, + numServeNodes: Int, + sleepDuration: Duration + ): Unit = if (!self.isTest) { + InstanceAdmin.resizeClusters( + config, + numServeNodes, + sleepDuration + ) + } + + /** + * Updates all clusters within the specified Bigtable instance to a specified number of nodes. + * Useful for increasing the number of nodes at the beginning of a job and decreasing it at the + * end to lower costs yet still get high throughput during bulk ingests/dumps. + */ + def resizeClusters( projectId: String, instanceId: String, - numberOfNodes: Int, - clusterNames: Set[String], - sleepDuration: Duration + clusterIds: Set[String], + numServeNodes: Int ): Unit = { - val bigtableOptions = BigtableOptions - .builder() - .setProjectId(projectId) - .setInstanceId(instanceId) - .build - updateNumberOfBigtableNodes(bigtableOptions, numberOfNodes, clusterNames, sleepDuration) + val config = new CloudBigtableConfiguration.Builder() + .withProjectId(projectId) + .withInstanceId(instanceId) + .build() + resizeClusters(config, clusterIds, numServeNodes, DefaultSleepDuration) } /** - * Updates all clusters within the specified Bigtable instance to a specified number of nodes. + * Updates given clusters within the specified Bigtable instance to a specified number of nodes. * Useful for increasing the number of nodes at the beginning of a job and decreasing it at the * end to lower costs yet still get high throughput during bulk ingests/dumps. * + * @param clusterNames + * Names of clusters to be updated, all if empty * @param sleepDuration * How long to sleep after updating the number of nodes. Google recommends at least 20 minutes * before the new nodes are fully functional */ - def updateNumberOfBigtableNodes( - bigtableOptions: BigtableOptions, - numberOfNodes: Int, + def resizeClusters( + projectId: String, + instanceId: String, + clusterIds: Set[String], + numServeNodes: Int, sleepDuration: Duration - ): Unit = - updateNumberOfBigtableNodes( - bigtableOptions, - numberOfNodes, - DefaultClusterNames, - sleepDuration - ) + ): Unit = { + val config = new CloudBigtableConfiguration.Builder() + .withProjectId(projectId) + .withInstanceId(instanceId) + .build() + resizeClusters(config, clusterIds, numServeNodes, sleepDuration) + } /** * Updates given clusters within the specified Bigtable instance to a specified number of nodes. * Useful for increasing the number of nodes at the beginning of a job and decreasing it at the * end to lower costs yet still get high throughput during bulk ingests/dumps. * - * @param clusterNames + * @param clusterIds * Names of clusters to be updated, all if empty * @param sleepDuration * How long to sleep after updating the number of nodes. Google recommends at least 20 minutes * before the new nodes are fully functional */ - def updateNumberOfBigtableNodes( - bigtableOptions: BigtableOptions, - numberOfNodes: Int, - clusterNames: Set[String], + def resizeClusters( + config: CloudBigtableConfiguration, + clusterIds: Set[String], + numServeNodes: Int, sleepDuration: Duration ): Unit = if (!self.isTest) { - // No need to update the number of nodes in a test - BigtableUtil.updateNumberOfBigtableNodes( - bigtableOptions, - numberOfNodes, - sleepDuration, - clusterNames.asJava - ) + InstanceAdmin.resizeClusters(config, clusterIds, numServeNodes, sleepDuration) } /** @@ -184,13 +189,25 @@ final class ScioContextOps(private val self: ScioContext) extends AnyVal { * @return * map of clusterId to its number of nodes */ - def getBigtableClusterSizes(projectId: String, instanceId: String): Map[String, Int] = + def getBigtableClusterSizes(projectId: String, instanceId: String): Map[String, Int] = { + val config = new CloudBigtableConfiguration.Builder() + .withProjectId(projectId) + .withInstanceId(instanceId) + .build() + getBigtableClusterSizes(config) + } + + /** + * Get size of all clusters for specified Bigtable instance. + * + * @return + * map of clusterId to its number of nodes + */ + def getBigtableClusterSizes(config: CloudBigtableConfiguration): Map[String, Int] = if (!self.isTest) { - BigtableUtil - .getClusterSizes(projectId, instanceId) - .asScala - .iterator - .map { case (k, v) => k -> v.toInt } + InstanceAdmin + .listClusters(config) + .map(c => c.getInstanceId -> c.getServeNodes) .toMap } else { Map.empty @@ -201,204 +218,58 @@ final class ScioContextOps(private val self: ScioContext) extends AnyVal { * they do not exist. Also checks for existence of column families within each table and creates * them if they do not exist. * - * @param tablesAndColumnFamilies - * A map of tables and column families. Keys are table names. Values are a list of column family - * names. + * @param tables */ def ensureTables( projectId: String, instanceId: String, - tablesAndColumnFamilies: Map[String, Iterable[String]], - createDisposition: TableAdmin.CreateDisposition + tables: Iterable[HTableDescriptor] ): Unit = - if (!self.isTest) { - val bigtableOptions = BigtableOptions - .builder() - .setProjectId(projectId) - .setInstanceId(instanceId) - .build - TableAdmin.ensureTables(bigtableOptions, tablesAndColumnFamilies, createDisposition) - } - - def ensureTables( - projectId: String, - instanceId: String, - tablesAndColumnFamilies: Map[String, Iterable[String]] - ): Unit = ensureTables( - projectId, - instanceId, - tablesAndColumnFamilies, - TableAdmin.CreateDisposition.default - ) + ensureTables(projectId, instanceId, tables, TableAdmin.CreateDisposition.default) /** * Ensure that tables and column families exist. Checks for existence of tables or creates them if * they do not exist. Also checks for existence of column families within each table and creates * them if they do not exist. * - * @param tablesAndColumnFamilies - * A map of tables and column families. Keys are table names. Values are a list of column family - * names. + * @param tables */ def ensureTables( - bigtableOptions: BigtableOptions, - tablesAndColumnFamilies: Map[String, Iterable[String]], - createDisposition: TableAdmin.CreateDisposition - ): Unit = - if (!self.isTest) { - TableAdmin.ensureTables(bigtableOptions, tablesAndColumnFamilies, createDisposition) - } - - def ensureTables( - bigtableOptions: BigtableOptions, - tablesAndColumnFamilies: Map[String, Iterable[String]] - ): Unit = - ensureTables(bigtableOptions, tablesAndColumnFamilies, TableAdmin.CreateDisposition.default) - - /** - * Ensure that tables and column families exist. Checks for existence of tables or creates them if - * they do not exist. Also checks for existence of column families within each table and creates - * them if they do not exist. - * - * @param tablesAndColumnFamiliesWithExpiration - * A map of tables and column families. Keys are table names. Values are a list of column family - * names along with the desired cell expiration. Cell expiration is the duration before which - * garbage collection of a cell may occur. Note: minimum granularity is second. - */ - def ensureTablesWithExpiration( - projectId: String, - instanceId: String, - tablesAndColumnFamiliesWithExpiration: Map[String, Iterable[(String, Option[Duration])]], - createDisposition: TableAdmin.CreateDisposition - ): Unit = - if (!self.isTest) { - val bigtableOptions = BigtableOptions - .builder() - .setProjectId(projectId) - .setInstanceId(instanceId) - .build - TableAdmin.ensureTablesWithExpiration( - bigtableOptions, - tablesAndColumnFamiliesWithExpiration, - createDisposition - ) - } - - def ensureTablesWithExpiration( projectId: String, instanceId: String, - tablesAndColumnFamiliesWithExpiration: Map[String, Iterable[(String, Option[Duration])]] - ): Unit = ensureTablesWithExpiration( - projectId, - instanceId, - tablesAndColumnFamiliesWithExpiration, - TableAdmin.CreateDisposition.default - ) - - /** - * Ensure that tables and column families exist. Checks for existence of tables or creates them if - * they do not exist. Also checks for existence of column families within each table and creates - * them if they do not exist. - * - * @param tablesAndColumnFamiliesWithExpiration - * A map of tables and column families. Keys are table names. Values are a list of column family - * names along with the desired cell expiration. Cell expiration is the duration before which - * garbage collection of a cell may occur. Note: minimum granularity is second. - */ - def ensureTablesWithExpiration( - bigtableOptions: BigtableOptions, - tablesAndColumnFamiliesWithExpiration: Map[String, Iterable[(String, Option[Duration])]], + tables: Iterable[HTableDescriptor], createDisposition: TableAdmin.CreateDisposition - ): Unit = - if (!self.isTest) { - TableAdmin.ensureTablesWithExpiration( - bigtableOptions, - tablesAndColumnFamiliesWithExpiration, - createDisposition - ) - } - - def ensureTablesWithExpiration( - bigtableOptions: BigtableOptions, - tablesAndColumnFamiliesWithExpiration: Map[String, Iterable[(String, Option[Duration])]] - ): Unit = ensureTablesWithExpiration( - bigtableOptions, - tablesAndColumnFamiliesWithExpiration, - TableAdmin.CreateDisposition.default - ) + ): Unit = { + val config = new CloudBigtableConfiguration.Builder() + .withProjectId(projectId) + .withInstanceId(instanceId) + .build() + ensureTables(config, tables, createDisposition) + } /** * Ensure that tables and column families exist. Checks for existence of tables or creates them if * they do not exist. Also checks for existence of column families within each table and creates * them if they do not exist. * - * @param tablesAndColumnFamiliesWithGcRules + * @param tablesAndColumnFamilies * A map of tables and column families. Keys are table names. Values are a list of column family - * names along with the desired GcRule. + * names. */ - def ensureTablesWithGcRules( - projectId: String, - instanceId: String, - tablesAndColumnFamiliesWithGcRules: Map[String, Iterable[(String, Option[GcRule])]], - createDisposition: TableAdmin.CreateDisposition - ): Unit = - if (!self.isTest) { - val bigtableOptions = BigtableOptions - .builder() - .setProjectId(projectId) - .setInstanceId(instanceId) - .build - TableAdmin.ensureTablesWithGcRules( - bigtableOptions, - tablesAndColumnFamiliesWithGcRules, - createDisposition - ) - } - - def ensureTablesWithGcRules( - projectId: String, - instanceId: String, - tablesAndColumnFamiliesWithGcRules: Map[String, Iterable[(String, Option[GcRule])]] - ): Unit = ensureTablesWithGcRules( - projectId, - instanceId, - tablesAndColumnFamiliesWithGcRules, - TableAdmin.CreateDisposition.default - ) + def ensureTables( + config: CloudBigtableConfiguration, + tables: Iterable[HTableDescriptor] + ): Unit = ensureTables(config, tables, TableAdmin.CreateDisposition.default) - /** - * Ensure that tables and column families exist. Checks for existence of tables or creates them if - * they do not exist. Also checks for existence of column families within each table and creates - * them if they do not exist. - * - * @param tablesAndColumnFamiliesWithGcRule - * A map of tables and column families. Keys are table names. Values are a list of column family - * names along with the desired cell expiration. Cell expiration is the duration before which - * garbage collection of a cell may occur. Note: minimum granularity is second. - */ - def ensureTablesWithGcRules( - bigtableOptions: BigtableOptions, - tablesAndColumnFamiliesWithGcRule: Map[String, Iterable[(String, Option[GcRule])]], + def ensureTables( + config: CloudBigtableConfiguration, + tables: Iterable[HTableDescriptor], createDisposition: TableAdmin.CreateDisposition - ): Unit = + ): Unit = { if (!self.isTest) { - TableAdmin.ensureTablesWithGcRules( - bigtableOptions, - tablesAndColumnFamiliesWithGcRule, - createDisposition - ) + TableAdmin.ensureTables(config, tables, createDisposition) } - - def ensureTablesWithGcRules( - bigtableOptions: BigtableOptions, - tablesAndColumnFamiliesWithGcRule: Map[String, Iterable[(String, Option[GcRule])]] - ): Unit = - ensureTablesWithGcRules( - bigtableOptions, - tablesAndColumnFamiliesWithGcRule, - TableAdmin.CreateDisposition.default - ) - + } } trait ScioContextSyntax { diff --git a/scio-google-cloud-platform/src/main/scala/com/spotify/scio/coders/GcpKryoRegistrar.scala b/scio-google-cloud-platform/src/main/scala/com/spotify/scio/coders/GcpKryoRegistrar.scala index fd0a006534..5db5fe28f1 100644 --- a/scio-google-cloud-platform/src/main/scala/com/spotify/scio/coders/GcpKryoRegistrar.scala +++ b/scio-google-cloud-platform/src/main/scala/com/spotify/scio/coders/GcpKryoRegistrar.scala @@ -16,17 +16,13 @@ package com.spotify.scio.coders -import com.google.cloud.bigtable.grpc.scanner.BigtableRetriesExhaustedException import com.spotify.scio.bigquery.TableRow import com.spotify.scio.coders.instances.kryo.CoderSerializer -import com.spotify.scio.coders.instances.kryo.BigtableRetriesExhaustedExceptionSerializer import com.twitter.chill._ import org.apache.beam.sdk.io.gcp.bigquery.TableRowJsonCoder @KryoRegistrar class GcpKryoRegistrar extends IKryoRegistrar { - override def apply(k: Kryo): Unit = { + override def apply(k: Kryo): Unit = k.forClass[TableRow](new CoderSerializer(TableRowJsonCoder.of())) - k.forClass[BigtableRetriesExhaustedException](new BigtableRetriesExhaustedExceptionSerializer) - } } diff --git a/scio-google-cloud-platform/src/main/scala/com/spotify/scio/coders/instances/kryo/GcpSerializer.scala b/scio-google-cloud-platform/src/main/scala/com/spotify/scio/coders/instances/kryo/GcpSerializer.scala deleted file mode 100644 index f72191d44a..0000000000 --- a/scio-google-cloud-platform/src/main/scala/com/spotify/scio/coders/instances/kryo/GcpSerializer.scala +++ /dev/null @@ -1,43 +0,0 @@ -/* - * 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.coders.instances.kryo - -import com.esotericsoftware.kryo.serializers.DefaultSerializers.StringSerializer -import com.google.cloud.bigtable.grpc.scanner.BigtableRetriesExhaustedException -import com.twitter.chill._ - -private[coders] class BigtableRetriesExhaustedExceptionSerializer - extends KSerializer[BigtableRetriesExhaustedException] { - - private lazy val stringSerializer = new StringSerializer() - private lazy val statusExceptionSerializer = new StatusRuntimeExceptionSerializer() - - override def write(kryo: Kryo, output: Output, e: BigtableRetriesExhaustedException): Unit = { - kryo.writeObject(output, e.getMessage, stringSerializer) - kryo.writeObject(output, e.getCause, statusExceptionSerializer) - } - - override def read( - kryo: Kryo, - input: Input, - `type`: Class[BigtableRetriesExhaustedException] - ): BigtableRetriesExhaustedException = { - val message = kryo.readObject(input, classOf[String], stringSerializer) - val cause = kryo.readObject(input, classOf[Throwable], statusExceptionSerializer) - new BigtableRetriesExhaustedException(message, cause) - } -} diff --git a/scio-google-cloud-platform/src/test/java/com/spotify/scio/bigtable/BigtableBulkWriterTest.java b/scio-google-cloud-platform/src/test/java/com/spotify/scio/bigtable/BigtableBulkWriterTest.java deleted file mode 100644 index 8bad910623..0000000000 --- a/scio-google-cloud-platform/src/test/java/com/spotify/scio/bigtable/BigtableBulkWriterTest.java +++ /dev/null @@ -1,200 +0,0 @@ -/* - * Copyright 2018 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.bigtable; - -import static org.hamcrest.MatcherAssert.assertThat; -import static org.hamcrest.Matchers.containsInAnyOrder; - -import com.google.bigtable.v2.Mutation; -import com.google.protobuf.ByteString; -import java.nio.charset.StandardCharsets; -import java.util.ArrayList; -import java.util.Collections; -import java.util.List; -import org.apache.beam.sdk.coders.Coder; -import org.apache.beam.sdk.testing.PAssert; -import org.apache.beam.sdk.testing.TestPipeline; -import org.apache.beam.sdk.testing.TestStream; -import org.apache.beam.sdk.transforms.PTransform; -import org.apache.beam.sdk.transforms.SerializableFunction; -import org.apache.beam.sdk.values.KV; -import org.apache.beam.sdk.values.PCollection; -import org.apache.beam.sdk.values.TimestampedValue; -import org.apache.beam.sdk.values.TypeDescriptor; -import org.joda.time.Duration; -import org.joda.time.Instant; -import org.junit.Rule; -import org.junit.Test; -import org.junit.runner.RunWith; -import org.junit.runners.JUnit4; - -@RunWith(JUnit4.class) -public class BigtableBulkWriterTest { - - @Rule public final transient TestPipeline p = TestPipeline.create(); - - private static final Instant baseTime = new Instant(0); - - private static final TypeDescriptor>> BIGTABLE_WRITE_TYPE = - new TypeDescriptor>>() {}; - - @Test - public void testCreateBulkShards() throws Exception { - final List>> expected = new ArrayList<>(); - - final String key1 = "key1"; - final String value1 = "value1"; - final String key2 = "key2"; - final String value2 = "value2"; - final String key3 = "key3"; - final String value3 = "value3"; - final String key4 = "key4"; - final String value4 = "value4"; - - final TimestampedValue>> firstMutation = - makeMutation(key1, value1, Duration.standardMinutes(1)); - - expected.add(firstMutation.getValue()); - - final TimestampedValue>> secondMutation = - makeMutation(key2, value2, Duration.standardMinutes(5)); - expected.add(secondMutation.getValue()); - - final TimestampedValue>> thirdMutation = - makeMutation(key3, value3, Duration.standardMinutes(1)); - expected.add(thirdMutation.getValue()); - - final TimestampedValue>> fourthMutation = - makeMutation(key4, value4, Duration.standardMinutes(4)); - expected.add(fourthMutation.getValue()); - - final Coder>> bigtableCoder = - p.getCoderRegistry().getCoder(BIGTABLE_WRITE_TYPE); - - final TestStream>> kvTestStream = - TestStream.create(bigtableCoder) - .addElements(firstMutation) - .advanceProcessingTime(Duration.standardMinutes(2)) - .addElements(secondMutation) - .advanceProcessingTime(Duration.standardMinutes(11)) - .advanceWatermarkTo(baseTime.plus(Duration.standardMinutes(15))) - // Late elements are always observable within the global window - they arrive before - // the window closes, so they will appear in a pane, even if they arrive after the - // allowed lateness, and are taken into account alongside on-time elements - .addElements(thirdMutation) - .addElements(fourthMutation) - .advanceProcessingTime(Duration.standardMinutes(20)) - .advanceWatermarkToInfinity(); - - final Duration flushInterval = Duration.standardSeconds(1); - final int numOfShard = 1; - final PCollection>>> actual = - p.apply(kvTestStream).apply(new TestPTransform(numOfShard, flushInterval)); - - PAssert.that(actual).inEarlyGlobalWindowPanes().satisfies(new VerifyKVStreamFn(expected)); - - p.run(); - } - - private TimestampedValue>> makeMutation( - String key, String value, Duration baseTimeOffset) { - Instant timestamp = baseTime.plus(baseTimeOffset); - ByteString rowKey = ByteString.copyFromUtf8(key); - Iterable mutations = - Collections.singletonList( - Mutation.newBuilder() - .setSetCell(Mutation.SetCell.newBuilder().setValue(ByteString.copyFromUtf8(value))) - .build()); - return TimestampedValue.of(KV.of(rowKey, mutations), timestamp); - } - - /** - * Hepler class to verify output of {@link PCollection} by converting {@link ByteString} to {@link - * String} to able to verify values. - */ - private static class VerifyKVStreamFn - implements SerializableFunction< - Iterable>>>, Void> { - - private final Iterable>> expected; - - private VerifyKVStreamFn(Iterable>> expected) { - this.expected = expected; - } - - @Override - public Void apply(Iterable>>> input) { - verify(input, expected); - return null; - } - - private List>> convertExpected( - final Iterable>> input) { - List>> mutations = new ArrayList<>(); - for (KV> kv : input) { - final String key = kv.getKey().toString(StandardCharsets.UTF_8); - mutations.add(KV.of(key, kv.getValue())); - } - return mutations; - } - - private List>> convertActual( - final Iterable>>> input) { - List>> mutations = new ArrayList<>(); - for (Iterable>> kv : input) { - for (KV> value : kv) { - final String key = value.getKey().toString(StandardCharsets.UTF_8); - mutations.add(KV.of(key, value.getValue())); - } - } - return mutations; - } - - private void verify( - final Iterable>>> input, - final Iterable>> expected) { - final List>> actual = convertActual(input); - final List>> expectedValues = convertExpected(expected); - - final KV[] kvs = expectedValues.toArray(new KV[0]); - - assertThat(actual, containsInAnyOrder(kvs)); - } - } - - /** Hepler to test createBulkShards. */ - private static class TestPTransform - extends PTransform< - PCollection>>, - PCollection>>>> { - - private final int numOfShards; - private final Duration flushInterval; - - private TestPTransform(int numOfShards, Duration flushInterval) { - this.numOfShards = numOfShards; - this.flushInterval = flushInterval; - } - - @Override - public PCollection>>> expand( - PCollection>> input) { - return BigtableBulkWriter.createBulkShards(input, numOfShards, flushInterval); - } - } -} diff --git a/scio-google-cloud-platform/src/test/java/com/spotify/scio/bigtable/BigtableUtilTest.java b/scio-google-cloud-platform/src/test/java/com/spotify/scio/bigtable/BigtableUtilTest.java deleted file mode 100644 index f47ece0330..0000000000 --- a/scio-google-cloud-platform/src/test/java/com/spotify/scio/bigtable/BigtableUtilTest.java +++ /dev/null @@ -1,33 +0,0 @@ -/* - * Copyright 2022 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.bigtable; - -import org.junit.Assert; -import org.junit.Test; - -public class BigtableUtilTest { - - @Test - public void shorterNameTest() { - Assert.assertEquals( - BigtableUtil.shorterName( - "/projects/scio-test/instances/test-instance/clusters/sample-cluster"), - "sample-cluster"); - - Assert.assertEquals(BigtableUtil.shorterName("simple-name-cluster"), "simple-name-cluster"); - } -} diff --git a/scio-google-cloud-platform/src/test/scala/com/spotify/scio/bigtable/BigtableDoFnTest.scala b/scio-google-cloud-platform/src/test/scala/com/spotify/scio/bigtable/BigtableDoFnTest.scala deleted file mode 100644 index b785d6fd6c..0000000000 --- a/scio-google-cloud-platform/src/test/scala/com/spotify/scio/bigtable/BigtableDoFnTest.scala +++ /dev/null @@ -1,100 +0,0 @@ -/* - * Copyright 2019 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.bigtable - -import java.util.concurrent.ConcurrentLinkedQueue -import com.google.cloud.bigtable.grpc.BigtableSession -import com.google.common.cache.{Cache, CacheBuilder} -import com.google.common.util.concurrent.{Futures, ListenableFuture} -import com.spotify.scio.testing._ -import com.spotify.scio.transforms.BaseAsyncLookupDoFn.CacheSupplier -import com.spotify.scio.transforms.JavaAsyncConverters._ -import com.spotify.scio.util.TransformingCache.SimpleTransformingCache - -import scala.jdk.CollectionConverters._ -import scala.util.{Failure, Success} - -class BigtableDoFnTest extends PipelineSpec { - "BigtableDoFn" should "work" in { - val fn = new TestBigtableDoFn - val output = runWithData(1 to 10)(_.parDo(fn)) - .map(kv => (kv.getKey, kv.getValue.get())) - output should contain theSameElementsAs (1 to 10).map(x => (x, x.toString)) - } - - it should "work with cache" in { - val fn = new TestCachingBigtableDoFn - val output = runWithData((1 to 10) ++ (6 to 15))(_.parDo(fn)) - .map(kv => (kv.getKey, kv.getValue.get())) - output should contain theSameElementsAs ((1 to 10) ++ (6 to 15)).map(x => (x, x.toString)) - BigtableDoFnTest.queue.asScala.toSet should contain theSameElementsAs (1 to 15) - BigtableDoFnTest.queue.size() should be <= 20 - } - - it should "work with failures" in { - val fn = new TestFailingBigtableDoFn - val output = runWithData(1 to 10)(_.parDo(fn)).map { kv => - val r = kv.getValue.asScala match { - case Success(v) => v - case Failure(e) => e.getMessage - } - (kv.getKey, r) - } - output should contain theSameElementsAs (1 to 10).map { x => - val prefix = if (x % 2 == 0) "success" else "failure" - (x, prefix + x.toString) - } - } -} - -object BigtableDoFnTest { - val queue: ConcurrentLinkedQueue[Int] = new ConcurrentLinkedQueue[Int]() -} - -class TestBigtableDoFn extends BigtableDoFn[Int, String](null) { - override def newClient(): BigtableSession = null - override def asyncLookup(session: BigtableSession, input: Int): ListenableFuture[String] = - Futures.immediateFuture(input.toString) -} - -class TestCachingBigtableDoFn extends BigtableDoFn[Int, String](null, 100, new TestCacheSupplier) { - override def newClient(): BigtableSession = null - override def asyncLookup(session: BigtableSession, input: Int): ListenableFuture[String] = { - BigtableDoFnTest.queue.add(input) - Futures.immediateFuture(input.toString) - } -} - -class TestFailingBigtableDoFn extends BigtableDoFn[Int, String](null) { - override def newClient(): BigtableSession = null - override def asyncLookup(session: BigtableSession, input: Int): ListenableFuture[String] = - if (input % 2 == 0) { - Futures.immediateFuture("success" + input) - } else { - Futures.immediateFailedFuture(new RuntimeException("failure" + input)) - } -} - -// Here we need a custom supplier because guava cache only supports object -// We can overcome by using a TransformingCache and boxing -class TestCacheSupplier extends CacheSupplier[Int, String] { - override def get(): Cache[Int, String] = - new SimpleTransformingCache[Int, java.lang.Integer, String](CacheBuilder.newBuilder().build()) { - override protected def transformKey(key: Int): java.lang.Integer = Int.box(key) - } -} diff --git a/scio-google-cloud-platform/src/test/scala/com/spotify/scio/bigtable/BigtableIOTest.scala b/scio-google-cloud-platform/src/test/scala/com/spotify/scio/bigtable/BigtableIOTest.scala index eb47617917..973fe7865b 100644 --- a/scio-google-cloud-platform/src/test/scala/com/spotify/scio/bigtable/BigtableIOTest.scala +++ b/scio-google-cloud-platform/src/test/scala/com/spotify/scio/bigtable/BigtableIOTest.scala @@ -17,31 +17,47 @@ package com.spotify.scio.bigtable -import com.google.bigtable.v2.Mutation.SetCell -import com.google.bigtable.v2.{Mutation, Row} -import com.google.protobuf.ByteString +import cats.Eq +import com.google.cloud.bigtable.hbase.adapters.read.RowCell import com.spotify.scio.testing._ +import org.apache.hadoop.hbase.Cell +import org.apache.hadoop.hbase.client.{Mutation, Put, Result} + +import java.nio.charset.StandardCharsets +import java.time.Instant +import scala.jdk.CollectionConverters._ class BigtableIOTest extends ScioIOSpec { val projectId = "project" val instanceId = "instance" + val family = "count".getBytes(StandardCharsets.UTF_8) + val qualifier = "int".getBytes(StandardCharsets.UTF_8) + "BigtableIO" should "work with input" in { - val xs = (1 to 100).map { x => - Row.newBuilder().setKey(ByteString.copyFromUtf8(x.toString)).build() + val ts = Instant.now().toEpochMilli + implicit val eqResult: Eq[Result] = Eq.by(_.toString) + val xs = (1 to 10).map { x => + val cell = new RowCell( + x.toString.getBytes(StandardCharsets.UTF_8), + family, + qualifier, + ts, + BigInt(x).toByteArray + ) + Result.create(List[Cell](cell).asJava) } + testJobTestInput(xs)(BigtableIO(projectId, instanceId, _))(_.bigtable(projectId, instanceId, _)) } it should "work with output" in { - val xs = (1 to 100).map { x => - val k = ByteString.copyFromUtf8(x.toString) - val m = Mutation - .newBuilder() - .setSetCell(SetCell.newBuilder().setValue(ByteString.copyFromUtf8(x.toString))) - .build() - (k, Iterable(m)) + implicit val eqResult: Eq[Mutation] = Eq.by(_.toString) + val xs: Seq[Mutation] = (1 to 10).map { x => + new Put(x.toString.getBytes(StandardCharsets.UTF_8)) + .addColumn(family, qualifier, BigInt(x).toByteArray) } + testJobTestOutput(xs)(BigtableIO(projectId, instanceId, _))( _.saveAsBigtable(projectId, instanceId, _) ) diff --git a/scio-google-cloud-platform/src/test/scala/com/spotify/scio/bigtable/RichRowTest.scala b/scio-google-cloud-platform/src/test/scala/com/spotify/scio/bigtable/RichRowTest.scala deleted file mode 100644 index dcd9857cba..0000000000 --- a/scio-google-cloud-platform/src/test/scala/com/spotify/scio/bigtable/RichRowTest.scala +++ /dev/null @@ -1,101 +0,0 @@ -/* - * Copyright 2019 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.bigtable - -import com.google.bigtable.v2.{Cell, Column, Family, Row} -import com.google.protobuf.ByteString -import org.scalatest.matchers.should.Matchers -import org.scalatest.flatspec.AnyFlatSpec - -import scala.jdk.CollectionConverters._ -import scala.collection.immutable.ListMap -import scala.collection.immutable - -class RichRowTest extends AnyFlatSpec with Matchers { - def bs(s: String): ByteString = ByteString.copyFromUtf8(s) - - val FAMILY_NAME = "family" - - val dataMap: Map[ByteString, ListMap[Long, ByteString]] = Seq( - "a" -> Seq(10 -> "x", 9 -> "y", 8 -> "z"), - "b" -> Seq(7 -> "u", 6 -> "v", 5 -> "w"), - "c" -> Seq(4 -> "r", 3 -> "s", 2 -> "t") - ).map { case (q, cs) => - val kvs = cs.map(kv => (kv._1.toLong, bs(kv._2))) - (bs(q), ListMap(kvs: _*)) - }.toMap - - val columns: immutable.Iterable[Column] = dataMap.map { case (q, cs) => - val cells = cs.map { case (t, v) => - Cell.newBuilder().setTimestampMicros(t).setValue(v).build() - } - Column - .newBuilder() - .setQualifier(q) - .addAllCells(cells.asJava) - .build() - } - - val row: Row = Row - .newBuilder() - .addFamilies( - Family - .newBuilder() - .setName(FAMILY_NAME) - .addAllColumns(columns.asJava) - ) - .build() - - "RichRow" should "support getColumnCells" in { - for ((q, cs) <- dataMap) { - val cells = cs.map { case (t, v) => - Cell.newBuilder().setTimestampMicros(t).setValue(v).build() - } - row.getColumnCells(FAMILY_NAME, q) shouldBe cells - } - } - - it should "support getColumnLatestCell" in { - for ((q, cs) <- dataMap) { - val cells = cs.map { case (t, v) => - Cell.newBuilder().setTimestampMicros(t).setValue(v).build() - } - row.getColumnLatestCell(FAMILY_NAME, q) shouldBe cells.headOption - } - } - - it should "support getFamilyMap" in { - val familyMap = dataMap.map { case (q, cs) => (q, cs.head._2) } - row.getFamilyMap(FAMILY_NAME) shouldBe familyMap - } - - it should "support getMap" in { - row.getMap shouldBe Map(FAMILY_NAME -> dataMap) - } - - it should "support getNoVersionMap" in { - val noVerMap = dataMap.map { case (q, cs) => (q, cs.head._2) } - row.getNoVersionMap shouldBe Map(FAMILY_NAME -> noVerMap) - } - - it should "support getValue" in { - for ((q, cs) <- dataMap) { - row.getValue(FAMILY_NAME, q) shouldBe Some(cs.head._2) - } - } -} diff --git a/scio-google-cloud-platform/src/test/scala/com/spotify/scio/coders/instance/kryo/GcpSerializerTest.scala b/scio-google-cloud-platform/src/test/scala/com/spotify/scio/coders/instance/kryo/GcpSerializerTest.scala deleted file mode 100644 index 836c4c8d0d..0000000000 --- a/scio-google-cloud-platform/src/test/scala/com/spotify/scio/coders/instance/kryo/GcpSerializerTest.scala +++ /dev/null @@ -1,68 +0,0 @@ -/* - * 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.coders.instance.kryo - -import com.google.cloud.bigtable.grpc.scanner.BigtableRetriesExhaustedException -import com.spotify.scio.coders.instances.kryo.GrpcSerializerTest.eqStatusRuntimeException -import com.spotify.scio.coders.{Coder, CoderMaterializer} -import io.grpc.{Metadata, Status, StatusRuntimeException} -import org.apache.beam.sdk.util.CoderUtils -import org.scalactic.Equality -import org.scalatest.flatspec.AnyFlatSpec -import org.scalatest.matchers.should.Matchers - -object GcpSerializerTest { - - implicit val eqBigtableRetriesExhaustedException: Equality[BigtableRetriesExhaustedException] = { - case (a: BigtableRetriesExhaustedException, b: BigtableRetriesExhaustedException) => - a.getMessage == b.getMessage && - ((Option(a.getCause), Option(b.getCause)) match { - case (None, None) => true - case (Some(ac: StatusRuntimeException), Some(bc: StatusRuntimeException)) => - eqStatusRuntimeException.areEqual(ac, bc) - case _ => - false - }) - case _ => false - } - -} - -class GcpSerializerTest extends AnyFlatSpec with Matchers { - - import GcpSerializerTest._ - - "GcpSerializer" should "roundtrip" in { - val metadata = new Metadata() - metadata.put(Metadata.Key.of[String]("k", Metadata.ASCII_STRING_MARSHALLER), "v") - val cause = new StatusRuntimeException( - Status.OK.withCause(new RuntimeException("bar")).withDescription("bar"), - metadata - ) - roundtrip(new BigtableRetriesExhaustedException("Error", cause)) - } - - private def roundtrip(t: BigtableRetriesExhaustedException): Unit = { - val kryoBCoder = CoderMaterializer.beamWithDefault(Coder[BigtableRetriesExhaustedException]) - - val bytes = CoderUtils.encodeToByteArray(kryoBCoder, t) - val copy = CoderUtils.decodeFromByteArray(kryoBCoder, bytes) - - t shouldEqual copy - } - -} diff --git a/scio-test/src/main/scala/com/spotify/scio/testing/ScioIOSpec.scala b/scio-test/src/main/scala/com/spotify/scio/testing/ScioIOSpec.scala index ded1019d31..d0eb3fdeb8 100644 --- a/scio-test/src/main/scala/com/spotify/scio/testing/ScioIOSpec.scala +++ b/scio-test/src/main/scala/com/spotify/scio/testing/ScioIOSpec.scala @@ -17,6 +17,8 @@ package com.spotify.scio.testing +import cats.Eq + import java.io.File import com.spotify.scio._ import com.spotify.scio.io._ @@ -110,79 +112,71 @@ trait ScioIOSpec extends PipelineSpec { FileUtils.deleteDirectory(tmpDir) } - def testJobTestInput[T: ClassTag: Coder](xs: Seq[T], in: String = "in")( + def testJobTestInput[T: ClassTag: Coder: Eq](xs: Seq[T], in: String = "in")( ioFn: String => ScioIO[T] )(readFn: (ScioContext, String) => SCollection[T]): Unit = { def runMain(args: Array[String]): Unit = { val (sc, argz) = ContextAndArgs(args) - readFn(sc, argz("input")).saveAsTextFile("out") + readFn(sc, argz("input")).write(CustomIO[T]("out")) sc.run() - () } val builder = com.spotify.scio.testing .JobTest("null") .input(ioFn(in), xs) - .output(TextIO("out")) { coll => - coll should containInAnyOrder(xs.map(_.toString)) - () + .output(CustomIO[T]("out")) { coll => + coll should containInAnyOrder(xs) } builder.setUp() runMain(Array(s"--input=$in") :+ s"--appName=${builder.testId}") builder.tearDown() - the[IllegalArgumentException] thrownBy { + val e = the[IllegalArgumentException] thrownBy { val builder = com.spotify.scio.testing .JobTest("null") .input(CustomIO[T](in), xs) - .output(TextIO("out")) { coll => - coll should containInAnyOrder(xs.map(_.toString)) - () + .output(CustomIO[T]("out")) { coll => + coll should containInAnyOrder(xs) } builder.setUp() runMain(Array(s"--input=$in") :+ s"--appName=${builder.testId}") builder.tearDown() - } should have message s"requirement failed: Missing test input: ${ioFn(in).testId}, " + - s"available: [CustomIO($in)]" - () + } + e should have message s"requirement failed: Missing test input: ${ioFn(in).testId}, available: [CustomIO($in)]" } - def testJobTestOutput[T: Coder, WT](xs: Seq[T], out: String = "out")( + def testJobTestOutput[T: Coder: Eq](xs: Seq[T], out: String = "out")( ioFn: String => ScioIO[T] - )(writeFn: (SCollection[T], String) => ClosedTap[WT]): Unit = { + )(writeFn: (SCollection[T], String) => ClosedTap[_]): Unit = { def runMain(args: Array[String]): Unit = { val (sc, argz) = ContextAndArgs(args) writeFn(sc.parallelize(xs), argz("output")) sc.run() - () } val builder = com.spotify.scio.testing .JobTest("null") .output(ioFn(out)) { coll => coll should containInAnyOrder(xs) - () } builder.setUp() runMain(Array(s"--output=$out") :+ s"--appName=${builder.testId}") builder.tearDown() - the[IllegalArgumentException] thrownBy { + val e = the[IllegalArgumentException] thrownBy { val builder = com.spotify.scio.testing .JobTest("null") .output(CustomIO[T](out)) { coll => coll should containInAnyOrder(xs) - () } builder.setUp() runMain(Array(s"--output=$out") :+ s"--appName=${builder.testId}") builder.tearDown() - } should have message s"requirement failed: Missing test output: ${ioFn(out).testId}, " + - s"available: [CustomIO($out)]" - () + } + e should have message s"requirement failed: Missing test output: ${ioFn(out).testId}, available: [CustomIO($out)]" } - def testJobTest[T: Coder](xs: Seq[T], in: String = "in", out: String = "out")( + def testJobTest[T: Coder: Eq](xs: Seq[T], in: String = "in", out: String = "out")( ioFn: String => ScioIO[T] )( readFn: (ScioContext, String) => SCollection[T] @@ -192,7 +186,6 @@ trait ScioIOSpec extends PipelineSpec { val data = readFn(sc, argz("input")) writeFn(data, argz("output")) sc.run() - () } val builder = com.spotify.scio.testing @@ -206,33 +199,31 @@ trait ScioIOSpec extends PipelineSpec { runMain(Array(s"--input=$in", s"--output=$out") :+ s"--appName=${builder.testId}") builder.tearDown() - the[IllegalArgumentException] thrownBy { + val e1 = the[IllegalArgumentException] thrownBy { val builder = com.spotify.scio.testing .JobTest("null") .input(CustomIO[T](in), xs) .output(ioFn(out)) { coll => coll should containInAnyOrder(xs) - () } builder.setUp() runMain(Array(s"--input=$in", s"--output=$out") :+ s"--appName=${builder.testId}") builder.tearDown() - } should have message s"requirement failed: Missing test input: ${ioFn(in).testId}, " + - s"available: [CustomIO($in)]" + } + e1 should have message s"requirement failed: Missing test input: ${ioFn(in).testId}, available: [CustomIO($in)]" - the[IllegalArgumentException] thrownBy { + val e2 = the[IllegalArgumentException] thrownBy { val builder = com.spotify.scio.testing .JobTest("null") .input(ioFn(in), xs) .output(CustomIO[T](out)) { coll => coll should containInAnyOrder(xs) - () } builder.setUp() runMain(Array(s"--input=$in", s"--output=$out") :+ s"--appName=${builder.testId}") builder.tearDown() - } should have message s"requirement failed: Missing test output: ${ioFn(out).testId}, " + - s"available: [CustomIO($out)]" - () + } + + e2 should have message s"requirement failed: Missing test output: ${ioFn(out).testId}, available: [CustomIO($out)]" } } From d99672953b8051bfad2ee33db29701244dbae395 Mon Sep 17 00:00:00 2001 From: Michel Davit Date: Thu, 21 Sep 2023 15:22:11 +0200 Subject: [PATCH 2/7] Add missing headers --- .../extra/MagnolifyBigtableExample.scala | 16 ++++++++++++++++ .../spotify/scio/bigtable/BigtableCoders.scala | 16 ++++++++++++++++ .../spotify/scio/bigtable/InstanceAdmin.scala | 16 ++++++++++++++++ 3 files changed, 48 insertions(+) diff --git a/scio-examples/src/main/scala/com/spotify/scio/examples/extra/MagnolifyBigtableExample.scala b/scio-examples/src/main/scala/com/spotify/scio/examples/extra/MagnolifyBigtableExample.scala index 253cec5f48..253c554ffa 100644 --- a/scio-examples/src/main/scala/com/spotify/scio/examples/extra/MagnolifyBigtableExample.scala +++ b/scio-examples/src/main/scala/com/spotify/scio/examples/extra/MagnolifyBigtableExample.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. + */ + ///* // * Copyright 2020 Spotify AB. // * diff --git a/scio-google-cloud-platform/src/main/scala/com/spotify/scio/bigtable/BigtableCoders.scala b/scio-google-cloud-platform/src/main/scala/com/spotify/scio/bigtable/BigtableCoders.scala index 8984493a1c..f75033a17d 100644 --- a/scio-google-cloud-platform/src/main/scala/com/spotify/scio/bigtable/BigtableCoders.scala +++ b/scio-google-cloud-platform/src/main/scala/com/spotify/scio/bigtable/BigtableCoders.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.bigtable import com.spotify.scio.coders.Coder diff --git a/scio-google-cloud-platform/src/main/scala/com/spotify/scio/bigtable/InstanceAdmin.scala b/scio-google-cloud-platform/src/main/scala/com/spotify/scio/bigtable/InstanceAdmin.scala index 67791f11d1..105376b26e 100644 --- a/scio-google-cloud-platform/src/main/scala/com/spotify/scio/bigtable/InstanceAdmin.scala +++ b/scio-google-cloud-platform/src/main/scala/com/spotify/scio/bigtable/InstanceAdmin.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.bigtable import com.google.bigtable.repackaged.com.google.cloud.bigtable.admin.v2.BigtableInstanceAdminClient From 9a4bbd349e118ac3209cd6a029b08f43177f8a09 Mon Sep 17 00:00:00 2001 From: Michel Davit Date: Thu, 21 Sep 2023 15:29:46 +0200 Subject: [PATCH 3/7] scalafix --- .../com/spotify/scio/bigtable/syntax/ScioContextSyntax.scala | 3 +-- .../test/scala/com/spotify/scio/bigtable/BigtableIOTest.scala | 4 ++-- 2 files changed, 3 insertions(+), 4 deletions(-) diff --git a/scio-google-cloud-platform/src/main/scala/com/spotify/scio/bigtable/syntax/ScioContextSyntax.scala b/scio-google-cloud-platform/src/main/scala/com/spotify/scio/bigtable/syntax/ScioContextSyntax.scala index aae1053c82..034bcc7f4e 100644 --- a/scio-google-cloud-platform/src/main/scala/com/spotify/scio/bigtable/syntax/ScioContextSyntax.scala +++ b/scio-google-cloud-platform/src/main/scala/com/spotify/scio/bigtable/syntax/ScioContextSyntax.scala @@ -18,7 +18,6 @@ package com.spotify.scio.bigtable.syntax import com.google.cloud.bigtable.beam.{CloudBigtableConfiguration, CloudBigtableTableConfiguration} -import com.google.cloud.bigtable.config.BigtableOptions import com.spotify.scio.ScioContext import com.spotify.scio.bigtable.{BigtableRead, InstanceAdmin, TableAdmin} import com.spotify.scio.values.SCollection @@ -30,7 +29,7 @@ import org.joda.time.Duration object ScioContextOps { private val DefaultSleepDuration = Duration.standardMinutes(20) - private val DefaultClusterNames: Set[String] = Set.empty + Set.empty } /** Enhanced version of [[ScioContext]] with Bigtable methods. */ diff --git a/scio-google-cloud-platform/src/test/scala/com/spotify/scio/bigtable/BigtableIOTest.scala b/scio-google-cloud-platform/src/test/scala/com/spotify/scio/bigtable/BigtableIOTest.scala index 973fe7865b..5c1eaf2e90 100644 --- a/scio-google-cloud-platform/src/test/scala/com/spotify/scio/bigtable/BigtableIOTest.scala +++ b/scio-google-cloud-platform/src/test/scala/com/spotify/scio/bigtable/BigtableIOTest.scala @@ -31,8 +31,8 @@ class BigtableIOTest extends ScioIOSpec { val projectId = "project" val instanceId = "instance" - val family = "count".getBytes(StandardCharsets.UTF_8) - val qualifier = "int".getBytes(StandardCharsets.UTF_8) + val family: Array[Byte] = "count".getBytes(StandardCharsets.UTF_8) + val qualifier: Array[Byte] = "int".getBytes(StandardCharsets.UTF_8) "BigtableIO" should "work with input" in { val ts = Instant.now().toEpochMilli From cf141d75587e2e80f4547e16b861a0670fb4760a Mon Sep 17 00:00:00 2001 From: Michel Davit Date: Thu, 21 Sep 2023 16:03:55 +0200 Subject: [PATCH 4/7] Fix scio-example test --- .../examples/extra/BigtableExampleTest.scala | 34 +++++++++++-------- 1 file changed, 19 insertions(+), 15 deletions(-) diff --git a/scio-examples/src/test/scala/com/spotify/scio/examples/extra/BigtableExampleTest.scala b/scio-examples/src/test/scala/com/spotify/scio/examples/extra/BigtableExampleTest.scala index 9a46b4a9a7..f38c4309ec 100644 --- a/scio-examples/src/test/scala/com/spotify/scio/examples/extra/BigtableExampleTest.scala +++ b/scio-examples/src/test/scala/com/spotify/scio/examples/extra/BigtableExampleTest.scala @@ -17,6 +17,7 @@ package com.spotify.scio.examples.extra +import cats.Eq import com.google.cloud.bigtable.hbase.adapters.read.RowCell import com.spotify.scio.bigtable._ import com.spotify.scio.io._ @@ -26,6 +27,7 @@ import org.apache.hadoop.hbase.client.{Mutation, Result} import java.nio.charset.StandardCharsets import java.util.Collections +import scala.collection.immutable.Seq class BigtableExampleTest extends PipelineSpec { import BigtableExample._ @@ -38,10 +40,12 @@ class BigtableExampleTest extends PipelineSpec { val textIn: Seq[String] = Seq("a b c d e", "a b a b") val wordCount: Seq[(String, Long)] = Seq("a" -> 3L, "b" -> 3L, "c" -> 1L, "d" -> 1L, "e" -> 1L) - val expectedMutations: Seq[Mutation] = - wordCount.map { case (word, count) => BigtableExample.toPutMutation(word, count) } "BigtableWriteExample" should "work" in { + val expectedMutations: Seq[Mutation] = wordCount + .map { case (word, count) => BigtableExample.toPutMutation(word, count) } + implicit val eqMutation: Eq[Mutation] = Eq.by(_.toString) + JobTest[com.spotify.scio.examples.extra.BigtableWriteExample.type] .args(bigtableOptions :+ "--input=in.txt": _*) .input(TextIO("in.txt"), textIn) @@ -51,21 +55,21 @@ class BigtableExampleTest extends PipelineSpec { .run() } - def toResult(key: String, value: Long): Result = { - val cell = new RowCell( - key.getBytes(StandardCharsets.UTF_8), - FAMILY_NAME, - COLUMN_QUALIFIER, - 0L, - BigInt(value).toByteArray - ) - Result.create(Collections.singletonList[Cell](cell)) - } + "BigtableReadExample" should "work" in { + def toResult(key: String, value: Long): Result = { + val cell = new RowCell( + key.getBytes(StandardCharsets.UTF_8), + FAMILY_NAME, + COLUMN_QUALIFIER, + 0L, + BigInt(value).toByteArray + ) + Result.create(Collections.singletonList[Cell](cell)) + } - val results: Seq[Result] = wordCount.map { case (word, count) => toResult(word, count) } - val expectedText: Seq[String] = wordCount.map { case (word, count) => s"$word:$count" } + val results: Seq[Result] = wordCount.map { case (word, count) => toResult(word, count) } + val expectedText: Seq[String] = wordCount.map { case (word, count) => s"$word:$count" } - "BigtableReadExample" should "work" in { JobTest[com.spotify.scio.examples.extra.BigtableReadExample.type] .args(bigtableOptions :+ "--output=out.txt": _*) .input(BigtableIO("my-project", "my-instance", "my-table"), results) From b300792c2a371209fc3d707556c549a9007fb826 Mon Sep 17 00:00:00 2001 From: Michel Davit Date: Thu, 21 Sep 2023 17:21:41 +0200 Subject: [PATCH 5/7] Fix undeclared dependency issues --- .scala-steward.conf | 1 + build.sbt | 32 +++++++++++++++++++++++++++++++- 2 files changed, 32 insertions(+), 1 deletion(-) diff --git a/.scala-steward.conf b/.scala-steward.conf index e13fb36936..151ea0e132 100644 --- a/.scala-steward.conf +++ b/.scala-steward.conf @@ -34,6 +34,7 @@ updates.ignore = [ { groupId = "org.apache.commons", artifactId = "commons-math3" }, { groupId = "org.apache.flink" }, { groupId = "org.apache.hadoop" }, + { groupId = "org.apache.hbase" } { groupId = "org.apache.httpcomponents" }, { groupId = "org.apache.spark" }, { groupId = "org.checkerframework" }, diff --git a/build.sbt b/build.sbt index 2ce5346db0..8aa03d9570 100644 --- a/build.sbt +++ b/build.sbt @@ -110,6 +110,7 @@ val elasticsearch7Version = "7.17.9" val elasticsearch8Version = "8.9.1" val fansiVersion = "0.4.0" val featranVersion = "0.8.0" +val hbaseVersion = "1.7.2" val httpAsyncClientVersion = "4.1.5" val hamcrestVersion = "2.2" val jakartaJsonVersion = "2.1.2" @@ -234,6 +235,12 @@ lazy val java17Settings = sys.props("java.version") match { case _ => Def.settings() } +// test libs to exclude +val testLibs: Seq[ExclusionRule] = Seq( + "junit" % "junit", + "org.hamcrest" % "hamcrest-core" +) + val commonSettings = formatSettings ++ mimaSettings ++ java17Settings ++ @@ -733,7 +740,9 @@ lazy val `scio-google-cloud-platform`: Project = project "com.google.cloud" % "google-cloud-core" % googleCloudCoreVersion, "com.google.cloud" % "google-cloud-spanner" % googleCloudSpannerVersion, "com.google.cloud.bigdataoss" % "util" % bigdataossVersion, - "com.google.cloud.bigtable" % "bigtable-hbase-beam" % bigtableHbaseBeamVersion, + "com.google.cloud.bigtable" % "bigtable-hbase-beam" % bigtableHbaseBeamVersion excludeAll (testLibs: _*), + "com.google.cloud.bigtable" % "bigtable-hbase-1.x-shaded" % bigtableHbaseBeamVersion excludeAll (testLibs: _*), + "org.apache.hbase" % "hbase-shaded-client" % hbaseVersion excludeAll (testLibs: _*), "com.google.guava" % "guava" % guavaVersion, "com.google.http-client" % "google-http-client" % googleHttpClientsVersion, "com.google.http-client" % "google-http-client-gson" % googleHttpClientsVersion, @@ -1276,6 +1285,12 @@ lazy val `scio-repl`: Project = project case s if s.endsWith(".proto") => // arbitrary pick last conflicting proto file MergeStrategy.last + case PathList("dependencies.properties") => + // arbitrary pick last dependencies property file + MergeStrategy.last + case PathList("THIRD-PARTY.txt") => + // drop conflicting THIRD-PARTY.txt + MergeStrategy.discard case PathList("git.properties") => // drop conflicting git properties MergeStrategy.discard @@ -1285,12 +1300,27 @@ lazy val `scio-repl`: Project = project case PathList("META-INF", "gradle", "incremental.annotation.processors") => // drop conflicting kotlin compiler info MergeStrategy.discard + case PathList("META-INF", "native-image", "incremental.annotation.processors") => + // drop conflicting kotlin compiler info + MergeStrategy.discard case PathList("META-INF", "io.netty.versions.properties") => // merge conflicting netty property files MergeStrategy.filterDistinctLines case PathList("META-INF", "native-image", "native-image.properties") => // merge conflicting native-image property files MergeStrategy.filterDistinctLines + case PathList( + "META-INF", + "native-image", + "com.google.api", + "gax-grpc", + "native-image.properties" + ) => + // merge conflicting native-image property files + MergeStrategy.filterDistinctLines + case PathList("mozilla", "public-suffix-list.txt") => + // merge conflicting public-suffix-list files + MergeStrategy.filterDistinctLines case s => old(s) } } From a750d053fe0831f693fdbd14d1255d26a55377b2 Mon Sep 17 00:00:00 2001 From: Michel Davit Date: Thu, 21 Sep 2023 17:45:00 +0200 Subject: [PATCH 6/7] Update function name --- .../com/spotify/scio/examples/extra/BigtableExample.scala | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/scio-examples/src/main/scala/com/spotify/scio/examples/extra/BigtableExample.scala b/scio-examples/src/main/scala/com/spotify/scio/examples/extra/BigtableExample.scala index 7129131922..375a36a37a 100644 --- a/scio-examples/src/main/scala/com/spotify/scio/examples/extra/BigtableExample.scala +++ b/scio-examples/src/main/scala/com/spotify/scio/examples/extra/BigtableExample.scala @@ -38,7 +38,7 @@ object BigtableExample { .addColumn(FAMILY_NAME, COLUMN_QUALIFIER, 0L, BigInt(value).toByteArray) // Convert a Bigtable `Result` from reading to a formatted key-value string - def fromRow(r: Result): String = { + def fromResult(r: Result): String = { val key = new String(r.getRow, StandardCharsets.UTF_8) val value = BigInt(r.getValue(FAMILY_NAME, COLUMN_QUALIFIER)).toLong s"$key:$value" @@ -110,7 +110,7 @@ object BigtableReadExample { val btTableId = args("bigtableTableId") sc.bigtable(btProjectId, btInstanceId, btTableId) - .map(BigtableExample.fromRow) + .map(BigtableExample.fromResult) .saveAsTextFile(args("output")) sc.run() From 8a72c6616742474429ef0e867704b9cd333ca288 Mon Sep 17 00:00:00 2001 From: Michel Davit Date: Fri, 22 Sep 2023 09:00:35 +0200 Subject: [PATCH 7/7] Fix dependencies and doc --- build.sbt | 12 +- .../scio/testing/BigtableMatchers.scala | 128 ------------ .../scio/testing/BigtableMatchersTest.scala | 197 ------------------ site/src/main/paradox/FAQ.md | 28 +-- 4 files changed, 16 insertions(+), 349 deletions(-) delete mode 100644 scio-test/src/main/scala/com/spotify/scio/testing/BigtableMatchers.scala delete mode 100644 scio-test/src/test/scala/com/spotify/scio/testing/BigtableMatchersTest.scala diff --git a/build.sbt b/build.sbt index 8aa03d9570..4c3bb8e2d5 100644 --- a/build.sbt +++ b/build.sbt @@ -76,7 +76,6 @@ val floggerVersion = "0.7.4" val gaxVersion = "2.32.0" val googleApiCommonVersion = "2.15.0" val googleAuthVersion = "1.19.0" -val googleCloudBigTableVersion = "2.26.0" val googleCloudCoreVersion = "2.22.0" val googleCloudDatastoreVersion = "0.107.3" val googleCloudMonitoringVersion = "3.24.0" @@ -609,7 +608,6 @@ lazy val `scio-test`: Project = project moduleFilter("junit", "junit") ).reduce(_ | _), libraryDependencies ++= Seq( - "com.google.api.grpc" % "proto-google-cloud-bigtable-v2" % googleCloudBigTableVersion, "com.google.http-client" % "google-http-client" % googleHttpClientsVersion, "com.lihaoyi" %% "fansi" % fansiVersion, "com.lihaoyi" %% "pprint" % pprintVersion, @@ -728,22 +726,18 @@ lazy val `scio-google-cloud-platform`: Project = project "com.google.api-client" % "google-api-client" % googleClientsVersion, "com.google.api.grpc" % "grpc-google-cloud-pubsub-v1" % googleCloudPubSubVersion, "com.google.api.grpc" % "proto-google-cloud-bigquerystorage-v1beta1" % bigQueryStorageBetaVersion, - "com.google.api.grpc" % "proto-google-cloud-bigtable-admin-v2" % googleCloudBigTableVersion, - "com.google.api.grpc" % "proto-google-cloud-bigtable-v2" % googleCloudBigTableVersion, "com.google.api.grpc" % "proto-google-cloud-datastore-v1" % googleCloudDatastoreVersion, "com.google.api.grpc" % "proto-google-cloud-pubsub-v1" % googleCloudPubSubVersion, "com.google.apis" % "google-api-services-bigquery" % googleApiServicesBigQueryVersion, "com.google.auth" % "google-auth-library-credentials" % googleAuthVersion, "com.google.auth" % "google-auth-library-oauth2-http" % googleAuthVersion, "com.google.cloud" % "google-cloud-bigquerystorage" % bigQueryStorageVersion, - "com.google.cloud" % "google-cloud-bigtable" % googleCloudBigTableVersion, "com.google.cloud" % "google-cloud-core" % googleCloudCoreVersion, "com.google.cloud" % "google-cloud-spanner" % googleCloudSpannerVersion, "com.google.cloud.bigdataoss" % "util" % bigdataossVersion, "com.google.cloud.bigtable" % "bigtable-hbase-beam" % bigtableHbaseBeamVersion excludeAll (testLibs: _*), "com.google.cloud.bigtable" % "bigtable-hbase-1.x-shaded" % bigtableHbaseBeamVersion excludeAll (testLibs: _*), "org.apache.hbase" % "hbase-shaded-client" % hbaseVersion excludeAll (testLibs: _*), - "com.google.guava" % "guava" % guavaVersion, "com.google.http-client" % "google-http-client" % googleHttpClientsVersion, "com.google.http-client" % "google-http-client-gson" % googleHttpClientsVersion, "com.google.protobuf" % "protobuf-java" % protobufVersion, @@ -1160,7 +1154,6 @@ lazy val `scio-examples`: Project = project "com.fasterxml.jackson.datatype" % "jackson-datatype-jsr310" % jacksonVersion, "com.fasterxml.jackson.module" %% "jackson-module-scala" % jacksonVersion, "com.google.api-client" % "google-api-client" % googleClientsVersion, - "com.google.api.grpc" % "proto-google-cloud-bigtable-v2" % googleCloudBigTableVersion, "com.google.api.grpc" % "proto-google-cloud-datastore-v1" % googleCloudDatastoreVersion, "com.google.apis" % "google-api-services-bigquery" % googleApiServicesBigQueryVersion, "com.google.apis" % "google-api-services-pubsub" % googleApiServicesPubsubVersion, @@ -1174,7 +1167,7 @@ lazy val `scio-examples`: Project = project "com.google.protobuf" % "protobuf-java" % protobufVersion, "com.softwaremill.magnolia1_2" %% "magnolia" % magnoliaVersion, "com.spotify" %% "magnolify-avro" % magnolifyVersion, - "com.spotify" %% "magnolify-bigtable" % magnolifyVersion, + // "com.spotify" %% "magnolify-bigtable" % magnolifyVersion, "com.spotify" %% "magnolify-datastore" % magnolifyVersion, "com.spotify" %% "magnolify-shared" % magnolifyVersion, "com.spotify" %% "magnolify-tensorflow" % magnolifyVersion, @@ -1511,7 +1504,6 @@ lazy val site: Project = project ScalaUnidoc / unidoc / unidocAllClasspaths := (ScalaUnidoc / unidoc / unidocAllClasspaths).value .map { cp => cp.filterNot(_.data.getCanonicalPath.matches(""".*guava-11\..*""")) - .filterNot(_.data.getCanonicalPath.matches(""".*bigtable-client-core-0\..*""")) }, // mdoc // pre-compile md using mdoc @@ -1584,8 +1576,6 @@ ThisBuild / dependencyOverrides ++= Seq( "com.google.api" % "gax-httpjson" % gaxVersion, "com.google.api-client" % "google-api-client" % googleClientsVersion, "com.google.api.grpc" % "grpc-google-common-protos" % googleCommonsProtoVersion, - "com.google.api.grpc" % "proto-google-cloud-bigtable-admin-v2" % googleCloudBigTableVersion, - "com.google.api.grpc" % "proto-google-cloud-bigtable-v2" % googleCloudBigTableVersion, "com.google.api.grpc" % "proto-google-cloud-datastore-v1" % googleCloudDatastoreVersion, "com.google.api.grpc" % "proto-google-common-protos" % googleCommonsProtoVersion, "com.google.api.grpc" % "proto-google-iam-v1" % googleIAMVersion, diff --git a/scio-test/src/main/scala/com/spotify/scio/testing/BigtableMatchers.scala b/scio-test/src/main/scala/com/spotify/scio/testing/BigtableMatchers.scala deleted file mode 100644 index e057efef30..0000000000 --- a/scio-test/src/main/scala/com/spotify/scio/testing/BigtableMatchers.scala +++ /dev/null @@ -1,128 +0,0 @@ -/* - * Copyright 2019 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.testing - -import com.google.bigtable.v2.Mutation -import com.google.bigtable.v2.Mutation.MutationCase -import com.google.protobuf.ByteString -import com.spotify.scio.values.SCollection - -import org.scalatest.matchers.{MatchResult, Matcher} - -/** - * Trait with ScalaTest [[org.scalatest.matchers.Matcher Matcher]] s for - * [[com.spotify.scio.values.SCollection SCollection]] s specific to Bigtable output. - */ -trait BigtableMatchers extends SCollectionMatchers { - - type BTRow = (ByteString, Iterable[Mutation]) - type BTCollection = SCollection[BTRow] - - /** Provide an implicit BT serializer for common cell value type String. */ - implicit def stringBTSerializer(s: String): ByteString = - ByteString.copyFromUtf8(s) - - /** Check that the BT collection contains only the given keys, in any order. */ - def containRowKeys(expectedKeys: ByteString*): Matcher[BTCollection] = - new Matcher[BTCollection] { - override def apply(left: BTCollection): MatchResult = - containInAnyOrder(expectedKeys).apply(left.keys) - } - - /** Check that the BT collection contains only the given column families, unique, in any order. */ - def containColumnFamilies(expectedCFs: String*): Matcher[BTCollection] = - new Matcher[BTCollection] { - override def apply(left: BTCollection): MatchResult = { - val foundCFs = left.flatMap { case (_, cells) => - cells.map(_.getSetCell.getFamilyName) - } - - containInAnyOrder(expectedCFs).apply(foundCFs.distinct) - } - } - - /** - * Check that the BT collection contains a cell with the given row key, column family, and - * deserialized cell value. Column qualifier defaults to the same as column family. - */ - def containSetCellValue[V](key: ByteString, cf: String, value: V)(implicit - ser: V => ByteString - ): Matcher[BTCollection] = - containSetCellValue(key, cf, cf, value) - - /** - * Check that the BT collection contains a cell with the given row key, column family, column - * qualifier, and deserialized cell value. - * @param key - * Row key the cell should be in - * @param cf - * Column family the cell should have - * @param cq - * Column qualifier the cell should have - * @param value - * Deserialized value of the set cell - * @param ser - * Serializer to convert value type V to ByteString for BT format - * @tparam V - * Class of expected value - */ - def containSetCellValue[V](key: ByteString, cf: String, cq: String, value: V)(implicit - ser: V => ByteString - ): Matcher[BTCollection] = - new Matcher[BTCollection] { - override def apply(left: BTCollection): MatchResult = { - val flattenedRows = left.flatMap { case (rowKey, rowValue) => - rowValue.map { cell => - ( - rowKey, - cell.getSetCell.getFamilyName, - cell.getSetCell.getColumnQualifier, - cell.getSetCell.getValue - ) - } - } - - containValue( - ( - key, - cf, - ByteString.copyFromUtf8(cq), - ser.apply(value) - ) - ).apply(flattenedRows) - } - } - - /** - * Check that the BT collection contains a cell with the given row key and enumerated - * MutationCase, making no assumptions about the contents of the rest of the collection. - */ - def containCellMutationCase[V]( - key: ByteString, - mutation: MutationCase - ): Matcher[BTCollection] = - new Matcher[BTCollection] { - override def apply(left: BTCollection): MatchResult = { - val flattenedRows = left.flatMap { case (rowKey, rowValue) => - rowValue.map(cell => (rowKey, cell.getMutationCase.toString)) - } - - containValue((key, mutation.toString)).apply(flattenedRows) - } - } -} diff --git a/scio-test/src/test/scala/com/spotify/scio/testing/BigtableMatchersTest.scala b/scio-test/src/test/scala/com/spotify/scio/testing/BigtableMatchersTest.scala deleted file mode 100644 index 045489d24b..0000000000 --- a/scio-test/src/test/scala/com/spotify/scio/testing/BigtableMatchersTest.scala +++ /dev/null @@ -1,197 +0,0 @@ -/* - * Copyright 2019 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.testing - -import com.google.bigtable.v2.Mutation.{MutationCase, SetCell} -import com.google.bigtable.v2._ -import com.google.protobuf.ByteString - -class BigtableMatchersTest extends PipelineSpec with BigtableMatchers { - private lazy val key1 = ByteString.copyFromUtf8("k1") - private lazy val key2 = ByteString.copyFromUtf8("k2") - private lazy val emptyCell = Seq(Mutation.newBuilder().build()) - - "BigtableMatchers" should "support containRowKeys" in { - val tableData: Seq[BTRow] = Seq( - (key1, emptyCell), - (key2, emptyCell) - ) - - // should cases - runWithContext { - _.parallelize(tableData) should containRowKeys(key1, key2) - } - - an[AssertionError] should be thrownBy { - runWithContext { - _.parallelize(tableData) should containRowKeys(key1) - } - } - - // should not cases - runWithContext { - _.parallelize(tableData) shouldNot containRowKeys(key1) - } - - an[AssertionError] should be thrownBy { - runWithContext { - _.parallelize(tableData) shouldNot containRowKeys(key1, key2) - } - } - } - - private lazy val columnFamily1 = "cf1" - private lazy val columnFamily2 = "cf2" - - it should "support containColumnFamilies" in { - val cell1 = Mutation - .newBuilder() - .setSetCell( - SetCell - .newBuilder() - .setFamilyName(columnFamily1) - ) - .build() - - val cell2 = Mutation - .newBuilder() - .setSetCell( - SetCell - .newBuilder() - .setFamilyName(columnFamily2) - ) - .build() - - val tableData: Seq[BTRow] = Seq( - (key1, Seq(cell1)), - (key2, Seq(cell2)) - ) - - // should cases - runWithContext { - _.parallelize(tableData) should containColumnFamilies(columnFamily1, columnFamily2) - } - - an[AssertionError] should be thrownBy { - runWithContext { - _.parallelize(tableData) should containColumnFamilies(columnFamily1) - } - } - - // should not cases - runWithContext { - _.parallelize(tableData) shouldNot containColumnFamilies(columnFamily1) - } - - an[AssertionError] should be thrownBy { - runWithContext { - _.parallelize(tableData) shouldNot containColumnFamilies(columnFamily1, columnFamily2) - } - } - } - - val cellValue1 = "cv1" - val cellValue2 = "cv2" - - it should "support containSetCellValue" in { - val cell1 = Mutation - .newBuilder() - .setSetCell( - SetCell - .newBuilder() - .setFamilyName(columnFamily1) - .setColumnQualifier(columnFamily1) - .setValue(ByteString.copyFromUtf8(cellValue1)) - ) - .build() - - val cell2 = Mutation - .newBuilder() - .setSetCell( - SetCell - .newBuilder() - .setFamilyName(columnFamily2) - .setColumnQualifier(columnFamily2) - .setValue(ByteString.copyFromUtf8(cellValue2)) - ) - .build() - - val tableData: Seq[BTRow] = Seq( - (key1, Seq(cell1, cell2)) - ) - - // should cases - runWithContext { sc => - sc.parallelize(tableData) should containSetCellValue(key1, columnFamily1, cellValue1) - sc.parallelize(tableData) should containSetCellValue(key1, columnFamily2, cellValue2) - } - - an[AssertionError] should be thrownBy { - runWithContext { - _.parallelize(tableData) should containSetCellValue(key2, columnFamily1, cellValue1) - } - } - - // should not cases - runWithContext { - _.parallelize(tableData) shouldNot containSetCellValue(key2, columnFamily1, cellValue1) - } - - an[AssertionError] should be thrownBy { - runWithContext { - _.parallelize(tableData) shouldNot containSetCellValue(key1, columnFamily1, cellValue1) - } - } - } - - it should "support containCellMutationCase" in { - val cell = Mutation - .newBuilder() - .setDeleteFromRow(Mutation.DeleteFromRow.newBuilder().build()) - .build() - - val tableData: Seq[BTRow] = Seq( - (key1, Seq(cell)) - ) - - // should cases - runWithContext { - _.parallelize(tableData) should containCellMutationCase(key1, MutationCase.DELETE_FROM_ROW) - } - - an[AssertionError] should be thrownBy { - runWithContext { - _.parallelize(tableData) should containCellMutationCase(key1, MutationCase.SET_CELL) - } - } - - // should not cases - runWithContext { - _.parallelize(tableData) shouldNot containCellMutationCase(key1, MutationCase.SET_CELL) - } - - an[AssertionError] should be thrownBy { - runWithContext { - _.parallelize(tableData) shouldNot containCellMutationCase( - key1, - MutationCase.DELETE_FROM_ROW - ) - } - } - } -} diff --git a/site/src/main/paradox/FAQ.md b/site/src/main/paradox/FAQ.md index a434205d70..dc4324f8d4 100644 --- a/site/src/main/paradox/FAQ.md +++ b/site/src/main/paradox/FAQ.md @@ -374,27 +374,29 @@ val btTableId = "" ``` ```scala mdoc:silent +import com.google.cloud.bigtable.beam.CloudBigtableTableConfiguration +import com.google.cloud.bigtable.hbase.BigtableOptionsFactory.{ + BIGTABLE_BUFFERED_MUTATOR_ENABLE_THROTTLING, + BIGTABLE_BUFFERED_MUTATOR_THROTTLING_THRESHOLD_MILLIS +} import com.spotify.scio.values._ import com.spotify.scio.bigtable._ -import com.google.cloud.bigtable.config.{BigtableOptions, BulkOptions} -import com.google.bigtable.v2.Mutation -import com.google.protobuf.ByteString +import org.apache.hadoop.hbase.client.Mutation def main(cmdlineArgs: Array[String]): Unit = { // ... - val data: SCollection[(ByteString, Iterable[Mutation])] = ??? + val data: SCollection[Mutation] = ??? - val btOptions = - BigtableOptions.builder() - .setProjectId(btProjectId) - .setInstanceId(btInstanceId) - .setBulkOptions(BulkOptions.builder() - .enableBulkMutationThrottling() - .setBulkMutationRpcTargetMs(10) // lower latency threshold, default is 100 - .build()) + val btConfig = new CloudBigtableTableConfiguration.Builder() + .withProjectId(btProjectId) + .withInstanceId(btInstanceId) + .withTableId(btTableId) + .withConfiguration(BIGTABLE_BUFFERED_MUTATOR_ENABLE_THROTTLING, "true") + // lower latency threshold, default is 100 + .withConfiguration(BIGTABLE_BUFFERED_MUTATOR_THROTTLING_THRESHOLD_MILLIS, "10") .build() - data.saveAsBigtable(btOptions, btTableId) + data.saveAsBigtable(btConfig) // ... }