-
Notifications
You must be signed in to change notification settings - Fork 4.5k
feat:large-row-skip-in-bigtable | added experimental options to skip … #34245
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
Changes from all commits
0eb2494
50f7924
f648a5a
c5150ff
16b04a2
e253472
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
There are no files selected for viewing
| Original file line number | Diff line number | Diff line change | ||||||||||||||||||||||||||||||||||||||||
|---|---|---|---|---|---|---|---|---|---|---|---|---|---|---|---|---|---|---|---|---|---|---|---|---|---|---|---|---|---|---|---|---|---|---|---|---|---|---|---|---|---|---|
|
|
@@ -142,6 +142,7 @@ static class BigtableReaderImpl implements Reader { | |||||||||||||||||||||||||||||||||||||||||
| private ServerStream<Row> stream; | ||||||||||||||||||||||||||||||||||||||||||
|
|
||||||||||||||||||||||||||||||||||||||||||
| private boolean exhausted; | ||||||||||||||||||||||||||||||||||||||||||
| private BigtableReadOptions bigtableReadOptions; | ||||||||||||||||||||||||||||||||||||||||||
|
|
||||||||||||||||||||||||||||||||||||||||||
| @VisibleForTesting | ||||||||||||||||||||||||||||||||||||||||||
| BigtableReaderImpl( | ||||||||||||||||||||||||||||||||||||||||||
|
|
@@ -150,13 +151,15 @@ static class BigtableReaderImpl implements Reader { | |||||||||||||||||||||||||||||||||||||||||
| String instanceId, | ||||||||||||||||||||||||||||||||||||||||||
| String tableId, | ||||||||||||||||||||||||||||||||||||||||||
| List<ByteKeyRange> ranges, | ||||||||||||||||||||||||||||||||||||||||||
| @Nullable RowFilter rowFilter) { | ||||||||||||||||||||||||||||||||||||||||||
| @Nullable RowFilter rowFilter, | ||||||||||||||||||||||||||||||||||||||||||
| BigtableReadOptions bigtableReadOptions) { | ||||||||||||||||||||||||||||||||||||||||||
| this.client = client; | ||||||||||||||||||||||||||||||||||||||||||
| this.projectId = projectId; | ||||||||||||||||||||||||||||||||||||||||||
| this.instanceId = instanceId; | ||||||||||||||||||||||||||||||||||||||||||
| this.tableId = tableId; | ||||||||||||||||||||||||||||||||||||||||||
| this.ranges = ranges; | ||||||||||||||||||||||||||||||||||||||||||
| this.rowFilter = rowFilter; | ||||||||||||||||||||||||||||||||||||||||||
| this.bigtableReadOptions = bigtableReadOptions; | ||||||||||||||||||||||||||||||||||||||||||
| } | ||||||||||||||||||||||||||||||||||||||||||
|
|
||||||||||||||||||||||||||||||||||||||||||
| @Override | ||||||||||||||||||||||||||||||||||||||||||
|
|
@@ -174,10 +177,18 @@ public boolean start() throws IOException { | |||||||||||||||||||||||||||||||||||||||||
| query.filter(Filters.FILTERS.fromProto(rowFilter)); | ||||||||||||||||||||||||||||||||||||||||||
| } | ||||||||||||||||||||||||||||||||||||||||||
| try { | ||||||||||||||||||||||||||||||||||||||||||
| stream = | ||||||||||||||||||||||||||||||||||||||||||
| client | ||||||||||||||||||||||||||||||||||||||||||
| .readRowsCallable(new BigtableRowProtoAdapter()) | ||||||||||||||||||||||||||||||||||||||||||
| .call(query, GrpcCallContext.createDefault()); | ||||||||||||||||||||||||||||||||||||||||||
| if (bigtableReadOptions != null | ||||||||||||||||||||||||||||||||||||||||||
| && Boolean.TRUE.equals(bigtableReadOptions.getExperimentalSkipLargeRows())) { | ||||||||||||||||||||||||||||||||||||||||||
| stream = | ||||||||||||||||||||||||||||||||||||||||||
| client | ||||||||||||||||||||||||||||||||||||||||||
| .skipLargeRowsCallable(new BigtableRowProtoAdapter()) | ||||||||||||||||||||||||||||||||||||||||||
| .call(query, GrpcCallContext.createDefault()); | ||||||||||||||||||||||||||||||||||||||||||
| } else { | ||||||||||||||||||||||||||||||||||||||||||
| stream = | ||||||||||||||||||||||||||||||||||||||||||
| client | ||||||||||||||||||||||||||||||||||||||||||
| .readRowsCallable(new BigtableRowProtoAdapter()) | ||||||||||||||||||||||||||||||||||||||||||
| .call(query, GrpcCallContext.createDefault()); | ||||||||||||||||||||||||||||||||||||||||||
| } | ||||||||||||||||||||||||||||||||||||||||||
|
Comment on lines
+180
to
+191
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Resolving of the feature should be done during pipeline construction not during execution Also you should factor out the common code: readRowsCallable = client.skipLargeRowsCallable(new BigtableRowProtoAdapter())
if (isLargeRowSkippingEnabled) {
readRowsCallable = client.readRowsCallable(new BigtableRowProtoAdapter());
}
readRowsCallable.call(...
Contributor
Author
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more.
and I had to revert these changes - 50f7924
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. I think you can pass in a unary callable when creating the BigtableReaderImpl instead of the client: Lines 653 to 672 in 3b5a2b6
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. No, I meant keep the unary callable where it is, but resolve unset value in the settings. ie this logic: What unset means should be resolved during pipeline construction |
||||||||||||||||||||||||||||||||||||||||||
| results = stream.iterator(); | ||||||||||||||||||||||||||||||||||||||||||
| serviceCallMetric.call("ok"); | ||||||||||||||||||||||||||||||||||||||||||
| } catch (StatusRuntimeException e) { | ||||||||||||||||||||||||||||||||||||||||||
|
|
@@ -667,7 +678,8 @@ public Reader createReader(BigtableSource source) throws IOException { | |||||||||||||||||||||||||||||||||||||||||
| instanceId, | ||||||||||||||||||||||||||||||||||||||||||
| source.getTableId().get(), | ||||||||||||||||||||||||||||||||||||||||||
| source.getRanges(), | ||||||||||||||||||||||||||||||||||||||||||
| source.getRowFilter()); | ||||||||||||||||||||||||||||||||||||||||||
| source.getRowFilter(), | ||||||||||||||||||||||||||||||||||||||||||
| source.getReadOptions()); | ||||||||||||||||||||||||||||||||||||||||||
| } | ||||||||||||||||||||||||||||||||||||||||||
| } | ||||||||||||||||||||||||||||||||||||||||||
|
|
||||||||||||||||||||||||||||||||||||||||||
|
|
||||||||||||||||||||||||||||||||||||||||||
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -119,6 +119,27 @@ public void testE2EBigtableRead() { | |
| checkLineageSourceMetric(r, tableId); | ||
| } | ||
|
|
||
| @Test | ||
| public void testE2EBigtableReadWithSkippingLargeRows() { | ||
| BigtableOptions.Builder bigtableOptionsBuilder = | ||
| new BigtableOptions.Builder().setProjectId(project).setInstanceId(options.getInstanceId()); | ||
|
|
||
| final String tableId = "BigtableReadTest"; | ||
| final long numRows = 1000L; | ||
|
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. how does this test large row skipping?
Contributor
Author
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. this doesn't. it came out in our discussion earlier, that we need a data integrity check where no data loss should happen.
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. I think you can do a bit better here testE2EBigtableReadWithSkippingLargeRows() {
//...
// add an error injector to trigger large row logic
ExperimentalOptions.addExperiment("bigtable_settings_override", InterceptorInjector.class.getName());
//...
}
static class LargeRowErrorInterceptor implements ClientInterceptor {
@Override
public <ReqT, RespT> ClientCall<ReqT, RespT> interceptCall(MethodDescriptor<ReqT, RespT> method, CallOptions callOptions, Channel next) {
return new ForwardingClientCall.SimpleForwardingClientCall<ReqT, RespT>(next.newCall(method, callOptions)) {
private boolean artificiallyClosed = false;
private int numMsgs = 0;
@Override
public void start(Listener<RespT> responseListener, Metadata headers) {
super.start(new ForwardingClientCallListener.SimpleForwardingClientCallListener<RespT>() {
@Override
public void onMessage(RespT message) {
if (++numMsgs > 10) {
artificiallyClosed = true;
delegate().onClose(
Status.WHATEVER_ERROR_TRIGGERS_PAGING,
new Metadata()
);
return;
}
super.onMessage(message);
}
@Override
public void onClose(Status status, Metadata trailers) {
if (!artificiallyClosed) {
super.onClose(status, trailers);
}
}
}, headers);
}
};
}
}
public static class InterceptorInjector implements BiFunction<BigtableDataSettings.Builder, PipelineOptions,
BigtableDataSettings.Builder> {
@Override
public BigtableDataSettings.Builder apply(BigtableDataSettings.Builder builder, PipelineOptions pipelineOptions) {
InstantiatingGrpcChannelProvider.Builder transportChannelProvider = ((InstantiatingGrpcChannelProvider) builder.stubSettings()
.getTransportChannelProvider())
.toBuilder();
ApiFunction<ManagedChannelBuilder, ManagedChannelBuilder> oldConf = transportChannelProvider.getChannelConfigurator();
transportChannelProvider.setChannelConfigurator(b -> {
if (oldConf!=null) {
b = oldConf.apply(b);
}
return b.intercept(new LargeRowErrorInterceptor());
});
return null;
}
} |
||
|
|
||
| Pipeline p = Pipeline.create(options); | ||
| PCollection<Long> count = | ||
| p.apply( | ||
| BigtableIO.read() | ||
| .withBigtableOptions(bigtableOptionsBuilder) | ||
| .withTableId(tableId) | ||
| .withExperimentalSkipLargeRows(true)) | ||
| .apply(Count.globally()); | ||
| PAssert.thatSingleton(count).isEqualTo(numRows); | ||
| PipelineResult r = p.run(); | ||
| checkLineageSourceMetric(r, tableId); | ||
| } | ||
|
|
||
| @Test | ||
| public void testE2EBigtableSegmentRead() { | ||
| tableAdminClient.createTable(CreateTableRequest.of(tableId).addFamily(COLUMN_FAMILY_NAME)); | ||
|
|
||
Uh oh!
There was an error while loading. Please reload this page.