You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@samza.apache.org by GitBox <gi...@apache.org> on 2021/11/11 17:58:56 UTC

[GitHub] [samza] cameronlee314 opened a new pull request #1558: SAMZA-2707: Convert metrics snapshot reporter code to java

cameronlee314 opened a new pull request #1558:
URL: https://github.com/apache/samza/pull/1558


   Issues: Code related to metrics snapshot reporting is in Scala, which is inconsistent with most of other Samza code. Scala is less well-known across Samza, it takes slightly longer to build, and Scala versioning is sometimes hard to deal with.
   
   Changes:
   1. Convert some classes (`org.apache.samza.metrics.reporter.Metrics*`) related to metrics snapshot reporting to Java.
   2. Added `RegistryWithSource` Java class to help with implementing `MetricsReporter`s in Java (in Scala, a tuple was used instead).
   3. Added some tests for the classes that were converted to Java
   
   Testing: Added tests and verified they passed on the Scala versions of the classes, and then verified they passed on Java versions of the classes
   
   API changes:
   1. (backwards compatible) Added `RegistryWithSource` Java class to help with implementing `MetricsReporter`s in Java
   2. (applies to usage of `org.apache.samza.metrics.reporter.Metrics*` classes from `samza-core`) Converted classes no longer provide direct access to constructor parameters (e.g. `Metrics.immutableMetrics()` is no longer accessible). "Container" objects like `Metrics`, `MetricsSnapshot`, and `MetricsHeader` still have getters for their fields.
   3. (applies to usage of `org.apache.samza.metrics.reporter.Metrics*` classes from `samza-core`) "Container" objects like `Metrics`, `MetricsSnapshot`, and `MetricsHeader` no longer treat their fields as `@BeanProperty`, so they no longer have setters. It's better that container objects are immutable anyways, and there weren't any usages of the setters within Samza code.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@samza.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [samza] cameronlee314 commented on a change in pull request #1558: SAMZA-2707: Convert metrics snapshot reporter code to java

Posted by GitBox <gi...@apache.org>.
cameronlee314 commented on a change in pull request #1558:
URL: https://github.com/apache/samza/pull/1558#discussion_r749594352



##########
File path: samza-core/src/main/java/org/apache/samza/metrics/reporter/MetricsSnapshotReporter.java
##########
@@ -0,0 +1,228 @@
+/*
+ * 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.samza.metrics.reporter;
+
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Set;
+import java.util.concurrent.Executors;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.TimeUnit;
+import com.google.common.util.concurrent.ThreadFactoryBuilder;
+import org.apache.samza.SamzaException;
+import org.apache.samza.config.ShellCommandConfig;
+import org.apache.samza.metrics.Counter;
+import org.apache.samza.metrics.Gauge;
+import org.apache.samza.metrics.MetricsReporter;
+import org.apache.samza.metrics.MetricsVisitor;
+import org.apache.samza.metrics.ReadableMetricsRegistry;
+import org.apache.samza.metrics.RegistryWithSource;
+import org.apache.samza.metrics.Timer;
+import org.apache.samza.serializers.Serializer;
+import org.apache.samza.system.OutgoingMessageEnvelope;
+import org.apache.samza.system.SystemProducer;
+import org.apache.samza.system.SystemStream;
+import org.apache.samza.util.Clock;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+/**
+ * MetricsSnapshotReporter is a generic metrics reporter that sends metrics to a stream.
+ *
+ * jobName // my-samza-job
+ * jobId // an id that differentiates multiple executions of the same job
+ * taskName // container_567890
+ * host // eat1-app128.gird
+ * version // 0.0.1
+ * blacklist // Regex of metrics to ignore when flushing
+ */
+public class MetricsSnapshotReporter implements MetricsReporter, Runnable {
+  private static final Logger LOG = LoggerFactory.getLogger(MetricsSnapshotReporter.class);
+
+  private final SystemProducer producer;
+  private final SystemStream out;
+  private final int reportingInterval;
+  private final String jobName;
+  private final String jobId;
+  private final String containerName;
+  private final String version;
+  private final String samzaVersion;
+  private final String host;
+  private final Serializer<MetricsSnapshot> serializer;
+  private final Optional<String> blacklist;
+  private final Clock clock;
+
+  private final String execEnvironmentContainerId;
+  private final ScheduledExecutorService executor;
+  private final long resetTime;
+  private final List<RegistryWithSource> registries = new ArrayList<>();
+  private final Set<String> blacklistedMetrics = new HashSet<>();
+
+  public MetricsSnapshotReporter(SystemProducer producer, SystemStream out, int reportingInterval, String jobName,
+      String jobId, String containerName, String version, String samzaVersion, String host,
+      Serializer<MetricsSnapshot> serializer, Optional<String> blacklist, Clock clock) {
+    this.producer = producer;
+    this.out = out;
+    this.reportingInterval = reportingInterval;
+    this.jobName = jobName;
+    this.jobId = jobId;
+    this.containerName = containerName;
+    this.version = version;
+    this.samzaVersion = samzaVersion;
+    this.host = host;
+    this.serializer = serializer;
+    this.blacklist = blacklist;
+    this.clock = clock;
+
+    this.execEnvironmentContainerId =
+        Optional.ofNullable(System.getenv(ShellCommandConfig.ENV_EXECUTION_ENV_CONTAINER_ID)).orElse("");
+    this.executor = Executors.newSingleThreadScheduledExecutor(
+        new ThreadFactoryBuilder().setNameFormat("Samza MetricsSnapshotReporter Thread-%d").setDaemon(true).build());
+    this.resetTime = this.clock.currentTimeMillis();
+    LOG.info(
+        "got metrics snapshot reporter properties [job name: {}, job id: {}, containerName: {}, version: {}, samzaVersion: {}, host: {}, reportingInterval {}]",
+        jobName, jobId, containerName, version, samzaVersion, host, reportingInterval);
+  }
+
+  @Override
+  public void start() {
+    LOG.info("Starting producer.");
+    this.producer.start();
+    LOG.info("Starting reporter timer.");
+    this.executor.scheduleWithFixedDelay(this, 0, reportingInterval, TimeUnit.SECONDS);
+  }
+
+  @Override
+  public void register(String source, ReadableMetricsRegistry registry) {
+    this.registries.add(new RegistryWithSource(source, registry));
+    LOG.info("Registering {} with producer.", source);
+    this.producer.register(source);
+  }
+
+  @Override
+  public void stop() {
+    // Scheduling an event with 0 delay to ensure flushing of metrics one last time before shutdown
+    this.executor.schedule(this, 0, TimeUnit.SECONDS);
+    LOG.info("Stopping reporter timer.");
+    // Allow the scheduled task above to finish, and block for termination (for max 60 seconds)
+    this.executor.shutdown();
+    try {
+      this.executor.awaitTermination(60, TimeUnit.SECONDS);
+    } catch (InterruptedException e) {
+      throw new SamzaException(e);
+    }
+    LOG.info("Stopping producer.");
+    this.producer.stop();
+    if (!this.executor.isTerminated()) {
+      LOG.warn("Unable to shutdown reporter timer.");
+    }
+  }
+
+  @Override
+  public void run() {
+    try {
+      innerRun();
+    } catch (Exception e) {
+      // Ignore all exceptions - because subsequent executions of this scheduled task will be suppressed
+      // by the executor if the current task throws an unhandled exception.
+      LOG.warn("Error while reporting metrics. Will retry in " + reportingInterval + " seconds.", e);
+    }
+  }
+
+  public void innerRun() {
+    LOG.debug("Begin flushing metrics.");
+    for (RegistryWithSource registryWithSource : this.registries) {
+      String source = registryWithSource.getSource();
+      ReadableMetricsRegistry registry = registryWithSource.getRegistry();
+      LOG.debug("Flushing metrics for {}.", source);
+      Map<String, Map<String, Object>> metricsMsg = new HashMap<>();
+
+      // metrics
+      registry.getGroups().forEach(group -> {
+        Map<String, Object> groupMsg = new HashMap<>();
+        registry.getGroup(group).forEach((name, metric) -> {
+          if (!shouldIgnore(group, name)) {
+            metric.visit(new MetricsVisitor() {
+              @Override
+              public void counter(Counter counter) {
+                groupMsg.put(name, counter.getCount());
+              }
+
+              @Override
+              public <T> void gauge(Gauge<T> gauge) {
+                groupMsg.put(name, gauge.getValue());
+              }
+
+              @Override
+              public void timer(Timer timer) {
+                groupMsg.put(name, timer.getSnapshot().getAverage());
+              }
+            });
+          }
+        });
+
+        // dont emit empty groups
+        if (!groupMsg.isEmpty()) {
+          metricsMsg.put(group, groupMsg);
+        }
+      });
+
+      // publish to Kafka only if the metricsMsg carries any metrics
+      if (!metricsMsg.isEmpty()) {
+        MetricsHeader header =
+            new MetricsHeader(this.jobName, this.jobId, this.containerName, this.execEnvironmentContainerId, source,
+                this.version, this.samzaVersion, this.host, this.clock.currentTimeMillis(), this.resetTime);
+        Metrics metrics = new Metrics(metricsMsg);
+        LOG.debug("Flushing metrics for {} to {} with header and map: header={}, map={}.", source, out,
+            header.getAsMap(), metrics.getAsMap());
+        MetricsSnapshot metricsSnapshot = new MetricsSnapshot(header, metrics);
+        Object maybeSerialized = (this.serializer != null) ? this.serializer.toBytes(metricsSnapshot) : metricsSnapshot;
+        try {
+          this.producer.send(source, new OutgoingMessageEnvelope(this.out, this.host, null, maybeSerialized));
+          // Always flush, since we don't want metrics to get batched up.
+          this.producer.flush(source);
+        } catch (Exception e) {
+          LOG.error(String.format("Exception when flushing metrics for source %s", source), e);
+        }
+      }
+    }
+    LOG.debug("Finished flushing metrics.");
+  }
+
+  protected boolean shouldIgnore(String group, String metricName) {
+    boolean isBlacklisted = this.blacklist.isPresent();

Review comment:
       The check for `isBlacklisted` in the `if` does short-circuit the flow, so it is already returning reasonably early.
   Since this class caches the blacklisted metrics, the flow here is slightly more complicated than it initially seems. I also initially felt there was a chance to simplify the logic, but then I just did the straightforward conversion because of the caching flow.

##########
File path: samza-core/src/main/java/org/apache/samza/metrics/reporter/MetricsSnapshotReporter.java
##########
@@ -0,0 +1,228 @@
+/*
+ * 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.samza.metrics.reporter;
+
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Set;
+import java.util.concurrent.Executors;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.TimeUnit;
+import com.google.common.util.concurrent.ThreadFactoryBuilder;
+import org.apache.samza.SamzaException;
+import org.apache.samza.config.ShellCommandConfig;
+import org.apache.samza.metrics.Counter;
+import org.apache.samza.metrics.Gauge;
+import org.apache.samza.metrics.MetricsReporter;
+import org.apache.samza.metrics.MetricsVisitor;
+import org.apache.samza.metrics.ReadableMetricsRegistry;
+import org.apache.samza.metrics.RegistryWithSource;
+import org.apache.samza.metrics.Timer;
+import org.apache.samza.serializers.Serializer;
+import org.apache.samza.system.OutgoingMessageEnvelope;
+import org.apache.samza.system.SystemProducer;
+import org.apache.samza.system.SystemStream;
+import org.apache.samza.util.Clock;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+/**
+ * MetricsSnapshotReporter is a generic metrics reporter that sends metrics to a stream.
+ *
+ * jobName // my-samza-job
+ * jobId // an id that differentiates multiple executions of the same job
+ * taskName // container_567890
+ * host // eat1-app128.gird
+ * version // 0.0.1
+ * blacklist // Regex of metrics to ignore when flushing
+ */
+public class MetricsSnapshotReporter implements MetricsReporter, Runnable {
+  private static final Logger LOG = LoggerFactory.getLogger(MetricsSnapshotReporter.class);
+
+  private final SystemProducer producer;
+  private final SystemStream out;
+  private final int reportingInterval;
+  private final String jobName;
+  private final String jobId;
+  private final String containerName;
+  private final String version;
+  private final String samzaVersion;
+  private final String host;
+  private final Serializer<MetricsSnapshot> serializer;
+  private final Optional<String> blacklist;
+  private final Clock clock;
+
+  private final String execEnvironmentContainerId;
+  private final ScheduledExecutorService executor;
+  private final long resetTime;
+  private final List<RegistryWithSource> registries = new ArrayList<>();
+  private final Set<String> blacklistedMetrics = new HashSet<>();
+
+  public MetricsSnapshotReporter(SystemProducer producer, SystemStream out, int reportingInterval, String jobName,
+      String jobId, String containerName, String version, String samzaVersion, String host,
+      Serializer<MetricsSnapshot> serializer, Optional<String> blacklist, Clock clock) {
+    this.producer = producer;
+    this.out = out;
+    this.reportingInterval = reportingInterval;
+    this.jobName = jobName;
+    this.jobId = jobId;
+    this.containerName = containerName;
+    this.version = version;
+    this.samzaVersion = samzaVersion;
+    this.host = host;
+    this.serializer = serializer;
+    this.blacklist = blacklist;
+    this.clock = clock;
+
+    this.execEnvironmentContainerId =
+        Optional.ofNullable(System.getenv(ShellCommandConfig.ENV_EXECUTION_ENV_CONTAINER_ID)).orElse("");
+    this.executor = Executors.newSingleThreadScheduledExecutor(
+        new ThreadFactoryBuilder().setNameFormat("Samza MetricsSnapshotReporter Thread-%d").setDaemon(true).build());
+    this.resetTime = this.clock.currentTimeMillis();
+    LOG.info(
+        "got metrics snapshot reporter properties [job name: {}, job id: {}, containerName: {}, version: {}, samzaVersion: {}, host: {}, reportingInterval {}]",
+        jobName, jobId, containerName, version, samzaVersion, host, reportingInterval);
+  }
+
+  @Override
+  public void start() {
+    LOG.info("Starting producer.");
+    this.producer.start();
+    LOG.info("Starting reporter timer.");
+    this.executor.scheduleWithFixedDelay(this, 0, reportingInterval, TimeUnit.SECONDS);
+  }
+
+  @Override
+  public void register(String source, ReadableMetricsRegistry registry) {
+    this.registries.add(new RegistryWithSource(source, registry));
+    LOG.info("Registering {} with producer.", source);
+    this.producer.register(source);
+  }
+
+  @Override
+  public void stop() {
+    // Scheduling an event with 0 delay to ensure flushing of metrics one last time before shutdown
+    this.executor.schedule(this, 0, TimeUnit.SECONDS);
+    LOG.info("Stopping reporter timer.");
+    // Allow the scheduled task above to finish, and block for termination (for max 60 seconds)
+    this.executor.shutdown();
+    try {
+      this.executor.awaitTermination(60, TimeUnit.SECONDS);
+    } catch (InterruptedException e) {
+      throw new SamzaException(e);
+    }
+    LOG.info("Stopping producer.");
+    this.producer.stop();
+    if (!this.executor.isTerminated()) {
+      LOG.warn("Unable to shutdown reporter timer.");
+    }
+  }
+
+  @Override
+  public void run() {
+    try {
+      innerRun();
+    } catch (Exception e) {
+      // Ignore all exceptions - because subsequent executions of this scheduled task will be suppressed
+      // by the executor if the current task throws an unhandled exception.
+      LOG.warn("Error while reporting metrics. Will retry in " + reportingInterval + " seconds.", e);
+    }
+  }
+
+  public void innerRun() {
+    LOG.debug("Begin flushing metrics.");
+    for (RegistryWithSource registryWithSource : this.registries) {
+      String source = registryWithSource.getSource();
+      ReadableMetricsRegistry registry = registryWithSource.getRegistry();
+      LOG.debug("Flushing metrics for {}.", source);
+      Map<String, Map<String, Object>> metricsMsg = new HashMap<>();
+
+      // metrics
+      registry.getGroups().forEach(group -> {
+        Map<String, Object> groupMsg = new HashMap<>();
+        registry.getGroup(group).forEach((name, metric) -> {
+          if (!shouldIgnore(group, name)) {
+            metric.visit(new MetricsVisitor() {
+              @Override
+              public void counter(Counter counter) {
+                groupMsg.put(name, counter.getCount());
+              }
+
+              @Override
+              public <T> void gauge(Gauge<T> gauge) {
+                groupMsg.put(name, gauge.getValue());
+              }
+
+              @Override
+              public void timer(Timer timer) {
+                groupMsg.put(name, timer.getSnapshot().getAverage());
+              }
+            });
+          }
+        });
+
+        // dont emit empty groups
+        if (!groupMsg.isEmpty()) {
+          metricsMsg.put(group, groupMsg);
+        }
+      });
+
+      // publish to Kafka only if the metricsMsg carries any metrics
+      if (!metricsMsg.isEmpty()) {
+        MetricsHeader header =
+            new MetricsHeader(this.jobName, this.jobId, this.containerName, this.execEnvironmentContainerId, source,
+                this.version, this.samzaVersion, this.host, this.clock.currentTimeMillis(), this.resetTime);
+        Metrics metrics = new Metrics(metricsMsg);
+        LOG.debug("Flushing metrics for {} to {} with header and map: header={}, map={}.", source, out,
+            header.getAsMap(), metrics.getAsMap());
+        MetricsSnapshot metricsSnapshot = new MetricsSnapshot(header, metrics);
+        Object maybeSerialized = (this.serializer != null) ? this.serializer.toBytes(metricsSnapshot) : metricsSnapshot;
+        try {
+          this.producer.send(source, new OutgoingMessageEnvelope(this.out, this.host, null, maybeSerialized));
+          // Always flush, since we don't want metrics to get batched up.
+          this.producer.flush(source);
+        } catch (Exception e) {
+          LOG.error(String.format("Exception when flushing metrics for source %s", source), e);
+        }
+      }
+    }
+    LOG.debug("Finished flushing metrics.");
+  }
+
+  protected boolean shouldIgnore(String group, String metricName) {
+    boolean isBlacklisted = this.blacklist.isPresent();
+    String fullMetricName = group + "." + metricName;
+
+    if (isBlacklisted && !this.blacklistedMetrics.contains(fullMetricName)) {

Review comment:
       That is possible, but I believe it is correct behavior to filter out the metric for all registries. The blacklist applies to all of the registries, so if the metric name is the same across multiple registries, then the metric should be filtered out across all the registries.

##########
File path: samza-core/src/main/java/org/apache/samza/metrics/reporter/MetricsSnapshotReporterFactory.java
##########
@@ -0,0 +1,140 @@
+/*
+ * 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.samza.metrics.reporter;
+
+import java.util.Optional;
+import org.apache.samza.SamzaException;
+import org.apache.samza.config.Config;
+import org.apache.samza.config.JobConfig;
+import org.apache.samza.config.MetricsConfig;
+import org.apache.samza.config.SerializerConfig;
+import org.apache.samza.config.StreamConfig;
+import org.apache.samza.config.SystemConfig;
+import org.apache.samza.metrics.MetricsRegistryMap;
+import org.apache.samza.metrics.MetricsReporter;
+import org.apache.samza.metrics.MetricsReporterFactory;
+import org.apache.samza.serializers.MetricsSnapshotSerdeV2;
+import org.apache.samza.serializers.Serde;
+import org.apache.samza.serializers.SerdeFactory;
+import org.apache.samza.system.SystemFactory;
+import org.apache.samza.system.SystemProducer;
+import org.apache.samza.system.SystemStream;
+import org.apache.samza.util.ReflectionUtil;
+import org.apache.samza.util.StreamUtil;
+import org.apache.samza.util.SystemClock;
+import org.apache.samza.util.Util;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+public class MetricsSnapshotReporterFactory implements MetricsReporterFactory {
+  private static final Logger LOG = LoggerFactory.getLogger(MetricsSnapshotReporterFactory.class);
+
+  @Override
+  public MetricsReporter getMetricsReporter(String reporterName, String containerName, Config config) {
+    LOG.info("Creating new metrics snapshot reporter.");
+    MetricsRegistryMap registry = new MetricsRegistryMap();
+
+    SystemStream systemStream = getSystemStream(reporterName, config);
+    SystemProducer producer = getProducer(reporterName, config, registry);
+    int reportingInterval = getReportingInterval(reporterName, config);
+    String jobName = getJobName(config);
+    String jobId = getJobId(config);
+    Serde<MetricsSnapshot> serde = getSerde(reporterName, config);
+    Optional<String> blacklist = getBlacklist(reporterName, config);
+
+    MetricsSnapshotReporter reporter =
+        new MetricsSnapshotReporter(producer, systemStream, reportingInterval, jobName, jobId, containerName,
+            Util.getTaskClassVersion(config), Util.getSamzaVersion(), Util.getLocalHost().getHostName(), serde,
+            blacklist, SystemClock.instance());
+    reporter.register(this.getClass().getSimpleName(), registry);
+    return reporter;
+  }
+
+  protected SystemProducer getProducer(String reporterName, Config config, MetricsRegistryMap registry) {
+    SystemConfig systemConfig = new SystemConfig(config);
+    String systemName = getSystemStream(reporterName, config).getSystem();
+    String systemFactoryClassName = systemConfig.getSystemFactory(systemName)
+        .orElseThrow(() -> new SamzaException(
+            String.format("Trying to fetch system factory for system %s, which isn't defined in config.", systemName)));
+    SystemFactory systemFactory = ReflectionUtil.getObj(systemFactoryClassName, SystemFactory.class);
+    LOG.info("Got system factory {}.", systemFactory);
+    SystemProducer producer = systemFactory.getProducer(systemName, config, registry);
+    LOG.info("Got producer {}.", producer);
+    return producer;
+  }
+
+  protected SystemStream getSystemStream(String reporterName, Config config) {
+    MetricsConfig metricsConfig = new MetricsConfig(config);
+    String metricsSystemStreamName = metricsConfig.getMetricsSnapshotReporterStream(reporterName)
+        .orElseThrow(() -> new SamzaException("No metrics stream defined in config."));
+    SystemStream systemStream = StreamUtil.getSystemStreamFromNames(metricsSystemStreamName);
+    LOG.info("Got system stream {}.", systemStream);
+    return systemStream;
+  }
+
+  protected Serde<MetricsSnapshot> getSerde(String reporterName, Config config) {
+    StreamConfig streamConfig = new StreamConfig(config);
+    SystemConfig systemConfig = new SystemConfig(config);
+    SystemStream systemStream = getSystemStream(reporterName, config);
+
+    Optional<String> streamSerdeName = streamConfig.getStreamMsgSerde(systemStream);
+    Optional<String> systemSerdeName = systemConfig.getSystemMsgSerde(systemStream.getSystem());
+    String serdeName = streamSerdeName.orElse(systemSerdeName.orElse(null));
+    SerializerConfig serializerConfig = new SerializerConfig(config);
+    Serde<MetricsSnapshot> serde;
+    if (serdeName != null) {
+      Optional<String> serdeFactoryClass = serializerConfig.getSerdeFactoryClass(serdeName);
+      if (serdeFactoryClass.isPresent()) {
+        SerdeFactory<MetricsSnapshot> serdeFactory = ReflectionUtil.getObj(serdeFactoryClass.get(), SerdeFactory.class);
+        serde = serdeFactory.getSerde(serdeName, config);
+      } else {
+        serde = null;

Review comment:
       I was sticking with existing legacy behavior here. It does seem like it could be nice to use `new MetricsSnapshotSerdeV2()`, but maybe there is a legacy case where no serde is desired, and this is a way to achieve no serde.

##########
File path: samza-core/src/main/java/org/apache/samza/metrics/reporter/Metrics.java
##########
@@ -0,0 +1,79 @@
+/*
+ * 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.samza.metrics.reporter;
+
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.Objects;
+
+
+public class Metrics {
+  private final Map<String, Map<String, Object>> immutableMetrics = new HashMap<>();
+
+  public Metrics() {
+    this(Collections.emptyMap());
+  }
+
+  public Metrics(Map<String, Map<String, Object>> metrics) {
+    metrics.forEach((groupKey, groupValue) -> {
+      Map<String, Object> immutableMetricGroup = new HashMap<>();
+      groupValue.forEach(immutableMetricGroup::put);

Review comment:
       Ah good call, there is a more concise way to do this. Updated.

##########
File path: samza-core/src/main/java/org/apache/samza/metrics/reporter/Metrics.java
##########
@@ -0,0 +1,79 @@
+/*
+ * 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.samza.metrics.reporter;
+
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.Objects;
+
+
+public class Metrics {

Review comment:
       It does seem like a better name, but since this PR is just for converting from Scala to Java, I would prefer to not make a change like that here. It will also introduce some more backwards incompatibility with other code that uses this class.

##########
File path: samza-api/src/main/java/org/apache/samza/metrics/RegistryWithSource.java
##########
@@ -16,28 +16,26 @@
  * specific language governing permissions and limitations
  * under the License.
  */
+package org.apache.samza.metrics;
 
-package org.apache.samza.metrics.reporter
-
-import java.util.Map
-import java.util.HashMap
-import scala.beans.BeanProperty
+/**
+ * Simple holder for {@link ReadableMetricsRegistry} and its source, used for implementations of
+ * {@link MetricsReporter}.
+ */
+public class RegistryWithSource {

Review comment:
       I prefer to not use `Pair` because it isn't as explicit about what is inside of it. It is usually not too hard to figure out what `getLeft` and `getRight` returns, but it is sometimes not obvious, especially when one of the items is a primitive type like a `String`. In this case, if `Pair` was used, then a reader would need to trace back through to where the `Pair` was added to the list in order to figure out that the `String` part of the `Pair` is "source". Although it is an extra class, I prefer the explicit readability.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@samza.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [samza] cameronlee314 commented on a change in pull request #1558: SAMZA-2707: Convert metrics snapshot reporter code to java

Posted by GitBox <gi...@apache.org>.
cameronlee314 commented on a change in pull request #1558:
URL: https://github.com/apache/samza/pull/1558#discussion_r754704863



##########
File path: samza-core/src/main/java/org/apache/samza/metrics/reporter/MetricsSnapshotReporter.java
##########
@@ -0,0 +1,228 @@
+/*
+ * 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.samza.metrics.reporter;
+
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Set;
+import java.util.concurrent.Executors;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.TimeUnit;
+import com.google.common.util.concurrent.ThreadFactoryBuilder;
+import org.apache.samza.SamzaException;
+import org.apache.samza.config.ShellCommandConfig;
+import org.apache.samza.metrics.Counter;
+import org.apache.samza.metrics.Gauge;
+import org.apache.samza.metrics.MetricsReporter;
+import org.apache.samza.metrics.MetricsVisitor;
+import org.apache.samza.metrics.ReadableMetricsRegistry;
+import org.apache.samza.metrics.MetricsRegistryWithSource;
+import org.apache.samza.metrics.Timer;
+import org.apache.samza.serializers.Serializer;
+import org.apache.samza.system.OutgoingMessageEnvelope;
+import org.apache.samza.system.SystemProducer;
+import org.apache.samza.system.SystemStream;
+import org.apache.samza.util.Clock;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+/**
+ * MetricsSnapshotReporter is a generic metrics reporter that sends metrics to a stream.
+ *
+ * jobName // my-samza-job
+ * jobId // an id that differentiates multiple executions of the same job
+ * taskName // container_567890
+ * host // eat1-app128.gird
+ * version // 0.0.1
+ * blacklist // Regex of metrics to ignore when flushing
+ */
+public class MetricsSnapshotReporter implements MetricsReporter, Runnable {
+  private static final Logger LOG = LoggerFactory.getLogger(MetricsSnapshotReporter.class);
+
+  private final SystemProducer producer;
+  private final SystemStream out;
+  private final int reportingInterval;
+  private final String jobName;
+  private final String jobId;
+  private final String containerName;
+  private final String version;
+  private final String samzaVersion;
+  private final String host;
+  private final Serializer<MetricsSnapshot> serializer;
+  private final Optional<String> blacklist;

Review comment:
       I like the idea of `Pattern`. I would like to stick with `Optional` though for the "no filter" case since it is a more explicit indication.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@samza.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [samza] ZitingShen commented on a change in pull request #1558: SAMZA-2707: Convert metrics snapshot reporter code to java

Posted by GitBox <gi...@apache.org>.
ZitingShen commented on a change in pull request #1558:
URL: https://github.com/apache/samza/pull/1558#discussion_r750596387



##########
File path: samza-core/src/main/java/org/apache/samza/metrics/reporter/MetricsHeader.java
##########
@@ -0,0 +1,160 @@
+/*
+ * 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.samza.metrics.reporter;
+
+import java.util.HashMap;
+import java.util.Map;
+import java.util.Objects;
+
+
+public class MetricsHeader {
+  private static final String JOB_NAME = "job-name";
+  private static final String JOB_ID = "job-id";
+  private static final String CONTAINER_NAME = "container-name";
+  private static final String EXEC_ENV_CONTAINER_ID = "exec-env-container-id";
+  private static final String SOURCE = "source";
+  private static final String VERSION = "version";
+  private static final String SAMZA_VERSION = "samza-version";
+  private static final String HOST = "host";
+  private static final String TIME = "time";
+  private static final String RESET_TIME = "reset-time";
+
+  private final String jobName;
+  private final String jobId;
+  private final String containerName;
+  private final String execEnvironmentContainerId;
+  private final String source;
+  private final String version;
+  private final String samzaVersion;
+  private final String host;
+  private final long time;
+  private final long resetTime;
+
+  public MetricsHeader(String jobName, String jobId, String containerName, String execEnvironmentContainerId,
+      String source, String version, String samzaVersion, String host, long time, long resetTime) {
+    this.jobName = jobName;
+    this.jobId = jobId;
+    this.containerName = containerName;
+    this.execEnvironmentContainerId = execEnvironmentContainerId;
+    this.source = source;
+    this.version = version;
+    this.samzaVersion = samzaVersion;
+    this.host = host;
+    this.time = time;
+    this.resetTime = resetTime;
+  }
+
+  public Map<String, Object> getAsMap() {
+    Map<String, Object> map = new HashMap<>();
+    map.put(JOB_NAME, jobName);
+    map.put(JOB_ID, jobId);
+    map.put(CONTAINER_NAME, containerName);
+    map.put(EXEC_ENV_CONTAINER_ID, execEnvironmentContainerId);
+    map.put(SOURCE, source);
+    map.put(VERSION, version);
+    map.put(SAMZA_VERSION, samzaVersion);
+    map.put(HOST, host);
+    map.put(TIME, time);
+    map.put(RESET_TIME, resetTime);
+    return map;
+  }
+
+  public String getJobName() {
+    return jobName;
+  }
+
+  public String getJobId() {
+    return jobId;
+  }
+
+  public String getContainerName() {
+    return containerName;
+  }
+
+  public String getExecEnvironmentContainerId() {
+    return execEnvironmentContainerId;
+  }
+
+  public String getSource() {
+    return source;
+  }
+
+  public String getVersion() {
+    return version;
+  }
+
+  public String getSamzaVersion() {
+    return samzaVersion;
+  }
+
+  public String getHost() {
+    return host;
+  }
+
+  public long getTime() {
+    return time;
+  }
+
+  public long getResetTime() {
+    return resetTime;
+  }
+
+  public static MetricsHeader fromMap(Map<String, Object> map) {
+    return new MetricsHeader(map.get(JOB_NAME).toString(),
+        map.get(JOB_ID).toString(),
+        map.get(CONTAINER_NAME).toString(),
+        map.get(EXEC_ENV_CONTAINER_ID).toString(),
+        map.get(SOURCE).toString(),
+        map.get(VERSION).toString(),
+        map.get(SAMZA_VERSION).toString(),
+        map.get(HOST).toString(),
+        ((Number) map.get(TIME)).longValue(),
+        ((Number) map.get(RESET_TIME)).longValue());

Review comment:
       ohhh never mind I misread the parentheses




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@samza.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [samza] ryucc commented on a change in pull request #1558: SAMZA-2707: Convert metrics snapshot reporter code to java

Posted by GitBox <gi...@apache.org>.
ryucc commented on a change in pull request #1558:
URL: https://github.com/apache/samza/pull/1558#discussion_r748824201



##########
File path: samza-core/src/main/java/org/apache/samza/metrics/reporter/Metrics.java
##########
@@ -0,0 +1,79 @@
+/*
+ * 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.samza.metrics.reporter;
+
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.Objects;
+
+
+public class Metrics {
+  private final Map<String, Map<String, Object>> immutableMetrics = new HashMap<>();
+
+  public Metrics() {
+    this(Collections.emptyMap());
+  }
+
+  public Metrics(Map<String, Map<String, Object>> metrics) {
+    metrics.forEach((groupKey, groupValue) -> {
+      Map<String, Object> immutableMetricGroup = new HashMap<>();
+      groupValue.forEach(immutableMetricGroup::put);

Review comment:
       Is this line `immutableMetricGroup.putAll(groupValue)`? Or just `Map<String, Object> immutableMetricGroup = groupValue.clone()`?
   
   Don't think this is a deep clone either way.

##########
File path: samza-core/src/main/java/org/apache/samza/metrics/reporter/MetricsSnapshotReporter.java
##########
@@ -0,0 +1,228 @@
+/*
+ * 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.samza.metrics.reporter;
+
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Set;
+import java.util.concurrent.Executors;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.TimeUnit;
+import com.google.common.util.concurrent.ThreadFactoryBuilder;
+import org.apache.samza.SamzaException;
+import org.apache.samza.config.ShellCommandConfig;
+import org.apache.samza.metrics.Counter;
+import org.apache.samza.metrics.Gauge;
+import org.apache.samza.metrics.MetricsReporter;
+import org.apache.samza.metrics.MetricsVisitor;
+import org.apache.samza.metrics.ReadableMetricsRegistry;
+import org.apache.samza.metrics.RegistryWithSource;
+import org.apache.samza.metrics.Timer;
+import org.apache.samza.serializers.Serializer;
+import org.apache.samza.system.OutgoingMessageEnvelope;
+import org.apache.samza.system.SystemProducer;
+import org.apache.samza.system.SystemStream;
+import org.apache.samza.util.Clock;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+/**
+ * MetricsSnapshotReporter is a generic metrics reporter that sends metrics to a stream.
+ *
+ * jobName // my-samza-job
+ * jobId // an id that differentiates multiple executions of the same job
+ * taskName // container_567890
+ * host // eat1-app128.gird
+ * version // 0.0.1
+ * blacklist // Regex of metrics to ignore when flushing
+ */
+public class MetricsSnapshotReporter implements MetricsReporter, Runnable {
+  private static final Logger LOG = LoggerFactory.getLogger(MetricsSnapshotReporter.class);
+
+  private final SystemProducer producer;
+  private final SystemStream out;
+  private final int reportingInterval;
+  private final String jobName;
+  private final String jobId;
+  private final String containerName;
+  private final String version;
+  private final String samzaVersion;
+  private final String host;
+  private final Serializer<MetricsSnapshot> serializer;
+  private final Optional<String> blacklist;
+  private final Clock clock;
+
+  private final String execEnvironmentContainerId;
+  private final ScheduledExecutorService executor;
+  private final long resetTime;
+  private final List<RegistryWithSource> registries = new ArrayList<>();
+  private final Set<String> blacklistedMetrics = new HashSet<>();
+
+  public MetricsSnapshotReporter(SystemProducer producer, SystemStream out, int reportingInterval, String jobName,
+      String jobId, String containerName, String version, String samzaVersion, String host,
+      Serializer<MetricsSnapshot> serializer, Optional<String> blacklist, Clock clock) {
+    this.producer = producer;
+    this.out = out;
+    this.reportingInterval = reportingInterval;
+    this.jobName = jobName;
+    this.jobId = jobId;
+    this.containerName = containerName;
+    this.version = version;
+    this.samzaVersion = samzaVersion;
+    this.host = host;
+    this.serializer = serializer;
+    this.blacklist = blacklist;
+    this.clock = clock;
+
+    this.execEnvironmentContainerId =
+        Optional.ofNullable(System.getenv(ShellCommandConfig.ENV_EXECUTION_ENV_CONTAINER_ID)).orElse("");
+    this.executor = Executors.newSingleThreadScheduledExecutor(
+        new ThreadFactoryBuilder().setNameFormat("Samza MetricsSnapshotReporter Thread-%d").setDaemon(true).build());
+    this.resetTime = this.clock.currentTimeMillis();
+    LOG.info(
+        "got metrics snapshot reporter properties [job name: {}, job id: {}, containerName: {}, version: {}, samzaVersion: {}, host: {}, reportingInterval {}]",
+        jobName, jobId, containerName, version, samzaVersion, host, reportingInterval);
+  }
+
+  @Override
+  public void start() {
+    LOG.info("Starting producer.");
+    this.producer.start();
+    LOG.info("Starting reporter timer.");
+    this.executor.scheduleWithFixedDelay(this, 0, reportingInterval, TimeUnit.SECONDS);
+  }
+
+  @Override
+  public void register(String source, ReadableMetricsRegistry registry) {
+    this.registries.add(new RegistryWithSource(source, registry));
+    LOG.info("Registering {} with producer.", source);
+    this.producer.register(source);
+  }
+
+  @Override
+  public void stop() {
+    // Scheduling an event with 0 delay to ensure flushing of metrics one last time before shutdown
+    this.executor.schedule(this, 0, TimeUnit.SECONDS);
+    LOG.info("Stopping reporter timer.");
+    // Allow the scheduled task above to finish, and block for termination (for max 60 seconds)
+    this.executor.shutdown();
+    try {
+      this.executor.awaitTermination(60, TimeUnit.SECONDS);
+    } catch (InterruptedException e) {
+      throw new SamzaException(e);
+    }
+    LOG.info("Stopping producer.");
+    this.producer.stop();
+    if (!this.executor.isTerminated()) {
+      LOG.warn("Unable to shutdown reporter timer.");
+    }
+  }
+
+  @Override
+  public void run() {
+    try {
+      innerRun();
+    } catch (Exception e) {
+      // Ignore all exceptions - because subsequent executions of this scheduled task will be suppressed
+      // by the executor if the current task throws an unhandled exception.
+      LOG.warn("Error while reporting metrics. Will retry in " + reportingInterval + " seconds.", e);
+    }
+  }
+
+  public void innerRun() {
+    LOG.debug("Begin flushing metrics.");
+    for (RegistryWithSource registryWithSource : this.registries) {
+      String source = registryWithSource.getSource();
+      ReadableMetricsRegistry registry = registryWithSource.getRegistry();
+      LOG.debug("Flushing metrics for {}.", source);
+      Map<String, Map<String, Object>> metricsMsg = new HashMap<>();
+
+      // metrics
+      registry.getGroups().forEach(group -> {
+        Map<String, Object> groupMsg = new HashMap<>();
+        registry.getGroup(group).forEach((name, metric) -> {
+          if (!shouldIgnore(group, name)) {
+            metric.visit(new MetricsVisitor() {
+              @Override
+              public void counter(Counter counter) {
+                groupMsg.put(name, counter.getCount());
+              }
+
+              @Override
+              public <T> void gauge(Gauge<T> gauge) {
+                groupMsg.put(name, gauge.getValue());
+              }
+
+              @Override
+              public void timer(Timer timer) {
+                groupMsg.put(name, timer.getSnapshot().getAverage());
+              }
+            });
+          }
+        });
+
+        // dont emit empty groups
+        if (!groupMsg.isEmpty()) {
+          metricsMsg.put(group, groupMsg);
+        }
+      });
+
+      // publish to Kafka only if the metricsMsg carries any metrics
+      if (!metricsMsg.isEmpty()) {
+        MetricsHeader header =
+            new MetricsHeader(this.jobName, this.jobId, this.containerName, this.execEnvironmentContainerId, source,
+                this.version, this.samzaVersion, this.host, this.clock.currentTimeMillis(), this.resetTime);
+        Metrics metrics = new Metrics(metricsMsg);
+        LOG.debug("Flushing metrics for {} to {} with header and map: header={}, map={}.", source, out,
+            header.getAsMap(), metrics.getAsMap());
+        MetricsSnapshot metricsSnapshot = new MetricsSnapshot(header, metrics);
+        Object maybeSerialized = (this.serializer != null) ? this.serializer.toBytes(metricsSnapshot) : metricsSnapshot;
+        try {
+          this.producer.send(source, new OutgoingMessageEnvelope(this.out, this.host, null, maybeSerialized));
+          // Always flush, since we don't want metrics to get batched up.
+          this.producer.flush(source);
+        } catch (Exception e) {
+          LOG.error(String.format("Exception when flushing metrics for source %s", source), e);
+        }
+      }
+    }
+    LOG.debug("Finished flushing metrics.");
+  }
+
+  protected boolean shouldIgnore(String group, String metricName) {
+    boolean isBlacklisted = this.blacklist.isPresent();

Review comment:
       Can return early if false? This might reduce 1 layer of the nested if.

##########
File path: samza-core/src/main/java/org/apache/samza/metrics/reporter/MetricsSnapshotReporter.java
##########
@@ -0,0 +1,228 @@
+/*
+ * 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.samza.metrics.reporter;
+
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Set;
+import java.util.concurrent.Executors;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.TimeUnit;
+import com.google.common.util.concurrent.ThreadFactoryBuilder;
+import org.apache.samza.SamzaException;
+import org.apache.samza.config.ShellCommandConfig;
+import org.apache.samza.metrics.Counter;
+import org.apache.samza.metrics.Gauge;
+import org.apache.samza.metrics.MetricsReporter;
+import org.apache.samza.metrics.MetricsVisitor;
+import org.apache.samza.metrics.ReadableMetricsRegistry;
+import org.apache.samza.metrics.RegistryWithSource;
+import org.apache.samza.metrics.Timer;
+import org.apache.samza.serializers.Serializer;
+import org.apache.samza.system.OutgoingMessageEnvelope;
+import org.apache.samza.system.SystemProducer;
+import org.apache.samza.system.SystemStream;
+import org.apache.samza.util.Clock;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+/**
+ * MetricsSnapshotReporter is a generic metrics reporter that sends metrics to a stream.
+ *
+ * jobName // my-samza-job
+ * jobId // an id that differentiates multiple executions of the same job
+ * taskName // container_567890
+ * host // eat1-app128.gird
+ * version // 0.0.1
+ * blacklist // Regex of metrics to ignore when flushing
+ */
+public class MetricsSnapshotReporter implements MetricsReporter, Runnable {
+  private static final Logger LOG = LoggerFactory.getLogger(MetricsSnapshotReporter.class);
+
+  private final SystemProducer producer;
+  private final SystemStream out;
+  private final int reportingInterval;
+  private final String jobName;
+  private final String jobId;
+  private final String containerName;
+  private final String version;
+  private final String samzaVersion;
+  private final String host;
+  private final Serializer<MetricsSnapshot> serializer;
+  private final Optional<String> blacklist;
+  private final Clock clock;
+
+  private final String execEnvironmentContainerId;
+  private final ScheduledExecutorService executor;
+  private final long resetTime;
+  private final List<RegistryWithSource> registries = new ArrayList<>();
+  private final Set<String> blacklistedMetrics = new HashSet<>();
+
+  public MetricsSnapshotReporter(SystemProducer producer, SystemStream out, int reportingInterval, String jobName,
+      String jobId, String containerName, String version, String samzaVersion, String host,
+      Serializer<MetricsSnapshot> serializer, Optional<String> blacklist, Clock clock) {
+    this.producer = producer;
+    this.out = out;
+    this.reportingInterval = reportingInterval;
+    this.jobName = jobName;
+    this.jobId = jobId;
+    this.containerName = containerName;
+    this.version = version;
+    this.samzaVersion = samzaVersion;
+    this.host = host;
+    this.serializer = serializer;
+    this.blacklist = blacklist;
+    this.clock = clock;
+
+    this.execEnvironmentContainerId =
+        Optional.ofNullable(System.getenv(ShellCommandConfig.ENV_EXECUTION_ENV_CONTAINER_ID)).orElse("");
+    this.executor = Executors.newSingleThreadScheduledExecutor(
+        new ThreadFactoryBuilder().setNameFormat("Samza MetricsSnapshotReporter Thread-%d").setDaemon(true).build());
+    this.resetTime = this.clock.currentTimeMillis();
+    LOG.info(
+        "got metrics snapshot reporter properties [job name: {}, job id: {}, containerName: {}, version: {}, samzaVersion: {}, host: {}, reportingInterval {}]",
+        jobName, jobId, containerName, version, samzaVersion, host, reportingInterval);
+  }
+
+  @Override
+  public void start() {
+    LOG.info("Starting producer.");
+    this.producer.start();
+    LOG.info("Starting reporter timer.");
+    this.executor.scheduleWithFixedDelay(this, 0, reportingInterval, TimeUnit.SECONDS);
+  }
+
+  @Override
+  public void register(String source, ReadableMetricsRegistry registry) {
+    this.registries.add(new RegistryWithSource(source, registry));
+    LOG.info("Registering {} with producer.", source);
+    this.producer.register(source);
+  }
+
+  @Override
+  public void stop() {
+    // Scheduling an event with 0 delay to ensure flushing of metrics one last time before shutdown
+    this.executor.schedule(this, 0, TimeUnit.SECONDS);
+    LOG.info("Stopping reporter timer.");
+    // Allow the scheduled task above to finish, and block for termination (for max 60 seconds)
+    this.executor.shutdown();
+    try {
+      this.executor.awaitTermination(60, TimeUnit.SECONDS);
+    } catch (InterruptedException e) {
+      throw new SamzaException(e);
+    }
+    LOG.info("Stopping producer.");
+    this.producer.stop();
+    if (!this.executor.isTerminated()) {
+      LOG.warn("Unable to shutdown reporter timer.");
+    }
+  }
+
+  @Override
+  public void run() {
+    try {
+      innerRun();
+    } catch (Exception e) {
+      // Ignore all exceptions - because subsequent executions of this scheduled task will be suppressed
+      // by the executor if the current task throws an unhandled exception.
+      LOG.warn("Error while reporting metrics. Will retry in " + reportingInterval + " seconds.", e);
+    }
+  }
+
+  public void innerRun() {
+    LOG.debug("Begin flushing metrics.");
+    for (RegistryWithSource registryWithSource : this.registries) {
+      String source = registryWithSource.getSource();
+      ReadableMetricsRegistry registry = registryWithSource.getRegistry();
+      LOG.debug("Flushing metrics for {}.", source);
+      Map<String, Map<String, Object>> metricsMsg = new HashMap<>();
+
+      // metrics
+      registry.getGroups().forEach(group -> {
+        Map<String, Object> groupMsg = new HashMap<>();
+        registry.getGroup(group).forEach((name, metric) -> {
+          if (!shouldIgnore(group, name)) {
+            metric.visit(new MetricsVisitor() {
+              @Override
+              public void counter(Counter counter) {
+                groupMsg.put(name, counter.getCount());
+              }
+
+              @Override
+              public <T> void gauge(Gauge<T> gauge) {
+                groupMsg.put(name, gauge.getValue());
+              }
+
+              @Override
+              public void timer(Timer timer) {
+                groupMsg.put(name, timer.getSnapshot().getAverage());
+              }
+            });
+          }
+        });
+
+        // dont emit empty groups
+        if (!groupMsg.isEmpty()) {
+          metricsMsg.put(group, groupMsg);
+        }
+      });
+
+      // publish to Kafka only if the metricsMsg carries any metrics
+      if (!metricsMsg.isEmpty()) {
+        MetricsHeader header =
+            new MetricsHeader(this.jobName, this.jobId, this.containerName, this.execEnvironmentContainerId, source,
+                this.version, this.samzaVersion, this.host, this.clock.currentTimeMillis(), this.resetTime);
+        Metrics metrics = new Metrics(metricsMsg);
+        LOG.debug("Flushing metrics for {} to {} with header and map: header={}, map={}.", source, out,
+            header.getAsMap(), metrics.getAsMap());
+        MetricsSnapshot metricsSnapshot = new MetricsSnapshot(header, metrics);
+        Object maybeSerialized = (this.serializer != null) ? this.serializer.toBytes(metricsSnapshot) : metricsSnapshot;
+        try {
+          this.producer.send(source, new OutgoingMessageEnvelope(this.out, this.host, null, maybeSerialized));
+          // Always flush, since we don't want metrics to get batched up.
+          this.producer.flush(source);
+        } catch (Exception e) {
+          LOG.error(String.format("Exception when flushing metrics for source %s", source), e);
+        }
+      }
+    }
+    LOG.debug("Finished flushing metrics.");
+  }
+
+  protected boolean shouldIgnore(String group, String metricName) {
+    boolean isBlacklisted = this.blacklist.isPresent();
+    String fullMetricName = group + "." + metricName;
+
+    if (isBlacklisted && !this.blacklistedMetrics.contains(fullMetricName)) {

Review comment:
       Possible that the same `fullMetricName` exists in different registries? In this case would the second occurrence be filtered out?

##########
File path: samza-core/src/main/java/org/apache/samza/metrics/reporter/Metrics.java
##########
@@ -0,0 +1,79 @@
+/*
+ * 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.samza.metrics.reporter;
+
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.Objects;
+
+
+public class Metrics {

Review comment:
       Is `MetricValues` a better naming? Also contrast with `MetricsHeader`

##########
File path: samza-core/src/main/java/org/apache/samza/metrics/reporter/MetricsSnapshotReporterFactory.java
##########
@@ -0,0 +1,140 @@
+/*
+ * 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.samza.metrics.reporter;
+
+import java.util.Optional;
+import org.apache.samza.SamzaException;
+import org.apache.samza.config.Config;
+import org.apache.samza.config.JobConfig;
+import org.apache.samza.config.MetricsConfig;
+import org.apache.samza.config.SerializerConfig;
+import org.apache.samza.config.StreamConfig;
+import org.apache.samza.config.SystemConfig;
+import org.apache.samza.metrics.MetricsRegistryMap;
+import org.apache.samza.metrics.MetricsReporter;
+import org.apache.samza.metrics.MetricsReporterFactory;
+import org.apache.samza.serializers.MetricsSnapshotSerdeV2;
+import org.apache.samza.serializers.Serde;
+import org.apache.samza.serializers.SerdeFactory;
+import org.apache.samza.system.SystemFactory;
+import org.apache.samza.system.SystemProducer;
+import org.apache.samza.system.SystemStream;
+import org.apache.samza.util.ReflectionUtil;
+import org.apache.samza.util.StreamUtil;
+import org.apache.samza.util.SystemClock;
+import org.apache.samza.util.Util;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+public class MetricsSnapshotReporterFactory implements MetricsReporterFactory {
+  private static final Logger LOG = LoggerFactory.getLogger(MetricsSnapshotReporterFactory.class);
+
+  @Override
+  public MetricsReporter getMetricsReporter(String reporterName, String containerName, Config config) {
+    LOG.info("Creating new metrics snapshot reporter.");
+    MetricsRegistryMap registry = new MetricsRegistryMap();
+
+    SystemStream systemStream = getSystemStream(reporterName, config);
+    SystemProducer producer = getProducer(reporterName, config, registry);
+    int reportingInterval = getReportingInterval(reporterName, config);
+    String jobName = getJobName(config);
+    String jobId = getJobId(config);
+    Serde<MetricsSnapshot> serde = getSerde(reporterName, config);
+    Optional<String> blacklist = getBlacklist(reporterName, config);
+
+    MetricsSnapshotReporter reporter =
+        new MetricsSnapshotReporter(producer, systemStream, reportingInterval, jobName, jobId, containerName,
+            Util.getTaskClassVersion(config), Util.getSamzaVersion(), Util.getLocalHost().getHostName(), serde,
+            blacklist, SystemClock.instance());
+    reporter.register(this.getClass().getSimpleName(), registry);
+    return reporter;
+  }
+
+  protected SystemProducer getProducer(String reporterName, Config config, MetricsRegistryMap registry) {
+    SystemConfig systemConfig = new SystemConfig(config);
+    String systemName = getSystemStream(reporterName, config).getSystem();
+    String systemFactoryClassName = systemConfig.getSystemFactory(systemName)
+        .orElseThrow(() -> new SamzaException(
+            String.format("Trying to fetch system factory for system %s, which isn't defined in config.", systemName)));
+    SystemFactory systemFactory = ReflectionUtil.getObj(systemFactoryClassName, SystemFactory.class);
+    LOG.info("Got system factory {}.", systemFactory);
+    SystemProducer producer = systemFactory.getProducer(systemName, config, registry);
+    LOG.info("Got producer {}.", producer);
+    return producer;
+  }
+
+  protected SystemStream getSystemStream(String reporterName, Config config) {
+    MetricsConfig metricsConfig = new MetricsConfig(config);
+    String metricsSystemStreamName = metricsConfig.getMetricsSnapshotReporterStream(reporterName)
+        .orElseThrow(() -> new SamzaException("No metrics stream defined in config."));
+    SystemStream systemStream = StreamUtil.getSystemStreamFromNames(metricsSystemStreamName);
+    LOG.info("Got system stream {}.", systemStream);
+    return systemStream;
+  }
+
+  protected Serde<MetricsSnapshot> getSerde(String reporterName, Config config) {
+    StreamConfig streamConfig = new StreamConfig(config);
+    SystemConfig systemConfig = new SystemConfig(config);
+    SystemStream systemStream = getSystemStream(reporterName, config);
+
+    Optional<String> streamSerdeName = streamConfig.getStreamMsgSerde(systemStream);
+    Optional<String> systemSerdeName = systemConfig.getSystemMsgSerde(systemStream.getSystem());
+    String serdeName = streamSerdeName.orElse(systemSerdeName.orElse(null));
+    SerializerConfig serializerConfig = new SerializerConfig(config);
+    Serde<MetricsSnapshot> serde;
+    if (serdeName != null) {
+      Optional<String> serdeFactoryClass = serializerConfig.getSerdeFactoryClass(serdeName);
+      if (serdeFactoryClass.isPresent()) {
+        SerdeFactory<MetricsSnapshot> serdeFactory = ReflectionUtil.getObj(serdeFactoryClass.get(), SerdeFactory.class);
+        serde = serdeFactory.getSerde(serdeName, config);
+      } else {
+        serde = null;

Review comment:
       Why not `serde =  new MetricsSnapshotSerdeV2();`?

##########
File path: samza-api/src/main/java/org/apache/samza/metrics/RegistryWithSource.java
##########
@@ -16,28 +16,26 @@
  * specific language governing permissions and limitations
  * under the License.
  */
+package org.apache.samza.metrics;
 
-package org.apache.samza.metrics.reporter
-
-import java.util.Map
-import java.util.HashMap
-import scala.beans.BeanProperty
+/**
+ * Simple holder for {@link ReadableMetricsRegistry} and its source, used for implementations of
+ * {@link MetricsReporter}.
+ */
+public class RegistryWithSource {

Review comment:
       Why not use ``org.apache.commons.lang3.tuple.Pair`` or use `Map<String, ReadableMetricsRegistry>` for ``registries``? I think the readability is similar, but we introduce 1 less class.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@samza.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [samza] ryucc commented on a change in pull request #1558: SAMZA-2707: Convert metrics snapshot reporter code to java

Posted by GitBox <gi...@apache.org>.
ryucc commented on a change in pull request #1558:
URL: https://github.com/apache/samza/pull/1558#discussion_r749620235



##########
File path: samza-core/src/main/java/org/apache/samza/metrics/reporter/MetricsSnapshotReporter.java
##########
@@ -0,0 +1,228 @@
+/*
+ * 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.samza.metrics.reporter;
+
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Set;
+import java.util.concurrent.Executors;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.TimeUnit;
+import com.google.common.util.concurrent.ThreadFactoryBuilder;
+import org.apache.samza.SamzaException;
+import org.apache.samza.config.ShellCommandConfig;
+import org.apache.samza.metrics.Counter;
+import org.apache.samza.metrics.Gauge;
+import org.apache.samza.metrics.MetricsReporter;
+import org.apache.samza.metrics.MetricsVisitor;
+import org.apache.samza.metrics.ReadableMetricsRegistry;
+import org.apache.samza.metrics.RegistryWithSource;
+import org.apache.samza.metrics.Timer;
+import org.apache.samza.serializers.Serializer;
+import org.apache.samza.system.OutgoingMessageEnvelope;
+import org.apache.samza.system.SystemProducer;
+import org.apache.samza.system.SystemStream;
+import org.apache.samza.util.Clock;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+/**
+ * MetricsSnapshotReporter is a generic metrics reporter that sends metrics to a stream.
+ *
+ * jobName // my-samza-job
+ * jobId // an id that differentiates multiple executions of the same job
+ * taskName // container_567890
+ * host // eat1-app128.gird
+ * version // 0.0.1
+ * blacklist // Regex of metrics to ignore when flushing
+ */
+public class MetricsSnapshotReporter implements MetricsReporter, Runnable {
+  private static final Logger LOG = LoggerFactory.getLogger(MetricsSnapshotReporter.class);
+
+  private final SystemProducer producer;
+  private final SystemStream out;
+  private final int reportingInterval;
+  private final String jobName;
+  private final String jobId;
+  private final String containerName;
+  private final String version;
+  private final String samzaVersion;
+  private final String host;
+  private final Serializer<MetricsSnapshot> serializer;
+  private final Optional<String> blacklist;
+  private final Clock clock;
+
+  private final String execEnvironmentContainerId;
+  private final ScheduledExecutorService executor;
+  private final long resetTime;
+  private final List<RegistryWithSource> registries = new ArrayList<>();
+  private final Set<String> blacklistedMetrics = new HashSet<>();
+
+  public MetricsSnapshotReporter(SystemProducer producer, SystemStream out, int reportingInterval, String jobName,
+      String jobId, String containerName, String version, String samzaVersion, String host,
+      Serializer<MetricsSnapshot> serializer, Optional<String> blacklist, Clock clock) {
+    this.producer = producer;
+    this.out = out;
+    this.reportingInterval = reportingInterval;
+    this.jobName = jobName;
+    this.jobId = jobId;
+    this.containerName = containerName;
+    this.version = version;
+    this.samzaVersion = samzaVersion;
+    this.host = host;
+    this.serializer = serializer;
+    this.blacklist = blacklist;
+    this.clock = clock;
+
+    this.execEnvironmentContainerId =
+        Optional.ofNullable(System.getenv(ShellCommandConfig.ENV_EXECUTION_ENV_CONTAINER_ID)).orElse("");
+    this.executor = Executors.newSingleThreadScheduledExecutor(
+        new ThreadFactoryBuilder().setNameFormat("Samza MetricsSnapshotReporter Thread-%d").setDaemon(true).build());
+    this.resetTime = this.clock.currentTimeMillis();
+    LOG.info(
+        "got metrics snapshot reporter properties [job name: {}, job id: {}, containerName: {}, version: {}, samzaVersion: {}, host: {}, reportingInterval {}]",
+        jobName, jobId, containerName, version, samzaVersion, host, reportingInterval);
+  }
+
+  @Override
+  public void start() {
+    LOG.info("Starting producer.");
+    this.producer.start();
+    LOG.info("Starting reporter timer.");
+    this.executor.scheduleWithFixedDelay(this, 0, reportingInterval, TimeUnit.SECONDS);
+  }
+
+  @Override
+  public void register(String source, ReadableMetricsRegistry registry) {
+    this.registries.add(new RegistryWithSource(source, registry));
+    LOG.info("Registering {} with producer.", source);
+    this.producer.register(source);
+  }
+
+  @Override
+  public void stop() {
+    // Scheduling an event with 0 delay to ensure flushing of metrics one last time before shutdown
+    this.executor.schedule(this, 0, TimeUnit.SECONDS);
+    LOG.info("Stopping reporter timer.");
+    // Allow the scheduled task above to finish, and block for termination (for max 60 seconds)
+    this.executor.shutdown();
+    try {
+      this.executor.awaitTermination(60, TimeUnit.SECONDS);
+    } catch (InterruptedException e) {
+      throw new SamzaException(e);
+    }
+    LOG.info("Stopping producer.");
+    this.producer.stop();
+    if (!this.executor.isTerminated()) {
+      LOG.warn("Unable to shutdown reporter timer.");
+    }
+  }
+
+  @Override
+  public void run() {
+    try {
+      innerRun();
+    } catch (Exception e) {
+      // Ignore all exceptions - because subsequent executions of this scheduled task will be suppressed
+      // by the executor if the current task throws an unhandled exception.
+      LOG.warn("Error while reporting metrics. Will retry in " + reportingInterval + " seconds.", e);
+    }
+  }
+
+  public void innerRun() {
+    LOG.debug("Begin flushing metrics.");
+    for (RegistryWithSource registryWithSource : this.registries) {
+      String source = registryWithSource.getSource();
+      ReadableMetricsRegistry registry = registryWithSource.getRegistry();
+      LOG.debug("Flushing metrics for {}.", source);
+      Map<String, Map<String, Object>> metricsMsg = new HashMap<>();
+
+      // metrics
+      registry.getGroups().forEach(group -> {
+        Map<String, Object> groupMsg = new HashMap<>();
+        registry.getGroup(group).forEach((name, metric) -> {
+          if (!shouldIgnore(group, name)) {
+            metric.visit(new MetricsVisitor() {
+              @Override
+              public void counter(Counter counter) {
+                groupMsg.put(name, counter.getCount());
+              }
+
+              @Override
+              public <T> void gauge(Gauge<T> gauge) {
+                groupMsg.put(name, gauge.getValue());
+              }
+
+              @Override
+              public void timer(Timer timer) {
+                groupMsg.put(name, timer.getSnapshot().getAverage());
+              }
+            });
+          }
+        });
+
+        // dont emit empty groups
+        if (!groupMsg.isEmpty()) {
+          metricsMsg.put(group, groupMsg);
+        }
+      });
+
+      // publish to Kafka only if the metricsMsg carries any metrics
+      if (!metricsMsg.isEmpty()) {
+        MetricsHeader header =
+            new MetricsHeader(this.jobName, this.jobId, this.containerName, this.execEnvironmentContainerId, source,
+                this.version, this.samzaVersion, this.host, this.clock.currentTimeMillis(), this.resetTime);
+        Metrics metrics = new Metrics(metricsMsg);
+        LOG.debug("Flushing metrics for {} to {} with header and map: header={}, map={}.", source, out,
+            header.getAsMap(), metrics.getAsMap());
+        MetricsSnapshot metricsSnapshot = new MetricsSnapshot(header, metrics);
+        Object maybeSerialized = (this.serializer != null) ? this.serializer.toBytes(metricsSnapshot) : metricsSnapshot;
+        try {
+          this.producer.send(source, new OutgoingMessageEnvelope(this.out, this.host, null, maybeSerialized));
+          // Always flush, since we don't want metrics to get batched up.
+          this.producer.flush(source);
+        } catch (Exception e) {
+          LOG.error(String.format("Exception when flushing metrics for source %s", source), e);
+        }
+      }
+    }
+    LOG.debug("Finished flushing metrics.");
+  }
+
+  protected boolean shouldIgnore(String group, String metricName) {
+    boolean isBlacklisted = this.blacklist.isPresent();
+    String fullMetricName = group + "." + metricName;
+
+    if (isBlacklisted && !this.blacklistedMetrics.contains(fullMetricName)) {

Review comment:
       Wouldn't 




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@samza.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [samza] rmatharu commented on a change in pull request #1558: SAMZA-2707: Convert metrics snapshot reporter code to java

Posted by GitBox <gi...@apache.org>.
rmatharu commented on a change in pull request #1558:
URL: https://github.com/apache/samza/pull/1558#discussion_r754523736



##########
File path: samza-core/src/test/java/org/apache/samza/metrics/reporter/TestMetricsSnapshotReporterFactory.java
##########
@@ -0,0 +1,117 @@
+/*
+ * 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.samza.metrics.reporter;
+
+import com.google.common.collect.ImmutableMap;
+import org.apache.samza.config.Config;
+import org.apache.samza.config.MapConfig;
+import org.apache.samza.metrics.MetricsRegistry;
+import org.apache.samza.metrics.MetricsRegistryMap;
+import org.apache.samza.serializers.MetricsSnapshotSerdeV2;
+import org.apache.samza.serializers.Serde;
+import org.apache.samza.serializers.SerdeFactory;
+import org.apache.samza.system.SystemAdmin;
+import org.apache.samza.system.SystemConsumer;
+import org.apache.samza.system.SystemFactory;
+import org.apache.samza.system.SystemProducer;
+import org.apache.samza.system.SystemStream;
+import org.junit.Before;
+import org.junit.Test;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNull;
+import static org.junit.Assert.assertTrue;
+import static org.mockito.Mockito.mock;
+
+
+public class TestMetricsSnapshotReporterFactory {
+  private static final String REPORTER = "metrics-reporter";
+  private static final SystemProducer SYSTEM_PRODUCER = mock(SystemProducer.class);
+  private static final Serde<MetricsSnapshot> SERDE = mock(Serde.class);
+
+  private MetricsSnapshotReporterFactory factory;
+
+  @Before
+  public void setup() {
+    this.factory = new MetricsSnapshotReporterFactory();
+  }
+
+  @Test
+  public void testGetProducer() {
+    Config config = new MapConfig(
+        ImmutableMap.of("metrics.reporter.metrics-reporter.stream", "system0.stream0", "systems.system0.samza.factory",
+            MockSystemFactory.class.getName()));
+    assertEquals(SYSTEM_PRODUCER, this.factory.getProducer(REPORTER, config, new MetricsRegistryMap()));
+  }
+

Review comment:
       Would it make sense to add a test to compare the serialized format of a sample object 
   before and after this change? 
   Similarly for deserialization -- to ensure the new code can deserialize things serialized using previous code.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@samza.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [samza] ZitingShen commented on a change in pull request #1558: SAMZA-2707: Convert metrics snapshot reporter code to java

Posted by GitBox <gi...@apache.org>.
ZitingShen commented on a change in pull request #1558:
URL: https://github.com/apache/samza/pull/1558#discussion_r749736685



##########
File path: samza-core/src/main/java/org/apache/samza/metrics/reporter/Metrics.java
##########
@@ -0,0 +1,76 @@
+/*
+ * 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.samza.metrics.reporter;
+
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.Objects;
+
+
+public class Metrics {
+  private final Map<String, Map<String, Object>> immutableMetrics = new HashMap<>();
+
+  public Metrics() {
+    this(Collections.emptyMap());
+  }
+
+  public Metrics(Map<String, Map<String, Object>> metrics) {
+    metrics.forEach((groupKey, groupValue) -> this.immutableMetrics.put(groupKey,
+        Collections.unmodifiableMap(new HashMap<>(groupValue))));
+  }
+
+  public <T> T get(String group, String metricName) {
+    return (T) this.immutableMetrics.get(group).get(metricName);
+  }
+
+  public Map<String, Object> get(String group) {
+    return this.immutableMetrics.get(group);
+  }
+
+  public Map<String, Map<String, Object>> getAsMap() {
+    return Collections.unmodifiableMap(this.immutableMetrics);

Review comment:
       I know this is how the code was written in the original Scala files, but would it be better if we construct `immutableMetrics` as `Collections.unmodifiableMap(...)`?  We are not modifying the map anyways, right?

##########
File path: samza-core/src/main/java/org/apache/samza/metrics/reporter/MetricsSnapshotReporter.java
##########
@@ -0,0 +1,228 @@
+/*
+ * 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.samza.metrics.reporter;
+
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Set;
+import java.util.concurrent.Executors;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.TimeUnit;
+import com.google.common.util.concurrent.ThreadFactoryBuilder;
+import org.apache.samza.SamzaException;
+import org.apache.samza.config.ShellCommandConfig;
+import org.apache.samza.metrics.Counter;
+import org.apache.samza.metrics.Gauge;
+import org.apache.samza.metrics.MetricsReporter;
+import org.apache.samza.metrics.MetricsVisitor;
+import org.apache.samza.metrics.ReadableMetricsRegistry;
+import org.apache.samza.metrics.RegistryWithSource;
+import org.apache.samza.metrics.Timer;
+import org.apache.samza.serializers.Serializer;
+import org.apache.samza.system.OutgoingMessageEnvelope;
+import org.apache.samza.system.SystemProducer;
+import org.apache.samza.system.SystemStream;
+import org.apache.samza.util.Clock;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+/**
+ * MetricsSnapshotReporter is a generic metrics reporter that sends metrics to a stream.
+ *
+ * jobName // my-samza-job
+ * jobId // an id that differentiates multiple executions of the same job
+ * taskName // container_567890
+ * host // eat1-app128.gird
+ * version // 0.0.1
+ * blacklist // Regex of metrics to ignore when flushing
+ */
+public class MetricsSnapshotReporter implements MetricsReporter, Runnable {
+  private static final Logger LOG = LoggerFactory.getLogger(MetricsSnapshotReporter.class);
+
+  private final SystemProducer producer;
+  private final SystemStream out;
+  private final int reportingInterval;
+  private final String jobName;
+  private final String jobId;
+  private final String containerName;
+  private final String version;
+  private final String samzaVersion;
+  private final String host;
+  private final Serializer<MetricsSnapshot> serializer;
+  private final Optional<String> blacklist;
+  private final Clock clock;
+
+  private final String execEnvironmentContainerId;
+  private final ScheduledExecutorService executor;
+  private final long resetTime;
+  private final List<RegistryWithSource> registries = new ArrayList<>();
+  private final Set<String> blacklistedMetrics = new HashSet<>();
+
+  public MetricsSnapshotReporter(SystemProducer producer, SystemStream out, int reportingInterval, String jobName,
+      String jobId, String containerName, String version, String samzaVersion, String host,
+      Serializer<MetricsSnapshot> serializer, Optional<String> blacklist, Clock clock) {
+    this.producer = producer;
+    this.out = out;
+    this.reportingInterval = reportingInterval;
+    this.jobName = jobName;
+    this.jobId = jobId;
+    this.containerName = containerName;
+    this.version = version;
+    this.samzaVersion = samzaVersion;
+    this.host = host;
+    this.serializer = serializer;
+    this.blacklist = blacklist;
+    this.clock = clock;

Review comment:
       You don't really need `clock`, right? `clock` is just a function that returns `System.currentTimeMillis`

##########
File path: samza-api/src/main/java/org/apache/samza/metrics/RegistryWithSource.java
##########
@@ -16,28 +16,26 @@
  * specific language governing permissions and limitations
  * under the License.
  */
+package org.apache.samza.metrics;
 
-package org.apache.samza.metrics.reporter
-
-import java.util.Map
-import java.util.HashMap
-import scala.beans.BeanProperty
+/**
+ * Simple holder for {@link ReadableMetricsRegistry} and its source, used for implementations of
+ * {@link MetricsReporter}.
+ */
+public class RegistryWithSource {

Review comment:
       Is it because there are duplicate resources so that we don't want to use `Map<String, ReadableMetricsRegistry>`?

##########
File path: samza-api/src/main/java/org/apache/samza/metrics/RegistryWithSource.java
##########
@@ -16,28 +16,26 @@
  * specific language governing permissions and limitations
  * under the License.
  */
+package org.apache.samza.metrics;
 
-package org.apache.samza.metrics.reporter
-
-import java.util.Map
-import java.util.HashMap
-import scala.beans.BeanProperty
+/**
+ * Simple holder for {@link ReadableMetricsRegistry} and its source, used for implementations of
+ * {@link MetricsReporter}.
+ */
+public class RegistryWithSource {
+  private final String source;
+  private final ReadableMetricsRegistry registry;
 
-object MetricsSnapshot {
-  def fromMap(map: Map[String, Map[String, Object]]) = {
-    val header = MetricsHeader.fromMap(map.get("header"))
-    val metrics = Metrics.fromMap(map.get("metrics").asInstanceOf[Map[String, Map[String, Object]]])
-    new MetricsSnapshot(header, metrics)
+  public RegistryWithSource(String source, ReadableMetricsRegistry registry) {

Review comment:
       Should this be named as `MetricsRegistryWithSource`? `RegistryWithSource` is very general but the constructor param limits it to be only applicable for `ReadableMetricsRegistry`

##########
File path: samza-core/src/main/java/org/apache/samza/metrics/reporter/MetricsHeader.java
##########
@@ -0,0 +1,160 @@
+/*
+ * 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.samza.metrics.reporter;
+
+import java.util.HashMap;
+import java.util.Map;
+import java.util.Objects;
+
+
+public class MetricsHeader {
+  private static final String JOB_NAME = "job-name";
+  private static final String JOB_ID = "job-id";
+  private static final String CONTAINER_NAME = "container-name";
+  private static final String EXEC_ENV_CONTAINER_ID = "exec-env-container-id";
+  private static final String SOURCE = "source";
+  private static final String VERSION = "version";
+  private static final String SAMZA_VERSION = "samza-version";
+  private static final String HOST = "host";
+  private static final String TIME = "time";
+  private static final String RESET_TIME = "reset-time";
+
+  private final String jobName;
+  private final String jobId;
+  private final String containerName;
+  private final String execEnvironmentContainerId;
+  private final String source;
+  private final String version;
+  private final String samzaVersion;
+  private final String host;
+  private final long time;
+  private final long resetTime;
+
+  public MetricsHeader(String jobName, String jobId, String containerName, String execEnvironmentContainerId,
+      String source, String version, String samzaVersion, String host, long time, long resetTime) {
+    this.jobName = jobName;
+    this.jobId = jobId;
+    this.containerName = containerName;
+    this.execEnvironmentContainerId = execEnvironmentContainerId;
+    this.source = source;
+    this.version = version;
+    this.samzaVersion = samzaVersion;
+    this.host = host;
+    this.time = time;
+    this.resetTime = resetTime;
+  }
+
+  public Map<String, Object> getAsMap() {
+    Map<String, Object> map = new HashMap<>();
+    map.put(JOB_NAME, jobName);
+    map.put(JOB_ID, jobId);
+    map.put(CONTAINER_NAME, containerName);
+    map.put(EXEC_ENV_CONTAINER_ID, execEnvironmentContainerId);
+    map.put(SOURCE, source);
+    map.put(VERSION, version);
+    map.put(SAMZA_VERSION, samzaVersion);
+    map.put(HOST, host);
+    map.put(TIME, time);
+    map.put(RESET_TIME, resetTime);
+    return map;
+  }
+
+  public String getJobName() {
+    return jobName;
+  }
+
+  public String getJobId() {
+    return jobId;
+  }
+
+  public String getContainerName() {
+    return containerName;
+  }
+
+  public String getExecEnvironmentContainerId() {
+    return execEnvironmentContainerId;
+  }
+
+  public String getSource() {
+    return source;
+  }
+
+  public String getVersion() {
+    return version;
+  }
+
+  public String getSamzaVersion() {
+    return samzaVersion;
+  }
+
+  public String getHost() {
+    return host;
+  }
+
+  public long getTime() {
+    return time;
+  }
+
+  public long getResetTime() {
+    return resetTime;
+  }
+
+  public static MetricsHeader fromMap(Map<String, Object> map) {
+    return new MetricsHeader(map.get(JOB_NAME).toString(),
+        map.get(JOB_ID).toString(),
+        map.get(CONTAINER_NAME).toString(),
+        map.get(EXEC_ENV_CONTAINER_ID).toString(),
+        map.get(SOURCE).toString(),
+        map.get(VERSION).toString(),
+        map.get(SAMZA_VERSION).toString(),
+        map.get(HOST).toString(),
+        ((Number) map.get(TIME)).longValue(),
+        ((Number) map.get(RESET_TIME)).longValue());

Review comment:
       Why do you need the `(Number)` cast?




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@samza.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [samza] cameronlee314 commented on a change in pull request #1558: SAMZA-2707: Convert metrics snapshot reporter code to java

Posted by GitBox <gi...@apache.org>.
cameronlee314 commented on a change in pull request #1558:
URL: https://github.com/apache/samza/pull/1558#discussion_r750594083



##########
File path: samza-core/src/main/java/org/apache/samza/metrics/reporter/MetricsSnapshotReporter.java
##########
@@ -0,0 +1,228 @@
+/*
+ * 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.samza.metrics.reporter;
+
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Set;
+import java.util.concurrent.Executors;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.TimeUnit;
+import com.google.common.util.concurrent.ThreadFactoryBuilder;
+import org.apache.samza.SamzaException;
+import org.apache.samza.config.ShellCommandConfig;
+import org.apache.samza.metrics.Counter;
+import org.apache.samza.metrics.Gauge;
+import org.apache.samza.metrics.MetricsReporter;
+import org.apache.samza.metrics.MetricsVisitor;
+import org.apache.samza.metrics.ReadableMetricsRegistry;
+import org.apache.samza.metrics.RegistryWithSource;
+import org.apache.samza.metrics.Timer;
+import org.apache.samza.serializers.Serializer;
+import org.apache.samza.system.OutgoingMessageEnvelope;
+import org.apache.samza.system.SystemProducer;
+import org.apache.samza.system.SystemStream;
+import org.apache.samza.util.Clock;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+/**
+ * MetricsSnapshotReporter is a generic metrics reporter that sends metrics to a stream.
+ *
+ * jobName // my-samza-job
+ * jobId // an id that differentiates multiple executions of the same job
+ * taskName // container_567890
+ * host // eat1-app128.gird
+ * version // 0.0.1
+ * blacklist // Regex of metrics to ignore when flushing
+ */
+public class MetricsSnapshotReporter implements MetricsReporter, Runnable {
+  private static final Logger LOG = LoggerFactory.getLogger(MetricsSnapshotReporter.class);
+
+  private final SystemProducer producer;
+  private final SystemStream out;
+  private final int reportingInterval;
+  private final String jobName;
+  private final String jobId;
+  private final String containerName;
+  private final String version;
+  private final String samzaVersion;
+  private final String host;
+  private final Serializer<MetricsSnapshot> serializer;
+  private final Optional<String> blacklist;
+  private final Clock clock;
+
+  private final String execEnvironmentContainerId;
+  private final ScheduledExecutorService executor;
+  private final long resetTime;
+  private final List<RegistryWithSource> registries = new ArrayList<>();
+  private final Set<String> blacklistedMetrics = new HashSet<>();
+
+  public MetricsSnapshotReporter(SystemProducer producer, SystemStream out, int reportingInterval, String jobName,
+      String jobId, String containerName, String version, String samzaVersion, String host,
+      Serializer<MetricsSnapshot> serializer, Optional<String> blacklist, Clock clock) {
+    this.producer = producer;
+    this.out = out;
+    this.reportingInterval = reportingInterval;
+    this.jobName = jobName;
+    this.jobId = jobId;
+    this.containerName = containerName;
+    this.version = version;
+    this.samzaVersion = samzaVersion;
+    this.host = host;
+    this.serializer = serializer;
+    this.blacklist = blacklist;
+    this.clock = clock;

Review comment:
       `Clock` is nice to inject because it makes testing easier (i.e. mock the clock instead of relying on actual system timing). It might not be used in the existing test for this class, but it's a pattern I like to follow in general.

##########
File path: samza-core/src/main/java/org/apache/samza/metrics/reporter/MetricsHeader.java
##########
@@ -0,0 +1,160 @@
+/*
+ * 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.samza.metrics.reporter;
+
+import java.util.HashMap;
+import java.util.Map;
+import java.util.Objects;
+
+
+public class MetricsHeader {
+  private static final String JOB_NAME = "job-name";
+  private static final String JOB_ID = "job-id";
+  private static final String CONTAINER_NAME = "container-name";
+  private static final String EXEC_ENV_CONTAINER_ID = "exec-env-container-id";
+  private static final String SOURCE = "source";
+  private static final String VERSION = "version";
+  private static final String SAMZA_VERSION = "samza-version";
+  private static final String HOST = "host";
+  private static final String TIME = "time";
+  private static final String RESET_TIME = "reset-time";
+
+  private final String jobName;
+  private final String jobId;
+  private final String containerName;
+  private final String execEnvironmentContainerId;
+  private final String source;
+  private final String version;
+  private final String samzaVersion;
+  private final String host;
+  private final long time;
+  private final long resetTime;
+
+  public MetricsHeader(String jobName, String jobId, String containerName, String execEnvironmentContainerId,
+      String source, String version, String samzaVersion, String host, long time, long resetTime) {
+    this.jobName = jobName;
+    this.jobId = jobId;
+    this.containerName = containerName;
+    this.execEnvironmentContainerId = execEnvironmentContainerId;
+    this.source = source;
+    this.version = version;
+    this.samzaVersion = samzaVersion;
+    this.host = host;
+    this.time = time;
+    this.resetTime = resetTime;
+  }
+
+  public Map<String, Object> getAsMap() {
+    Map<String, Object> map = new HashMap<>();
+    map.put(JOB_NAME, jobName);
+    map.put(JOB_ID, jobId);
+    map.put(CONTAINER_NAME, containerName);
+    map.put(EXEC_ENV_CONTAINER_ID, execEnvironmentContainerId);
+    map.put(SOURCE, source);
+    map.put(VERSION, version);
+    map.put(SAMZA_VERSION, samzaVersion);
+    map.put(HOST, host);
+    map.put(TIME, time);
+    map.put(RESET_TIME, resetTime);
+    return map;
+  }
+
+  public String getJobName() {
+    return jobName;
+  }
+
+  public String getJobId() {
+    return jobId;
+  }
+
+  public String getContainerName() {
+    return containerName;
+  }
+
+  public String getExecEnvironmentContainerId() {
+    return execEnvironmentContainerId;
+  }
+
+  public String getSource() {
+    return source;
+  }
+
+  public String getVersion() {
+    return version;
+  }
+
+  public String getSamzaVersion() {
+    return samzaVersion;
+  }
+
+  public String getHost() {
+    return host;
+  }
+
+  public long getTime() {
+    return time;
+  }
+
+  public long getResetTime() {
+    return resetTime;
+  }
+
+  public static MetricsHeader fromMap(Map<String, Object> map) {
+    return new MetricsHeader(map.get(JOB_NAME).toString(),
+        map.get(JOB_ID).toString(),
+        map.get(CONTAINER_NAME).toString(),
+        map.get(EXEC_ENV_CONTAINER_ID).toString(),
+        map.get(SOURCE).toString(),
+        map.get(VERSION).toString(),
+        map.get(SAMZA_VERSION).toString(),
+        map.get(HOST).toString(),
+        ((Number) map.get(TIME)).longValue(),
+        ((Number) map.get(RESET_TIME)).longValue());

Review comment:
       The value of the `map` is `Object`, so we need to convert to `long`. Using `Number.longValue` allows handling of `int`, `long`, `float`, etc., so we don't need to worry about how deserialization of numbers works for JSON.

##########
File path: samza-api/src/main/java/org/apache/samza/metrics/RegistryWithSource.java
##########
@@ -16,28 +16,26 @@
  * specific language governing permissions and limitations
  * under the License.
  */
+package org.apache.samza.metrics;
 
-package org.apache.samza.metrics.reporter
-
-import java.util.Map
-import java.util.HashMap
-import scala.beans.BeanProperty
+/**
+ * Simple holder for {@link ReadableMetricsRegistry} and its source, used for implementations of
+ * {@link MetricsReporter}.
+ */
+public class RegistryWithSource {

Review comment:
       Yes, the registry container is implemented as a list, so it is possible for the same source to be associated with two different registries.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@samza.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [samza] ryucc commented on a change in pull request #1558: SAMZA-2707: Convert metrics snapshot reporter code to java

Posted by GitBox <gi...@apache.org>.
ryucc commented on a change in pull request #1558:
URL: https://github.com/apache/samza/pull/1558#discussion_r749620235



##########
File path: samza-core/src/main/java/org/apache/samza/metrics/reporter/MetricsSnapshotReporter.java
##########
@@ -0,0 +1,228 @@
+/*
+ * 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.samza.metrics.reporter;
+
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Set;
+import java.util.concurrent.Executors;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.TimeUnit;
+import com.google.common.util.concurrent.ThreadFactoryBuilder;
+import org.apache.samza.SamzaException;
+import org.apache.samza.config.ShellCommandConfig;
+import org.apache.samza.metrics.Counter;
+import org.apache.samza.metrics.Gauge;
+import org.apache.samza.metrics.MetricsReporter;
+import org.apache.samza.metrics.MetricsVisitor;
+import org.apache.samza.metrics.ReadableMetricsRegistry;
+import org.apache.samza.metrics.RegistryWithSource;
+import org.apache.samza.metrics.Timer;
+import org.apache.samza.serializers.Serializer;
+import org.apache.samza.system.OutgoingMessageEnvelope;
+import org.apache.samza.system.SystemProducer;
+import org.apache.samza.system.SystemStream;
+import org.apache.samza.util.Clock;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+/**
+ * MetricsSnapshotReporter is a generic metrics reporter that sends metrics to a stream.
+ *
+ * jobName // my-samza-job
+ * jobId // an id that differentiates multiple executions of the same job
+ * taskName // container_567890
+ * host // eat1-app128.gird
+ * version // 0.0.1
+ * blacklist // Regex of metrics to ignore when flushing
+ */
+public class MetricsSnapshotReporter implements MetricsReporter, Runnable {
+  private static final Logger LOG = LoggerFactory.getLogger(MetricsSnapshotReporter.class);
+
+  private final SystemProducer producer;
+  private final SystemStream out;
+  private final int reportingInterval;
+  private final String jobName;
+  private final String jobId;
+  private final String containerName;
+  private final String version;
+  private final String samzaVersion;
+  private final String host;
+  private final Serializer<MetricsSnapshot> serializer;
+  private final Optional<String> blacklist;
+  private final Clock clock;
+
+  private final String execEnvironmentContainerId;
+  private final ScheduledExecutorService executor;
+  private final long resetTime;
+  private final List<RegistryWithSource> registries = new ArrayList<>();
+  private final Set<String> blacklistedMetrics = new HashSet<>();
+
+  public MetricsSnapshotReporter(SystemProducer producer, SystemStream out, int reportingInterval, String jobName,
+      String jobId, String containerName, String version, String samzaVersion, String host,
+      Serializer<MetricsSnapshot> serializer, Optional<String> blacklist, Clock clock) {
+    this.producer = producer;
+    this.out = out;
+    this.reportingInterval = reportingInterval;
+    this.jobName = jobName;
+    this.jobId = jobId;
+    this.containerName = containerName;
+    this.version = version;
+    this.samzaVersion = samzaVersion;
+    this.host = host;
+    this.serializer = serializer;
+    this.blacklist = blacklist;
+    this.clock = clock;
+
+    this.execEnvironmentContainerId =
+        Optional.ofNullable(System.getenv(ShellCommandConfig.ENV_EXECUTION_ENV_CONTAINER_ID)).orElse("");
+    this.executor = Executors.newSingleThreadScheduledExecutor(
+        new ThreadFactoryBuilder().setNameFormat("Samza MetricsSnapshotReporter Thread-%d").setDaemon(true).build());
+    this.resetTime = this.clock.currentTimeMillis();
+    LOG.info(
+        "got metrics snapshot reporter properties [job name: {}, job id: {}, containerName: {}, version: {}, samzaVersion: {}, host: {}, reportingInterval {}]",
+        jobName, jobId, containerName, version, samzaVersion, host, reportingInterval);
+  }
+
+  @Override
+  public void start() {
+    LOG.info("Starting producer.");
+    this.producer.start();
+    LOG.info("Starting reporter timer.");
+    this.executor.scheduleWithFixedDelay(this, 0, reportingInterval, TimeUnit.SECONDS);
+  }
+
+  @Override
+  public void register(String source, ReadableMetricsRegistry registry) {
+    this.registries.add(new RegistryWithSource(source, registry));
+    LOG.info("Registering {} with producer.", source);
+    this.producer.register(source);
+  }
+
+  @Override
+  public void stop() {
+    // Scheduling an event with 0 delay to ensure flushing of metrics one last time before shutdown
+    this.executor.schedule(this, 0, TimeUnit.SECONDS);
+    LOG.info("Stopping reporter timer.");
+    // Allow the scheduled task above to finish, and block for termination (for max 60 seconds)
+    this.executor.shutdown();
+    try {
+      this.executor.awaitTermination(60, TimeUnit.SECONDS);
+    } catch (InterruptedException e) {
+      throw new SamzaException(e);
+    }
+    LOG.info("Stopping producer.");
+    this.producer.stop();
+    if (!this.executor.isTerminated()) {
+      LOG.warn("Unable to shutdown reporter timer.");
+    }
+  }
+
+  @Override
+  public void run() {
+    try {
+      innerRun();
+    } catch (Exception e) {
+      // Ignore all exceptions - because subsequent executions of this scheduled task will be suppressed
+      // by the executor if the current task throws an unhandled exception.
+      LOG.warn("Error while reporting metrics. Will retry in " + reportingInterval + " seconds.", e);
+    }
+  }
+
+  public void innerRun() {
+    LOG.debug("Begin flushing metrics.");
+    for (RegistryWithSource registryWithSource : this.registries) {
+      String source = registryWithSource.getSource();
+      ReadableMetricsRegistry registry = registryWithSource.getRegistry();
+      LOG.debug("Flushing metrics for {}.", source);
+      Map<String, Map<String, Object>> metricsMsg = new HashMap<>();
+
+      // metrics
+      registry.getGroups().forEach(group -> {
+        Map<String, Object> groupMsg = new HashMap<>();
+        registry.getGroup(group).forEach((name, metric) -> {
+          if (!shouldIgnore(group, name)) {
+            metric.visit(new MetricsVisitor() {
+              @Override
+              public void counter(Counter counter) {
+                groupMsg.put(name, counter.getCount());
+              }
+
+              @Override
+              public <T> void gauge(Gauge<T> gauge) {
+                groupMsg.put(name, gauge.getValue());
+              }
+
+              @Override
+              public void timer(Timer timer) {
+                groupMsg.put(name, timer.getSnapshot().getAverage());
+              }
+            });
+          }
+        });
+
+        // dont emit empty groups
+        if (!groupMsg.isEmpty()) {
+          metricsMsg.put(group, groupMsg);
+        }
+      });
+
+      // publish to Kafka only if the metricsMsg carries any metrics
+      if (!metricsMsg.isEmpty()) {
+        MetricsHeader header =
+            new MetricsHeader(this.jobName, this.jobId, this.containerName, this.execEnvironmentContainerId, source,
+                this.version, this.samzaVersion, this.host, this.clock.currentTimeMillis(), this.resetTime);
+        Metrics metrics = new Metrics(metricsMsg);
+        LOG.debug("Flushing metrics for {} to {} with header and map: header={}, map={}.", source, out,
+            header.getAsMap(), metrics.getAsMap());
+        MetricsSnapshot metricsSnapshot = new MetricsSnapshot(header, metrics);
+        Object maybeSerialized = (this.serializer != null) ? this.serializer.toBytes(metricsSnapshot) : metricsSnapshot;
+        try {
+          this.producer.send(source, new OutgoingMessageEnvelope(this.out, this.host, null, maybeSerialized));
+          // Always flush, since we don't want metrics to get batched up.
+          this.producer.flush(source);
+        } catch (Exception e) {
+          LOG.error(String.format("Exception when flushing metrics for source %s", source), e);
+        }
+      }
+    }
+    LOG.debug("Finished flushing metrics.");
+  }
+
+  protected boolean shouldIgnore(String group, String metricName) {
+    boolean isBlacklisted = this.blacklist.isPresent();
+    String fullMetricName = group + "." + metricName;
+
+    if (isBlacklisted && !this.blacklistedMetrics.contains(fullMetricName)) {

Review comment:
       Wouldn't blacklistedMetrics.contains evaluate to true the second time, and not filter out the metric?




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@samza.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [samza] rmatharu commented on a change in pull request #1558: SAMZA-2707: Convert metrics snapshot reporter code to java

Posted by GitBox <gi...@apache.org>.
rmatharu commented on a change in pull request #1558:
URL: https://github.com/apache/samza/pull/1558#discussion_r754522571



##########
File path: samza-core/src/main/java/org/apache/samza/metrics/reporter/MetricsSnapshotReporter.java
##########
@@ -0,0 +1,228 @@
+/*
+ * 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.samza.metrics.reporter;
+
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Set;
+import java.util.concurrent.Executors;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.TimeUnit;
+import com.google.common.util.concurrent.ThreadFactoryBuilder;
+import org.apache.samza.SamzaException;
+import org.apache.samza.config.ShellCommandConfig;
+import org.apache.samza.metrics.Counter;
+import org.apache.samza.metrics.Gauge;
+import org.apache.samza.metrics.MetricsReporter;
+import org.apache.samza.metrics.MetricsVisitor;
+import org.apache.samza.metrics.ReadableMetricsRegistry;
+import org.apache.samza.metrics.MetricsRegistryWithSource;
+import org.apache.samza.metrics.Timer;
+import org.apache.samza.serializers.Serializer;
+import org.apache.samza.system.OutgoingMessageEnvelope;
+import org.apache.samza.system.SystemProducer;
+import org.apache.samza.system.SystemStream;
+import org.apache.samza.util.Clock;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+/**
+ * MetricsSnapshotReporter is a generic metrics reporter that sends metrics to a stream.
+ *
+ * jobName // my-samza-job
+ * jobId // an id that differentiates multiple executions of the same job
+ * taskName // container_567890
+ * host // eat1-app128.gird
+ * version // 0.0.1
+ * blacklist // Regex of metrics to ignore when flushing
+ */
+public class MetricsSnapshotReporter implements MetricsReporter, Runnable {
+  private static final Logger LOG = LoggerFactory.getLogger(MetricsSnapshotReporter.class);
+
+  private final SystemProducer producer;
+  private final SystemStream out;
+  private final int reportingInterval;
+  private final String jobName;
+  private final String jobId;
+  private final String containerName;
+  private final String version;
+  private final String samzaVersion;
+  private final String host;
+  private final Serializer<MetricsSnapshot> serializer;
+  private final Optional<String> blacklist;

Review comment:
       Maybe just a Pattern 
   which can be Pattern.compile("") to indicate empty/no filter




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@samza.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [samza] rmatharu commented on a change in pull request #1558: SAMZA-2707: Convert metrics snapshot reporter code to java

Posted by GitBox <gi...@apache.org>.
rmatharu commented on a change in pull request #1558:
URL: https://github.com/apache/samza/pull/1558#discussion_r754522138



##########
File path: samza-core/src/main/java/org/apache/samza/metrics/reporter/MetricsSnapshotReporter.java
##########
@@ -0,0 +1,228 @@
+/*
+ * 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.samza.metrics.reporter;
+
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Set;
+import java.util.concurrent.Executors;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.TimeUnit;
+import com.google.common.util.concurrent.ThreadFactoryBuilder;
+import org.apache.samza.SamzaException;
+import org.apache.samza.config.ShellCommandConfig;
+import org.apache.samza.metrics.Counter;
+import org.apache.samza.metrics.Gauge;
+import org.apache.samza.metrics.MetricsReporter;
+import org.apache.samza.metrics.MetricsVisitor;
+import org.apache.samza.metrics.ReadableMetricsRegistry;
+import org.apache.samza.metrics.MetricsRegistryWithSource;
+import org.apache.samza.metrics.Timer;
+import org.apache.samza.serializers.Serializer;
+import org.apache.samza.system.OutgoingMessageEnvelope;
+import org.apache.samza.system.SystemProducer;
+import org.apache.samza.system.SystemStream;
+import org.apache.samza.util.Clock;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+/**
+ * MetricsSnapshotReporter is a generic metrics reporter that sends metrics to a stream.
+ *
+ * jobName // my-samza-job
+ * jobId // an id that differentiates multiple executions of the same job
+ * taskName // container_567890
+ * host // eat1-app128.gird
+ * version // 0.0.1
+ * blacklist // Regex of metrics to ignore when flushing
+ */
+public class MetricsSnapshotReporter implements MetricsReporter, Runnable {
+  private static final Logger LOG = LoggerFactory.getLogger(MetricsSnapshotReporter.class);
+
+  private final SystemProducer producer;
+  private final SystemStream out;
+  private final int reportingInterval;

Review comment:
       Duration?




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@samza.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [samza] rmatharu commented on a change in pull request #1558: SAMZA-2707: Convert metrics snapshot reporter code to java

Posted by GitBox <gi...@apache.org>.
rmatharu commented on a change in pull request #1558:
URL: https://github.com/apache/samza/pull/1558#discussion_r754521797



##########
File path: samza-core/src/main/java/org/apache/samza/metrics/reporter/MetricsHeader.java
##########
@@ -0,0 +1,160 @@
+/*
+ * 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.samza.metrics.reporter;
+
+import java.util.HashMap;
+import java.util.Map;
+import java.util.Objects;
+
+
+public class MetricsHeader {
+  private static final String JOB_NAME = "job-name";
+  private static final String JOB_ID = "job-id";
+  private static final String CONTAINER_NAME = "container-name";
+  private static final String EXEC_ENV_CONTAINER_ID = "exec-env-container-id";
+  private static final String SOURCE = "source";
+  private static final String VERSION = "version";
+  private static final String SAMZA_VERSION = "samza-version";
+  private static final String HOST = "host";
+  private static final String TIME = "time";
+  private static final String RESET_TIME = "reset-time";

Review comment:
       In the scala world, these used to be long, will changing it here require any change in the serde to make sure the output on the wire remains the same?




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@samza.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [samza] cameronlee314 merged pull request #1558: SAMZA-2707: Convert metrics snapshot reporter code to java

Posted by GitBox <gi...@apache.org>.
cameronlee314 merged pull request #1558:
URL: https://github.com/apache/samza/pull/1558


   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@samza.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org