-
Notifications
You must be signed in to change notification settings - Fork 513
New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
Add Batched lookups for streaming GRPC endpoints and BigTable #5521
Merged
Merged
Changes from 3 commits
Commits
Show all changes
5 commits
Select commit
Hold shift + click to select a range
5db23e2
Report exception for unmatched batch request
RustedBones 4427ece
Update test got GrpcBatchDoFn
RustedBones d39cba0
Extend batched lookup functionality
lofifnc 8678ddd
Ack mima incompatibility
RustedBones 2918ffb
Merge branch 'main' into batched-bigtable-lookup
RustedBones File filter
Filter by extension
Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
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
45 changes: 45 additions & 0 deletions
45
scio-core/src/main/java/com/spotify/scio/transforms/UnmatchedRequestException.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,45 @@ | ||
/* | ||
* Copyright 2024 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.transforms; | ||
|
||
import java.util.Objects; | ||
|
||
public class UnmatchedRequestException extends RuntimeException { | ||
|
||
private final String id; | ||
|
||
public UnmatchedRequestException(String id) { | ||
super("Unmatched batch request for ID: " + id); | ||
this.id = id; | ||
} | ||
|
||
public String getId() { | ||
return id; | ||
} | ||
|
||
@Override | ||
public boolean equals(Object o) { | ||
if (o == null || getClass() != o.getClass()) return false; | ||
UnmatchedRequestException that = (UnmatchedRequestException) o; | ||
return Objects.equals(id, that.id); | ||
} | ||
|
||
@Override | ||
public int hashCode() { | ||
return Objects.hashCode(id); | ||
} | ||
} |
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 2024 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); | ||
} | ||
} | ||
} |
Oops, something went wrong.
Oops, something went wrong.
Add this suggestion to a batch that can be applied as a single commit.
This suggestion is invalid because no changes were made to the code.
Suggestions cannot be applied while the pull request is closed.
Suggestions cannot be applied while viewing a subset of changes.
Only one suggestion per line can be applied in a batch.
Add this suggestion to a batch that can be applied as a single commit.
Applying suggestions on deleted lines is not supported.
You must change the existing code in this line in order to create a valid suggestion.
Outdated suggestions cannot be applied.
This suggestion has been applied or marked resolved.
Suggestions cannot be applied from pending reviews.
Suggestions cannot be applied on multi-line comments.
Suggestions cannot be applied while the pull request is queued to merge.
Suggestion cannot be applied right now. Please check back later.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
As a heads-up, we will probably move to the
BigtableDataClient
in some next minor release, see here. Expect some breaking changes in that part.