-
Notifications
You must be signed in to change notification settings - Fork 4.3k
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
Introduce pipeline options to disable user counter and user stringset #33059
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 |
---|---|---|
|
@@ -18,6 +18,13 @@ | |
package org.apache.beam.sdk.metrics; | ||
|
||
import java.io.Serializable; | ||
import java.util.concurrent.atomic.AtomicReference; | ||
import org.apache.beam.sdk.annotations.Internal; | ||
import org.apache.beam.sdk.options.ExperimentalOptions; | ||
import org.apache.beam.sdk.options.PipelineOptions; | ||
import org.checkerframework.checker.nullness.qual.Nullable; | ||
import org.slf4j.Logger; | ||
import org.slf4j.LoggerFactory; | ||
|
||
/** | ||
* The <code>Metrics</code> is a utility class for producing various kinds of metrics for reporting | ||
|
@@ -50,9 +57,59 @@ | |
* example off how to query metrics. | ||
*/ | ||
public class Metrics { | ||
private static final Logger LOG = LoggerFactory.getLogger(Metrics.class); | ||
|
||
private Metrics() {} | ||
|
||
static class MetricsFlag { | ||
private static final AtomicReference<@Nullable MetricsFlag> INSTANCE = new AtomicReference<>(); | ||
final boolean counterDisabled; | ||
final boolean stringSetDisabled; | ||
|
||
private MetricsFlag(boolean counterDisabled, boolean stringSetDisabled) { | ||
this.counterDisabled = counterDisabled; | ||
this.stringSetDisabled = stringSetDisabled; | ||
} | ||
|
||
static boolean counterDisabled() { | ||
MetricsFlag flag = INSTANCE.get(); | ||
return flag != null && flag.counterDisabled; | ||
} | ||
|
||
static boolean stringSetDisabled() { | ||
MetricsFlag flag = INSTANCE.get(); | ||
return flag != null && flag.stringSetDisabled; | ||
} | ||
} | ||
|
||
/** | ||
* Initialize metrics flags if not already done so. | ||
* | ||
* <p>Should be called by worker at worker harness initialization. Should not be called by user | ||
* code (and it does not have an effect as the initialization completed before). | ||
*/ | ||
@Internal | ||
public static void setDefaultPipelineOptions(PipelineOptions options) { | ||
MetricsFlag flag = MetricsFlag.INSTANCE.get(); | ||
if (flag == null) { | ||
ExperimentalOptions exp = options.as(ExperimentalOptions.class); | ||
boolean counterDisabled = ExperimentalOptions.hasExperiment(exp, "disableCounterMetrics"); | ||
if (counterDisabled) { | ||
LOG.info("Counter metrics are disabled."); | ||
} | ||
boolean stringSetDisabled = ExperimentalOptions.hasExperiment(exp, "disableStringSetMetrics"); | ||
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. With this if a streaming customer want to disable StringSet or Lineage in their job they will need to restart the job passing this experiment. On service side we have an experiment to enable lineage (by default disabled). It is already present in pipeline option https://screenshot.googleplex.com/7WaPSvq85nHyCEt I was thinking we should keep lineage metric reporting disabled by default unless a job enable lineage. We have identified some cases where for long running streaming jobs the lineage information can get very large over time so it will be ideal to only capture this for job which opt for lineage. Can I please ask you while you are doing the same thing in this PR to add a
in file above and this similar function in Lineage class to set based on the lineage experiment above Thank you in advance. 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. Thanks, closed this one and opened an alternative PR: #33085 for this suggestion. |
||
if (stringSetDisabled) { | ||
LOG.info("StringSet metrics are disabled"); | ||
} | ||
MetricsFlag.INSTANCE.compareAndSet(null, new MetricsFlag(counterDisabled, stringSetDisabled)); | ||
} | ||
} | ||
|
||
@Internal | ||
static void resetDefaultPipelineOptions() { | ||
MetricsFlag.INSTANCE.set(null); | ||
} | ||
|
||
/** | ||
* Create a metric that can be incremented and decremented, and is aggregated by taking the sum. | ||
*/ | ||
|
@@ -174,6 +231,9 @@ private DelegatingStringSet(MetricName name) { | |
|
||
@Override | ||
public void add(String value) { | ||
if (MetricsFlag.stringSetDisabled()) { | ||
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. Should this part have unit test |
||
return; | ||
} | ||
MetricsContainer container = MetricsEnvironment.getCurrentContainer(); | ||
if (container != null) { | ||
container.getStringSet(name).add(value); | ||
|
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.
This makes this a little overloaded no? seems odd we set metric stuff in FS. But I see you want to do this in centralized place.
Would it better to have it separate in DataflowBatchWorkerHarness.java, StreamingDataflowWorker.java and FnHarness.java from where all this is called?
(I do not have a preference, just a though. Feel free to ignore this comment if you want to do it this way)
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.
setDefaultPipelineOptions will be called once per worker, on SDK harness startup, so the overloead is minimum
Separate call has much larger diff. Multiple runners has their own main() function (search for FileSystems.setDefaultPipelineOptions) give a hint