-
Notifications
You must be signed in to change notification settings - Fork 4.2k
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 ReadChangeStream IO param to adjust backlog estimates for replication delay #30995
Changes from all commits
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 |
---|---|---|
|
@@ -2047,6 +2047,9 @@ public enum ExistingPipelineOptions { | |
public abstract static class ReadChangeStream | ||
extends PTransform<PBegin, PCollection<KV<ByteString, ChangeStreamMutation>>> { | ||
|
||
private static final Duration DEFAULT_BACKLOG_REPLICATION_ADJUSTMENT = | ||
Duration.standardSeconds(30); | ||
|
||
static ReadChangeStream create() { | ||
BigtableConfig config = BigtableConfig.builder().setValidate(true).build(); | ||
BigtableConfig metadataTableconfig = BigtableConfig.builder().setValidate(true).build(); | ||
|
@@ -2075,6 +2078,8 @@ static ReadChangeStream create() { | |
|
||
abstract @Nullable Boolean getCreateOrUpdateMetadataTable(); | ||
|
||
abstract @Nullable Duration getBacklogReplicationAdjustment(); | ||
|
||
abstract ReadChangeStream.Builder toBuilder(); | ||
|
||
/** | ||
|
@@ -2259,6 +2264,26 @@ public ReadChangeStream withCreateOrUpdateMetadataTable(boolean shouldCreate) { | |
return toBuilder().setCreateOrUpdateMetadataTable(shouldCreate).build(); | ||
} | ||
|
||
/** | ||
* Returns a new {@link BigtableIO.ReadChangeStream} that overrides the replication delay | ||
* adjustment duration with the provided duration. | ||
* | ||
* <p>Backlog is calculated for each partition using watermarkLag * throughput. Replication | ||
* delay holds back the watermark for each partition. This can cause the backlog to stay | ||
* persistently above dataflow's downscaling threshold (10 seconds) even when a pipeline is | ||
* caught up. | ||
* | ||
* <p>This adjusts the backlog downward to account for this. For unreplicated instances it can | ||
* be set to zero to upscale as quickly as possible. | ||
* | ||
* <p>Optional: defaults to 30 seconds. | ||
* | ||
* <p>Does not modify this object. | ||
*/ | ||
public ReadChangeStream withBacklogReplicationAdjustment(Duration adjustment) { | ||
return toBuilder().setBacklogReplicationAdjustment(adjustment).build(); | ||
} | ||
|
||
@Override | ||
public PCollection<KV<ByteString, ChangeStreamMutation>> expand(PBegin input) { | ||
checkArgument( | ||
|
@@ -2312,6 +2337,10 @@ public PCollection<KV<ByteString, ChangeStreamMutation>> expand(PBegin input) { | |
if (getCreateOrUpdateMetadataTable() != null) { | ||
shouldCreateOrUpdateMetadataTable = getCreateOrUpdateMetadataTable(); | ||
} | ||
Duration backlogReplicationAdjustment = getBacklogReplicationAdjustment(); | ||
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. Nit: if using Optional return type can just be '= getBacklogReplicationAdjustment().orElse(DEFAULT_BACKLOG_REPLICATION_ADJUSTMENT); 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. nullable is fine. For streaming IO a consideration is upgrade compatibility, and newly added parameters then need to be nullable and set default in pipeline submission |
||
if (backlogReplicationAdjustment == null) { | ||
backlogReplicationAdjustment = DEFAULT_BACKLOG_REPLICATION_ADJUSTMENT; | ||
} | ||
|
||
ActionFactory actionFactory = new ActionFactory(); | ||
ChangeStreamMetrics metrics = new ChangeStreamMetrics(); | ||
|
@@ -2356,7 +2385,8 @@ public PCollection<KV<ByteString, ChangeStreamMutation>> expand(PBegin input) { | |
DetectNewPartitionsDoFn detectNewPartitionsDoFn = | ||
new DetectNewPartitionsDoFn(getEndTime(), actionFactory, daoFactory, metrics); | ||
ReadChangeStreamPartitionDoFn readChangeStreamPartitionDoFn = | ||
new ReadChangeStreamPartitionDoFn(daoFactory, actionFactory, metrics); | ||
new ReadChangeStreamPartitionDoFn( | ||
daoFactory, actionFactory, metrics, backlogReplicationAdjustment); | ||
|
||
PCollection<KV<ByteString, ChangeStreamRecord>> readChangeStreamOutput = | ||
input | ||
|
@@ -2397,6 +2427,8 @@ abstract ReadChangeStream.Builder setExistingPipelineOptions( | |
|
||
abstract ReadChangeStream.Builder setCreateOrUpdateMetadataTable(boolean shouldCreate); | ||
|
||
abstract ReadChangeStream.Builder setBacklogReplicationAdjustment(Duration adjustment); | ||
|
||
abstract ReadChangeStream build(); | ||
} | ||
} | ||
|
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,25 @@ | ||
/* | ||
* Licensed to the Apache Software Foundation (ASF) under one | ||
* or more contributor license agreements. See the NOTICE file | ||
* distributed with this work for additional information | ||
* regarding copyright ownership. The ASF licenses this file | ||
* to you 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.changestreams.dofn; | ||
|
||
import java.io.Serializable; | ||
import java.util.function.Supplier; | ||
|
||
/** Union of Supplier and Serializable interfaces to allow serialized supplier for testing. */ | ||
@FunctionalInterface | ||
interface SerializableSupplier<T> extends Supplier<T>, Serializable {} |
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.
Can this be Optional autovalue should support this
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.
I'd prefer to leave this as is for consistency with other optional parameters. We already do a lot of defaulting in ReadChangeStreamIO.expand instead of pushing down optional params into the DoFns and setting defaults there.
I don't have a strong preference for either approach but I think we should keep this consistent with the other parameters.