[
https://issues.apache.org/jira/browse/BEAM-6181?focusedWorklogId=186482&page=com.atlassian.jira.plugin.system.issuetabpanels:worklog-tabpanel#worklog-186482
]
ASF GitHub Bot logged work on BEAM-6181:
----------------------------------------
Author: ASF GitHub Bot
Created on: 17/Jan/19 18:23
Start Date: 17/Jan/19 18:23
Worklog Time Spent: 10m
Work Description: Ardagan commented on pull request #7323: [BEAM-6181]
Implemented msec counters support in FnApi world.
URL: https://github.com/apache/beam/pull/7323#discussion_r248786646
##########
File path:
runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/fn/control/UserMonitoringInfoToCounterUpdateTransformer.java
##########
@@ -0,0 +1,133 @@
+/*
+ * 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.runners.dataflow.worker.fn.control;
+
+import com.google.api.services.dataflow.model.CounterMetadata;
+import com.google.api.services.dataflow.model.CounterStructuredName;
+import com.google.api.services.dataflow.model.CounterStructuredNameAndMetadata;
+import com.google.api.services.dataflow.model.CounterUpdate;
+import java.util.Map;
+import java.util.Optional;
+import org.apache.beam.model.fnexecution.v1.BeamFnApi;
+import org.apache.beam.model.fnexecution.v1.BeamFnApi.MonitoringInfo;
+import org.apache.beam.model.fnexecution.v1.BeamFnApi.MonitoringInfoSpecs.Enum;
+import org.apache.beam.runners.core.metrics.SpecMonitoringInfoValidator;
+import
org.apache.beam.runners.dataflow.worker.DataflowExecutionContext.DataflowStepContext;
+import
org.apache.beam.runners.dataflow.worker.MetricsToCounterUpdateConverter.Origin;
+import
org.apache.beam.runners.dataflow.worker.counters.DataflowCounterUpdateExtractor;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Class for transforming MonitoringInfo's containing User counter values, to
relevant CounterUpdate
+ * proto.
+ */
+class UserMonitoringInfoToCounterUpdateTransformer
+ implements MonitoringInfoToCounterUpdateTransformer {
+
+ private static final Logger LOG =
LoggerFactory.getLogger(BeamFnMapTaskExecutor.class);
+
+ private final Map<String, DataflowStepContext> transformIdMapping;
+
+ private final SpecMonitoringInfoValidator specValidator;
+
+ public UserMonitoringInfoToCounterUpdateTransformer(
+ final SpecMonitoringInfoValidator specMonitoringInfoValidator,
+ final Map<String, DataflowStepContext> transformIdMapping) {
+ this.transformIdMapping = transformIdMapping;
+ this.specValidator = specMonitoringInfoValidator;
+ }
+
+ static final String BEAM_METRICS_USER_PREFIX =
+ Enum.USER_COUNTER
+ .getValueDescriptor()
+ .getOptions()
+ .getExtension(BeamFnApi.monitoringInfoSpec)
+ .getUrn();
+
+ private Optional<String> validate(MonitoringInfo monitoringInfo) {
+ Optional<String> validatorResult = specValidator.validate(monitoringInfo);
+ if (validatorResult.isPresent()) {
+ return validatorResult;
+ }
+
+ String urn = monitoringInfo.getUrn();
+ if (!urn.startsWith(BEAM_METRICS_USER_PREFIX)) {
+ throw new RuntimeException(
+ String.format(
+ "Received unexpected counter urn. Expected urn starting with:
%s, received: %s",
+ BEAM_METRICS_USER_PREFIX, urn));
+ }
+
+ final String ptransform = monitoringInfo.getLabelsMap().get("PTRANSFORM");
+ DataflowStepContext stepContext = transformIdMapping.get(ptransform);
+ if (stepContext == null) {
+ return Optional.of(
+ "Encountered user-counter MonitoringInfo with unknown ptransformId: "
+ + monitoringInfo.toString());
+ }
+ return Optional.empty();
+ }
+
+ /**
+ * Transforms user counter MonitoringInfo to relevant CounterUpdate.
+ *
+ * @return Relevant CounterUpdate or null if transformation failed.
+ */
+ @Override
+ public CounterUpdate transform(MonitoringInfo monitoringInfo) {
+ Optional<String> validationResult = validate(monitoringInfo);
+ if (validationResult.isPresent()) {
+ LOG.info(validationResult.get());
+ return null;
+ }
+
+ long value = monitoringInfo.getMetric().getCounterData().getInt64Value();
+ String urn = monitoringInfo.getUrn();
+
+ final String ptransform = monitoringInfo.getLabelsMap().get("PTRANSFORM");
+
+ CounterStructuredNameAndMetadata name = new
CounterStructuredNameAndMetadata();
+
+ String nameWithNamespace =
urn.substring(BEAM_METRICS_USER_PREFIX.length()).replace("^:", "");
+
+ final int lastColonIndex = nameWithNamespace.lastIndexOf(':');
Review comment:
ack
----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on GitHub and use the
URL above to go to the specific comment.
For queries about this service, please contact Infrastructure at:
[email protected]
Issue Time Tracking
-------------------
Worklog Id: (was: 186482)
Time Spent: 12h 50m (was: 12h 40m)
> Utilize MetricInfo for reporting user metrics in Portable Dataflow Java
> Runner.
> -------------------------------------------------------------------------------
>
> Key: BEAM-6181
> URL: https://issues.apache.org/jira/browse/BEAM-6181
> Project: Beam
> Issue Type: Bug
> Components: java-fn-execution
> Reporter: Mikhail Gryzykhin
> Assignee: Mikhail Gryzykhin
> Priority: Major
> Time Spent: 12h 50m
> Remaining Estimate: 0h
>
> New approach to report metrics in FnApi is to utilize MetricInfo structures.
> This approach is implemented in Python SDK and work is ongoing in Java SDK.
> This tasks includes plumbing User metrics reported via MetricInfos through
> Dataflow Java Runner.
--
This message was sent by Atlassian JIRA
(v7.6.3#76005)