Skip to content

Commit

Permalink
Bigtable: add an experiments flag to enable client side metrics (apac…
Browse files Browse the repository at this point in the history
  • Loading branch information
mutianf authored May 10, 2023
1 parent 6c58db5 commit eb1852a
Show file tree
Hide file tree
Showing 2 changed files with 23 additions and 0 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -247,6 +247,13 @@
* .withStartTime(startTime));
* }</pre>
*
* <h3>Enable client side metrics</h3>
*
* <p>Client side metrics can be enabled with an experiments flag when you run the pipeline:
* --experiments=bigtable_enable_client_side_metrics. These metrics can provide additional insights
* to your job. You can read more about client side metrics in this documentation:
* https://cloud.google.com/bigtable/docs/client-side-metrics.
*
* <h3>Permissions</h3>
*
* <p>Permission requirements depend on the {@link PipelineRunner} that is used to execute the
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -28,6 +28,7 @@
import java.util.UUID;
import java.util.concurrent.ConcurrentHashMap;
import java.util.concurrent.atomic.AtomicInteger;
import org.apache.beam.sdk.options.ExperimentalOptions;
import org.apache.beam.sdk.options.PipelineOptions;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
Expand All @@ -48,6 +49,9 @@ class BigtableServiceFactory implements Serializable {
private static final ConcurrentHashMap<UUID, AtomicInteger> refCounts = new ConcurrentHashMap<>();
private static final Object lock = new Object();

private static final String BIGTABLE_ENABLE_CLIENT_SIDE_METRICS =
"bigtable_enable_client_side_metrics";

@AutoValue
abstract static class ConfigId implements Serializable {

Expand Down Expand Up @@ -115,6 +119,12 @@ BigtableServiceEntry getServiceForReading(
}
BigtableDataSettings settings =
BigtableConfigTranslator.translateReadToVeneerSettings(config, opts, pipelineOptions);

if (ExperimentalOptions.hasExperiment(pipelineOptions, BIGTABLE_ENABLE_CLIENT_SIDE_METRICS)) {
LOG.info("Enabling client side metrics");
BigtableDataSettings.enableBuiltinMetrics();
}

BigtableService service;
if (opts.getWaitTimeout() != null) {
service = new BigtableServiceImpl(settings, opts.getWaitTimeout());
Expand Down Expand Up @@ -155,6 +165,12 @@ BigtableServiceEntry getServiceForWriting(

BigtableDataSettings settings =
BigtableConfigTranslator.translateWriteToVeneerSettings(config, opts, pipelineOptions);

if (ExperimentalOptions.hasExperiment(pipelineOptions, BIGTABLE_ENABLE_CLIENT_SIDE_METRICS)) {
LOG.info("Enabling client side metrics");
BigtableDataSettings.enableBuiltinMetrics();
}

BigtableService service = new BigtableServiceImpl(settings);
entry = BigtableServiceEntry.create(configId, service);
entries.put(configId.id(), entry);
Expand Down

0 comments on commit eb1852a

Please sign in to comment.