-
Notifications
You must be signed in to change notification settings - Fork 514
Commit
This commit does not belong to any branch on this repository, and may belong to a fork outside of the repository.
- Loading branch information
Showing
5 changed files
with
294 additions
and
16 deletions.
There are no files selected for viewing
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
126 changes: 126 additions & 0 deletions
126
scio-google-cloud-platform/src/main/java/com/spotify/scio/bigtable/BigtableBatchDoFn.java
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,126 @@ | ||
/* | ||
* 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.GuavaAsyncBatchLookupDoFn; | ||
import java.io.IOException; | ||
import java.util.List; | ||
import org.apache.beam.sdk.transforms.DoFn; | ||
import org.apache.beam.sdk.transforms.SerializableFunction; | ||
import org.apache.commons.lang3.tuple.Pair; | ||
|
||
/** | ||
* A {@link DoFn} which batches elements and performs asynchronous lookup for them using Google | ||
* Cloud Bigtable. | ||
* | ||
* @param <Input> input element type. | ||
* @param <BatchRequest> batched input element type | ||
* @param <BatchResponse> batched response from BigTable type | ||
* @param <Result> Bigtable lookup value type. | ||
*/ | ||
public abstract class BigtableBatchDoFn<Input, BatchRequest, BatchResponse, Result> | ||
extends GuavaAsyncBatchLookupDoFn<Input, BatchRequest, BatchResponse, Result, BigtableSession> { | ||
|
||
private final BigtableOptions options; | ||
|
||
/** Perform asynchronous Bigtable lookup. */ | ||
public abstract ListenableFuture<BatchResponse> asyncLookup( | ||
BigtableSession session, BatchRequest batchRequest); | ||
|
||
/** | ||
* Create a {@link BigtableBatchDoFn} instance. | ||
* | ||
* @param options Bigtable options. | ||
*/ | ||
public BigtableBatchDoFn( | ||
BigtableOptions options, | ||
int batchSize, | ||
SerializableFunction<List<Input>, BatchRequest> batchRequestFn, | ||
SerializableFunction<BatchResponse, List<Pair<String, Result>>> batchResponseFn, | ||
SerializableFunction<Input, String> idExtractorFn) { | ||
this(options, batchSize, batchRequestFn, batchResponseFn, idExtractorFn, 1000); | ||
} | ||
|
||
/** | ||
* Create a {@link BigtableBatchDoFn} 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 BigtableBatchDoFn( | ||
BigtableOptions options, | ||
int batchSize, | ||
SerializableFunction<List<Input>, BatchRequest> batchRequestFn, | ||
SerializableFunction<BatchResponse, List<Pair<String, Result>>> batchResponseFn, | ||
SerializableFunction<Input, String> idExtractorFn, | ||
int maxPendingRequests) { | ||
this( | ||
options, | ||
batchSize, | ||
batchRequestFn, | ||
batchResponseFn, | ||
idExtractorFn, | ||
maxPendingRequests, | ||
new BaseAsyncLookupDoFn.NoOpCacheSupplier<>()); | ||
} | ||
|
||
/** | ||
* Create a {@link BigtableBatchDoFn} 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 BigtableBatchDoFn( | ||
BigtableOptions options, | ||
int batchSize, | ||
SerializableFunction<List<Input>, BatchRequest> batchRequestFn, | ||
SerializableFunction<BatchResponse, List<Pair<String, Result>>> batchResponseFn, | ||
SerializableFunction<Input, String> idExtractorFn, | ||
int maxPendingRequests, | ||
BaseAsyncLookupDoFn.CacheSupplier<String, Result> cacheSupplier) { | ||
super( | ||
batchSize, | ||
batchRequestFn, | ||
batchResponseFn, | ||
idExtractorFn, | ||
maxPendingRequests, | ||
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); | ||
} | ||
} | ||
} |
136 changes: 136 additions & 0 deletions
136
...oogle-cloud-platform/src/test/scala/com/spotify/scio/bigtable/BigTableBatchDoFnTest.scala
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,136 @@ | ||
/* | ||
* 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.{CompletionException, 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 org.apache.commons.lang3.tuple.Pair | ||
|
||
import scala.jdk.CollectionConverters._ | ||
import scala.util.{Failure, Success} | ||
|
||
class BigtableBatchDoFnTest extends PipelineSpec { | ||
"BigtableDoFn" should "work" in { | ||
val fn = new TestBigtableBatchDoFn | ||
val output = runWithData(1 to 10)(_.parDo(fn)) | ||
.map(kv => (kv.getKey, kv.getValue.get().iterator().next())) | ||
output should contain theSameElementsAs (1 to 10).map(x => (x, x.toString)) | ||
} | ||
|
||
it should "work with cache" in { | ||
val fn = new TestCachingBigtableBatchDoFn | ||
val output = runWithData((1 to 10) ++ (6 to 15))(_.parDo(fn)) | ||
.map(kv => (kv.getKey, kv.getValue.get().iterator().next())) | ||
output should have size 20 | ||
output should contain theSameElementsAs ((1 to 10) ++ (6 to 15)).map(x => (x, x.toString)) | ||
BigtableBatchDoFnTest.queue.asScala.toSet should contain theSameElementsAs (1 to 15) | ||
BigtableBatchDoFnTest.queue.size() should be <= 20 | ||
} | ||
|
||
it should "work with failures" in { | ||
val fn = new TestFailingBigtableBatchDoFn | ||
|
||
val output = runWithData(Seq[Seq[Int]](1 to 4, 8 to 10))(_.flatten.parDo(fn)).map { kv => | ||
val r = kv.getValue.asScala match { | ||
case Success(v) => v.asScala.head | ||
case Failure(e: CompletionException) => e.getCause.getMessage | ||
case Failure(e) => e.getMessage | ||
} | ||
(kv.getKey, r) | ||
} | ||
output should contain theSameElementsAs ( | ||
(1 to 4).map(x => x -> x.toString) ++ | ||
(8 to 10).map(x => x -> "failure for 8,9,10") | ||
) | ||
} | ||
} | ||
|
||
object BigtableBatchDoFnTest { | ||
val queue: ConcurrentLinkedQueue[Int] = new ConcurrentLinkedQueue[Int]() | ||
|
||
def batchRequest(input: java.util.List[Int]): List[Int] = input.asScala.toList | ||
def batchResponse(input: List[String]): java.util.List[Pair[String, String]] = | ||
input.map(x => Pair.of(x, x)).asJava | ||
def idExtractor(input: Int): String = input.toString | ||
} | ||
|
||
class TestBigtableBatchDoFn | ||
extends BigtableBatchDoFn[Int, List[Int], List[String], String]( | ||
null, | ||
2, | ||
BigtableBatchDoFnTest.batchRequest, | ||
BigtableBatchDoFnTest.batchResponse, | ||
BigtableBatchDoFnTest.idExtractor | ||
) { | ||
override def newClient(): BigtableSession = null | ||
override def asyncLookup( | ||
session: BigtableSession, | ||
input: List[Int] | ||
): ListenableFuture[List[String]] = | ||
Futures.immediateFuture(input.map(_.toString)) | ||
} | ||
|
||
class TestCachingBigtableBatchDoFn | ||
extends BigtableBatchDoFn[Int, List[Int], List[String], String]( | ||
null, | ||
2, | ||
BigtableBatchDoFnTest.batchRequest, | ||
BigtableBatchDoFnTest.batchResponse, | ||
BigtableBatchDoFnTest.idExtractor, | ||
100, | ||
new TestCacheBatchSupplier | ||
) { | ||
override def newClient(): BigtableSession = null | ||
|
||
override def asyncLookup( | ||
session: BigtableSession, | ||
input: List[Int] | ||
): ListenableFuture[List[String]] = { | ||
input.foreach(BigtableBatchDoFnTest.queue.add) | ||
Futures.immediateFuture(input.map(_.toString)) | ||
} | ||
} | ||
|
||
class TestFailingBigtableBatchDoFn | ||
extends BigtableBatchDoFn[Int, List[Int], List[String], String]( | ||
null, | ||
4, | ||
BigtableBatchDoFnTest.batchRequest, | ||
BigtableBatchDoFnTest.batchResponse, | ||
BigtableBatchDoFnTest.idExtractor | ||
) { | ||
override def newClient(): BigtableSession = null | ||
override def asyncLookup( | ||
session: BigtableSession, | ||
input: List[Int] | ||
): ListenableFuture[List[String]] = | ||
if (input.size % 2 == 0) { | ||
Futures.immediateFuture(input.map(_.toString)) | ||
} else { | ||
Futures.immediateFailedFuture(new RuntimeException("failure for " + input.mkString(","))) | ||
} | ||
} | ||
|
||
class TestCacheBatchSupplier extends CacheSupplier[String, String] { | ||
override def get(): Cache[String, String] = CacheBuilder.newBuilder().build() | ||
} |
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters