Skip to content

Commit 74f1f70

Browse files
authored
Generic throttling metrics namespace and BigtableIO write throttling counter (#31924)
* BigtableIO write throttling counter * Introduce a generic throttling namespace and counter * Dataflow accumulates throttling time from generic throttling counter * Apply throttling counter to BigtableIO write * default to 3 min for throttlingReportTargetMs
1 parent 799405c commit 74f1f70

File tree

19 files changed

+233
-50
lines changed

19 files changed

+233
-50
lines changed

runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/BatchModeExecutionContext.java

Lines changed: 16 additions & 10 deletions
Original file line numberDiff line numberDiff line change
@@ -40,6 +40,7 @@
4040
import org.apache.beam.runners.dataflow.worker.profiler.ScopedProfiler.ProfileScope;
4141
import org.apache.beam.sdk.coders.Coder;
4242
import org.apache.beam.sdk.metrics.MetricName;
43+
import org.apache.beam.sdk.metrics.Metrics;
4344
import org.apache.beam.sdk.metrics.MetricsContainer;
4445
import org.apache.beam.sdk.options.PipelineOptions;
4546
import org.apache.beam.sdk.state.TimeDomain;
@@ -68,9 +69,6 @@ public class BatchModeExecutionContext
6869
private Object key;
6970

7071
private final MetricsContainerRegistry<MetricsContainerImpl> containerRegistry;
71-
72-
// TODO(https://github.com/apache/beam/issues/19632): Move throttle time Metric to a dedicated
73-
// namespace.
7472
protected static final String DATASTORE_THROTTLE_TIME_NAMESPACE =
7573
"org.apache.beam.sdk.io.gcp.datastore.DatastoreV1$DatastoreWriterFn";
7674
protected static final String HTTP_CLIENT_API_THROTTLE_TIME_NAMESPACE =
@@ -79,7 +77,6 @@ public class BatchModeExecutionContext
7977
"org.apache.beam.sdk.io.gcp.bigquery.BigQueryServicesImpl$DatasetServiceImpl";
8078
protected static final String BIGQUERY_READ_THROTTLE_TIME_NAMESPACE =
8179
"org.apache.beam.sdk.io.gcp.bigquery.BigQueryServicesImpl$StorageClientImpl";
82-
protected static final String THROTTLE_TIME_COUNTER_NAME = "throttling-msecs";
8380

8481
// TODO(BEAM-31814): Remove once Dataflow legacy runner supports this.
8582
private final boolean populateStringSetMetrics;
@@ -550,34 +547,43 @@ public Iterable<CounterUpdate> extractMsecCounters(boolean isFinalUpdate) {
550547
public Long extractThrottleTime() {
551548
long totalThrottleMsecs = 0L;
552549
for (MetricsContainerImpl container : containerRegistry.getContainers()) {
553-
// TODO(https://github.com/apache/beam/issues/19632): Update throttling counters to use
554-
// generic throttling-msecs metric.
550+
CounterCell userThrottlingTime =
551+
container.tryGetCounter(
552+
MetricName.named(
553+
Metrics.THROTTLE_TIME_NAMESPACE, Metrics.THROTTLE_TIME_COUNTER_NAME));
554+
if (userThrottlingTime != null) {
555+
totalThrottleMsecs += userThrottlingTime.getCumulative();
556+
}
557+
555558
CounterCell dataStoreThrottlingTime =
556559
container.tryGetCounter(
557-
MetricName.named(DATASTORE_THROTTLE_TIME_NAMESPACE, THROTTLE_TIME_COUNTER_NAME));
560+
MetricName.named(
561+
DATASTORE_THROTTLE_TIME_NAMESPACE, Metrics.THROTTLE_TIME_COUNTER_NAME));
558562
if (dataStoreThrottlingTime != null) {
559563
totalThrottleMsecs += dataStoreThrottlingTime.getCumulative();
560564
}
561565

562566
CounterCell httpClientApiThrottlingTime =
563567
container.tryGetCounter(
564568
MetricName.named(
565-
HTTP_CLIENT_API_THROTTLE_TIME_NAMESPACE, THROTTLE_TIME_COUNTER_NAME));
569+
HTTP_CLIENT_API_THROTTLE_TIME_NAMESPACE, Metrics.THROTTLE_TIME_COUNTER_NAME));
566570
if (httpClientApiThrottlingTime != null) {
567571
totalThrottleMsecs += httpClientApiThrottlingTime.getCumulative();
568572
}
569573

570574
CounterCell bigqueryStreamingInsertThrottleTime =
571575
container.tryGetCounter(
572576
MetricName.named(
573-
BIGQUERY_STREAMING_INSERT_THROTTLE_TIME_NAMESPACE, THROTTLE_TIME_COUNTER_NAME));
577+
BIGQUERY_STREAMING_INSERT_THROTTLE_TIME_NAMESPACE,
578+
Metrics.THROTTLE_TIME_COUNTER_NAME));
574579
if (bigqueryStreamingInsertThrottleTime != null) {
575580
totalThrottleMsecs += bigqueryStreamingInsertThrottleTime.getCumulative();
576581
}
577582

578583
CounterCell bigqueryReadThrottleTime =
579584
container.tryGetCounter(
580-
MetricName.named(BIGQUERY_READ_THROTTLE_TIME_NAMESPACE, THROTTLE_TIME_COUNTER_NAME));
585+
MetricName.named(
586+
BIGQUERY_READ_THROTTLE_TIME_NAMESPACE, Metrics.THROTTLE_TIME_COUNTER_NAME));
581587
if (bigqueryReadThrottleTime != null) {
582588
totalThrottleMsecs += bigqueryReadThrottleTime.getCumulative();
583589
}

runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/DataflowSystemMetrics.java

Lines changed: 2 additions & 3 deletions
Original file line numberDiff line numberDiff line change
@@ -20,15 +20,14 @@
2020
import org.apache.beam.runners.dataflow.worker.counters.CounterName;
2121
import org.apache.beam.runners.dataflow.worker.counters.NameContext;
2222
import org.apache.beam.sdk.metrics.MetricName;
23+
import org.apache.beam.sdk.metrics.Metrics;
2324
import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Preconditions;
2425

2526
/** This holds system metrics related constants used in Batch and Streaming. */
2627
public class DataflowSystemMetrics {
2728

2829
public static final MetricName THROTTLING_MSECS_METRIC_NAME =
29-
MetricName.named("dataflow-throttling-metrics", "throttling-msecs");
30-
31-
// TODO: Provide an utility in SDK 'ThrottlingReporter' to update throttling time.
30+
MetricName.named("dataflow-throttling-metrics", Metrics.THROTTLE_TIME_COUNTER_NAME);
3231

3332
/** System counters populated by streaming dataflow workers. */
3433
public enum StreamingSystemCounterNames {

runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/StreamingDataflowWorker.java

Lines changed: 2 additions & 10 deletions
Original file line numberDiff line numberDiff line change
@@ -93,13 +93,13 @@
9393
import org.apache.beam.sdk.fn.JvmInitializers;
9494
import org.apache.beam.sdk.io.FileSystems;
9595
import org.apache.beam.sdk.io.gcp.bigquery.BigQuerySinkMetrics;
96-
import org.apache.beam.sdk.metrics.MetricName;
9796
import org.apache.beam.sdk.metrics.MetricsEnvironment;
9897
import org.apache.beam.sdk.util.construction.CoderTranslation;
9998
import org.apache.beam.vendor.grpc.v1p60p1.io.grpc.ManagedChannel;
10099
import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.annotations.VisibleForTesting;
101100
import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Preconditions;
102-
import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.*;
101+
import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableSet;
102+
import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.Iterables;
103103
import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.net.HostAndPort;
104104
import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.util.concurrent.ThreadFactoryBuilder;
105105
import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.util.concurrent.Uninterruptibles;
@@ -113,14 +113,6 @@
113113
"nullness" // TODO(https://github.com/apache/beam/issues/20497)
114114
})
115115
public class StreamingDataflowWorker {
116-
117-
// TODO(https://github.com/apache/beam/issues/19632): Update throttling counters to use generic
118-
// throttling-msecs metric.
119-
public static final MetricName BIGQUERY_STREAMING_INSERT_THROTTLE_TIME =
120-
MetricName.named(
121-
"org.apache.beam.sdk.io.gcp.bigquery.BigQueryServicesImpl$DatasetServiceImpl",
122-
"throttling-msecs");
123-
124116
/**
125117
* Sinks are marked 'full' in {@link StreamingModeExecutionContext} once the amount of data sinked
126118
* (across all the sinks, if there are more than one) reaches this limit. This serves as hint for

runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/streaming/StageInfo.java

Lines changed: 3 additions & 11 deletions
Original file line numberDiff line numberDiff line change
@@ -17,7 +17,7 @@
1717
*/
1818
package org.apache.beam.runners.dataflow.worker.streaming;
1919

20-
import static org.apache.beam.runners.dataflow.worker.DataflowSystemMetrics.THROTTLING_MSECS_METRIC_NAME;
20+
import static org.apache.beam.sdk.metrics.Metrics.THROTTLE_TIME_COUNTER_NAME;
2121

2222
import com.google.api.services.dataflow.model.CounterStructuredName;
2323
import com.google.api.services.dataflow.model.CounterUpdate;
@@ -28,7 +28,6 @@
2828
import java.util.List;
2929
import org.apache.beam.runners.dataflow.worker.DataflowSystemMetrics;
3030
import org.apache.beam.runners.dataflow.worker.MetricsContainerRegistry;
31-
import org.apache.beam.runners.dataflow.worker.StreamingDataflowWorker;
3231
import org.apache.beam.runners.dataflow.worker.StreamingModeExecutionContext.StreamingModeExecutionStateRegistry;
3332
import org.apache.beam.runners.dataflow.worker.StreamingStepMetricsContainer;
3433
import org.apache.beam.runners.dataflow.worker.counters.Counter;
@@ -93,20 +92,13 @@ public List<CounterUpdate> extractCounterUpdates() {
9392
}
9493

9594
/**
96-
* Checks if the step counter affects any per-stage counters. Currently 'throttled_millis' is the
95+
* Checks if the step counter affects any per-stage counters. Currently 'throttled-msecs' is the
9796
* only counter updated.
9897
*/
9998
private void translateKnownStepCounters(CounterUpdate stepCounterUpdate) {
10099
CounterStructuredName structuredName =
101100
stepCounterUpdate.getStructuredNameAndMetadata().getName();
102-
if ((THROTTLING_MSECS_METRIC_NAME.getNamespace().equals(structuredName.getOriginNamespace())
103-
&& THROTTLING_MSECS_METRIC_NAME.getName().equals(structuredName.getName()))
104-
|| (StreamingDataflowWorker.BIGQUERY_STREAMING_INSERT_THROTTLE_TIME
105-
.getNamespace()
106-
.equals(structuredName.getOriginNamespace())
107-
&& StreamingDataflowWorker.BIGQUERY_STREAMING_INSERT_THROTTLE_TIME
108-
.getName()
109-
.equals(structuredName.getName()))) {
101+
if (THROTTLE_TIME_COUNTER_NAME.equals(structuredName.getName())) {
110102
long msecs = DataflowCounterUpdateExtractor.splitIntToLong(stepCounterUpdate.getInteger());
111103
if (msecs > 0) {
112104
throttledMsecs().addValue(msecs);

runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/BatchModeExecutionContextTest.java

Lines changed: 2 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -43,6 +43,7 @@
4343
import org.apache.beam.sdk.metrics.Counter;
4444
import org.apache.beam.sdk.metrics.Distribution;
4545
import org.apache.beam.sdk.metrics.MetricName;
46+
import org.apache.beam.sdk.metrics.Metrics;
4647
import org.apache.beam.sdk.metrics.MetricsContainer;
4748
import org.apache.beam.sdk.metrics.StringSet;
4849
import org.apache.beam.sdk.options.PipelineOptionsFactory;
@@ -266,7 +267,7 @@ public void extractThrottleTimeCounters() {
266267
.getCounter(
267268
MetricName.named(
268269
BatchModeExecutionContext.DATASTORE_THROTTLE_TIME_NAMESPACE,
269-
BatchModeExecutionContext.THROTTLE_TIME_COUNTER_NAME));
270+
Metrics.THROTTLE_TIME_COUNTER_NAME));
270271
counter.inc(12000);
271272
counter.inc(17000);
272273
counter.inc(1000);

sdks/java/core/src/main/java/org/apache/beam/sdk/metrics/Metrics.java

Lines changed: 7 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -109,6 +109,13 @@ public static StringSet stringSet(Class<?> namespace, String name) {
109109
return new DelegatingStringSet(MetricName.named(namespace, name));
110110
}
111111

112+
/*
113+
* A dedicated namespace for client throttling time. User DoFn can increment this metrics and then
114+
* runner will put back pressure on scaling decision, if supported.
115+
*/
116+
public static final String THROTTLE_TIME_NAMESPACE = "beam-throttling-metrics";
117+
public static final String THROTTLE_TIME_COUNTER_NAME = "throttling-msecs";
118+
112119
/**
113120
* Implementation of {@link Distribution} that delegates to the instance for the current context.
114121
*/

sdks/java/core/src/main/java/org/apache/beam/sdk/metrics/MetricsEnvironment.java

Lines changed: 8 additions & 3 deletions
Original file line numberDiff line numberDiff line change
@@ -22,6 +22,7 @@
2222
import java.util.concurrent.atomic.AtomicBoolean;
2323
import java.util.concurrent.atomic.AtomicReference;
2424
import org.apache.beam.sdk.annotations.Internal;
25+
import org.apache.beam.sdk.util.StringUtils;
2526
import org.checkerframework.checker.nullness.qual.NonNull;
2627
import org.checkerframework.checker.nullness.qual.Nullable;
2728
import org.slf4j.Logger;
@@ -134,10 +135,14 @@ public void close() throws IOException {
134135
if (container == null && REPORTED_MISSING_CONTAINER.compareAndSet(false, true)) {
135136
if (isMetricsSupported()) {
136137
LOG.error(
137-
"Unable to update metrics on the current thread. "
138-
+ "Most likely caused by using metrics outside the managed work-execution thread.");
138+
"Unable to update metrics on the current thread. Most likely caused by using metrics "
139+
+ "outside the managed work-execution thread:\n {}",
140+
StringUtils.arrayToNewlines(Thread.currentThread().getStackTrace(), 10));
139141
} else {
140-
LOG.warn("Reporting metrics are not supported in the current execution environment.");
142+
// rate limiting this log as it can be emitted each time metrics incremented
143+
LOG.warn(
144+
"Reporting metrics are not supported in the current execution environment:\n {}",
145+
StringUtils.arrayToNewlines(Thread.currentThread().getStackTrace(), 10));
141146
}
142147
}
143148
return container;

sdks/java/core/src/main/java/org/apache/beam/sdk/util/StringUtils.java

Lines changed: 35 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -22,6 +22,7 @@
2222
import java.util.ArrayList;
2323
import java.util.List;
2424
import org.apache.beam.sdk.annotations.Internal;
25+
import org.checkerframework.checker.nullness.qual.Nullable;
2526

2627
/** Utilities for working with JSON and other human-readable string formats. */
2728
@Internal
@@ -143,4 +144,38 @@ public static int getLevenshteinDistance(final String s, final String t) {
143144

144145
return v1[t.length()];
145146
}
147+
148+
/**
149+
* Convert Array to new lined String. Truncate to first {@code maxLine} elements.
150+
*
151+
* <p>Useful to truncate stacktrace and for logging.
152+
*/
153+
public static String arrayToNewlines(Object[] array, int maxLine) {
154+
int n = (maxLine > 0 && array.length > maxLine) ? maxLine : array.length;
155+
StringBuilder b = new StringBuilder();
156+
for (int i = 0; i < n; i++) {
157+
b.append(array[i]);
158+
b.append("\n");
159+
}
160+
if (array.length > maxLine) {
161+
b.append("...\n");
162+
}
163+
return b.toString();
164+
}
165+
166+
/**
167+
* Truncate String if length greater than maxLen, and append "..." to the end. Handles null.
168+
*
169+
* <p>Useful to truncate long logging message.
170+
*/
171+
public static String leftTruncate(@Nullable Object element, int maxLen) {
172+
if (element == null) {
173+
return "";
174+
}
175+
String s = element.toString();
176+
if (s.length() > maxLen) {
177+
return s.substring(0, maxLen) + "...";
178+
}
179+
return s;
180+
}
146181
}

sdks/java/core/src/test/java/org/apache/beam/sdk/util/StringUtilsTest.java

Lines changed: 23 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -17,9 +17,13 @@
1717
*/
1818
package org.apache.beam.sdk.util;
1919

20+
import static org.apache.commons.lang3.StringUtils.countMatches;
2021
import static org.junit.Assert.assertArrayEquals;
2122
import static org.junit.Assert.assertEquals;
23+
import static org.junit.Assert.assertTrue;
2224

25+
import java.util.UUID;
26+
import java.util.stream.IntStream;
2327
import org.junit.Test;
2428
import org.junit.runner.RunWith;
2529
import org.junit.runners.JUnit4;
@@ -54,4 +58,23 @@ public void testLevenshteinDistance() {
5458
assertEquals(1, StringUtils.getLevenshteinDistance("abc", "ab1c")); // insertion
5559
assertEquals(1, StringUtils.getLevenshteinDistance("abc", "a1c")); // modification
5660
}
61+
62+
@Test
63+
public void testArrayToNewlines() {
64+
Object[] uuids = IntStream.range(1, 10).mapToObj(unused -> UUID.randomUUID()).toArray();
65+
66+
String r1 = StringUtils.arrayToNewlines(uuids, 6);
67+
assertTrue(r1.endsWith("...\n"));
68+
assertEquals(7, countMatches(r1, "\n"));
69+
String r2 = StringUtils.arrayToNewlines(uuids, 15);
70+
String r3 = StringUtils.arrayToNewlines(uuids, 10);
71+
assertEquals(r3, r2);
72+
}
73+
74+
@Test
75+
public void testLeftTruncate() {
76+
assertEquals("", StringUtils.leftTruncate(null, 3));
77+
assertEquals("", StringUtils.leftTruncate("", 3));
78+
assertEquals("abc...", StringUtils.leftTruncate("abcd", 3));
79+
}
5780
}

sdks/java/extensions/google-cloud-platform-core/src/main/java/org/apache/beam/sdk/extensions/gcp/util/RetryHttpRequestInitializer.java

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -75,7 +75,7 @@ private static class LoggingHttpBackOffHandler
7575
private final Set<Integer> ignoredResponseCodes;
7676
// aggregate the total time spent in exponential backoff
7777
private final Counter throttlingMsecs =
78-
Metrics.counter(LoggingHttpBackOffHandler.class, "throttling-msecs");
78+
Metrics.counter(LoggingHttpBackOffHandler.class, Metrics.THROTTLE_TIME_COUNTER_NAME);
7979
private int ioExceptionRetries;
8080
private int unsuccessfulResponseRetries;
8181
private @Nullable CustomHttpErrors customHttpErrors;

0 commit comments

Comments
 (0)