public static BeamFnApi.Metrics.User.MetricName metricNameToProto(MetricName metricName) { return BeamFnApi.Metrics.User.MetricName.newBuilder() .setNamespace(metricName.getNamespace()) .setName(metricName.getName()) .build(); } }
public static MetricName metricNameFromProto(BeamFnApi.Metrics.User.MetricName protoMetricName) { return MetricName.named(protoMetricName.getNamespace(), protoMetricName.getName()); }
@Override public void onCompleted(ProcessBundleResponse response) { container.updateMetrics(stageName, response.getMonitoringInfosList()); } };
monitoringInfos.forEach( monitoringInfo -> { if (monitoringInfo.hasMetric()) { String urn = monitoringInfo.getUrn(); MetricName metricName = parseUrn(urn); BeamFnApi.Metric metric = monitoringInfo.getMetric(); if (metric.hasCounterData()) { BeamFnApi.CounterData counterData = metric.getCounterData(); org.apache.beam.sdk.metrics.Counter counter = metricsContainer.getCounter(metricName); if (counterData.getValueCase() == BeamFnApi.CounterData.ValueCase.INT64_VALUE) { counter.inc(counterData.getInt64Value()); } else { throw new IllegalArgumentException("Unsupported CounterData type: " + counterData); } else if (metric.hasDistributionData()) { BeamFnApi.DistributionData distributionData = metric.getDistributionData(); Distribution distribution = metricsContainer.getDistribution(metricName); if (distributionData.hasIntDistributionData()) { BeamFnApi.IntDistributionData intDistributionData = distributionData.getIntDistributionData(); distribution.update( intDistributionData.getSum(), intDistributionData.getCount(), intDistributionData.getMin(), intDistributionData.getMax()); } else { throw new IllegalArgumentException( "Unsupported DistributionData type: " + distributionData);
.getUnchecked(counterUpdate.getKey().stepName()) .add( BeamFnApi.Metrics.User.newBuilder() .setMetricName(metricNameToProto(counterUpdate.getKey().metricName())) .setCounterData( BeamFnApi.Metrics.User.CounterData.newBuilder() .setValue(counterUpdate.getUpdate())) .build()); .getUnchecked(gaugeUpdate.getKey().stepName()) .add( BeamFnApi.Metrics.User.newBuilder() .setMetricName(metricNameToProto(gaugeUpdate.getKey().metricName())) .setGaugeData( BeamFnApi.Metrics.User.GaugeData.newBuilder() .setValue(gaugeUpdate.getUpdate().value())) .build()); .getUnchecked(distributionUpdate.getKey().stepName()) .add( BeamFnApi.Metrics.User.newBuilder() .setMetricName(metricNameToProto(distributionUpdate.getKey().metricName())) .setDistributionData( BeamFnApi.Metrics.User.DistributionData.newBuilder() .setCount(distributionUpdate.getUpdate().count()) .setMax(distributionUpdate.getUpdate().max()) .setMin(distributionUpdate.getUpdate().min()) .setSum(distributionUpdate.getUpdate().sum())) .build());
MetricKey.create(ptransformName, metricNameFromProto(userMetricUpdate.getMetricName())); switch (userMetricUpdate.getDataCase()) { case COUNTER_DATA: counterUpdates.add( MetricUpdates.MetricUpdate.create( metricKey, userMetricUpdate.getCounterData().getValue())); break; case DISTRIBUTION_DATA: metricKey, DistributionData.create( userMetricUpdate.getDistributionData().getSum(), userMetricUpdate.getDistributionData().getCount(), userMetricUpdate.getDistributionData().getMin(), userMetricUpdate.getDistributionData().getMax()))); break; case GAUGE_DATA: gaugeUpdates.add( MetricUpdates.MetricUpdate.create( metricKey, GaugeData.create(userMetricUpdate.getGaugeData().getValue()))); break; case DATA_NOT_SET:
ExecutorService dataExecutor = Executors.newCachedThreadPool(); ProvisionInfo provisionInfo = ProvisionInfo.newBuilder() .setJobId("id") .setJobName("reference") .setPipelineOptions(options) .setWorkerId("foo") .setResourceLimits(Resources.getDefaultInstance()) .build(); try (GrpcFnServer<GrpcLoggingService> logging = GrpcFnServer.allocatePortAndCreateFor(
@Override public void onProgress(ProcessBundleProgressResponse progress) { flinkMetricContainer.updateMetrics(stepName, progress.getMonitoringInfosList()); }
/** * Parse a {@link MetricName} from a {@link * org.apache.beam.model.fnexecution.v1.BeamFnApi.MonitoringInfoUrns.Enum} * * <p>Should be consistent with {@code parse_namespace_and_name} in monitoring_infos.py * * <p>TODO: not flink-specific; where should it live? */ public static MetricName parseUrn(String urn) { if (urn.startsWith(USER_COUNTER_URN_PREFIX.toString())) { urn = urn.substring(USER_COUNTER_URN_PREFIX.toString().length()); } // If it is not a user counter, just use the first part of the URN, i.e. 'beam' String[] pieces = urn.split(":", 2); if (pieces.length != 2) { throw new IllegalArgumentException("Invalid metric URN: " + urn); } return MetricName.named(pieces[0], pieces[1]); }
@Override public void onCompleted(ProcessBundleResponse response) { flinkMetricContainer.updateMetrics(stepName, response.getMonitoringInfosList()); } };
@Override public void onProgress(ProcessBundleProgressResponse progress) { container.updateMetrics(stageName, progress.getMonitoringInfosList()); }