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/16 01:02:15 UTC

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

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