From 5ae1df9203342e65ea97fbb9471046e036bde74c Mon Sep 17 00:00:00 2001 From: Ray Matharu Date: Thu, 10 May 2018 16:31:07 -0700 Subject: [PATCH 01/33] Added exception metric (as string-guage), to be emitted using the snapshot reporter. Switched producer-shutdown logic to ensure metric-flush at shutdown-time --- .../samza/container/SamzaContainer.scala | 3 +++ .../container/SamzaContainerMetrics.scala | 3 +++ .../reporter/MetricsSnapshotReporter.scala | 20 +++++++++++++------ 3 files changed, 20 insertions(+), 6 deletions(-) diff --git a/samza-core/src/main/scala/org/apache/samza/container/SamzaContainer.scala b/samza-core/src/main/scala/org/apache/samza/container/SamzaContainer.scala index 5380fc98dc..d46363986d 100644 --- a/samza-core/src/main/scala/org/apache/samza/container/SamzaContainer.scala +++ b/samza-core/src/main/scala/org/apache/samza/container/SamzaContainer.scala @@ -28,6 +28,7 @@ import java.util.Base64 import java.util.concurrent.{ExecutorService, Executors, ScheduledExecutorService, TimeUnit} import com.google.common.util.concurrent.ThreadFactoryBuilder +import org.apache.commons.lang3.exception.ExceptionUtils import org.apache.samza.checkpoint.{CheckpointListener, CheckpointManagerFactory, OffsetManager, OffsetManagerMetrics} import org.apache.samza.config.JobConfig.Config2Job import org.apache.samza.config.MetricsConfig.Config2Metrics @@ -751,6 +752,8 @@ class SamzaContainer( } status = SamzaContainerStatus.FAILED exceptionSeen = e + metrics.exceptionAtShutdown.set(ExceptionUtils.getStackTrace(e)) + debug("Updated value of exceptionAtShutdown to %s" format ExceptionUtils.getStackTrace(e)) } try { diff --git a/samza-core/src/main/scala/org/apache/samza/container/SamzaContainerMetrics.scala b/samza-core/src/main/scala/org/apache/samza/container/SamzaContainerMetrics.scala index c1229565ad..c08433cabd 100644 --- a/samza-core/src/main/scala/org/apache/samza/container/SamzaContainerMetrics.scala +++ b/samza-core/src/main/scala/org/apache/samza/container/SamzaContainerMetrics.scala @@ -48,6 +48,9 @@ class SamzaContainerMetrics( val taskStoreRestorationMetrics: util.Map[TaskName, Gauge[Long]] = new util.HashMap[TaskName, Gauge[Long]]() + // A string-gauge metric to capture unhandled exceptions at this container that lead to a container shutdown + val exceptionAtShutdown = newGauge[String]("exception", "") + def addStoreRestorationGauge(taskName: TaskName, storeName: String) { taskStoreRestorationMetrics.put(taskName, newGauge("%s-%s-restore-time" format(taskName.toString, storeName), -1L)) } diff --git a/samza-core/src/main/scala/org/apache/samza/metrics/reporter/MetricsSnapshotReporter.scala b/samza-core/src/main/scala/org/apache/samza/metrics/reporter/MetricsSnapshotReporter.scala index 65ca49c2f3..221832a4ba 100644 --- a/samza-core/src/main/scala/org/apache/samza/metrics/reporter/MetricsSnapshotReporter.scala +++ b/samza-core/src/main/scala/org/apache/samza/metrics/reporter/MetricsSnapshotReporter.scala @@ -83,15 +83,18 @@ class MetricsSnapshotReporter( } def stop = { - info("Stopping producer.") - producer.stop + // Scheduling an event with 0 delay to ensure flushing of metrics one last time before shutdown + executor.schedule(this,0, TimeUnit.SECONDS) info("Stopping reporter timer.") - + // Allow the scheduled task above to finish, and block for termination (for max 60 seconds) executor.shutdown executor.awaitTermination(60, TimeUnit.SECONDS) + info("Stopping producer.") + producer.stop + if (!executor.isTerminated) { warn("Unable to shutdown reporter timer.") } @@ -133,10 +136,15 @@ class MetricsSnapshotReporter( metricsSnapshot } - producer.send(source, new OutgoingMessageEnvelope(out, host, null, maybeSerialized)) + try { - // Always flush, since we don't want metrics to get batched up. - producer.flush(source) + producer.send(source, new OutgoingMessageEnvelope(out, host, null, maybeSerialized)) + + // Always flush, since we don't want metrics to get batched up. + producer.flush(source) + } catch { + case e: Exception => error("Exception when flushing metrics for source %s " format(source), e) + } } debug("Finished flushing metrics.") From 5a3ef7bc09a5ace726c1dcb61d66e036b972e210 Mon Sep 17 00:00:00 2001 From: Ray Matharu Date: Tue, 29 May 2018 22:56:02 -0700 Subject: [PATCH 02/33] Adding ListGauge and integration with SamzaContainerMetric --- .../org/apache/samza/metrics/ListGauge.java | 90 +++++++++++++++++ .../apache/samza/metrics/MetricsRegistry.java | 9 ++ .../apache/samza/metrics/MetricsVisitor.java | 7 +- .../ReadableMetricsRegistryListener.java | 2 + .../samza/util/NoOpMetricsRegistry.java | 7 ++ .../apache/samza/metrics/TestListGauge.java | 99 +++++++++++++++++++ .../org/apache/samza/metrics/TestTimer.java | 2 +- .../system/eventhub/TestMetricsRegistry.java | 45 +++++---- .../org/apache/samza/metrics/MetricGroup.java | 38 +++---- .../samza/container/SamzaContainer.scala | 6 +- .../container/SamzaContainerMetrics.scala | 4 +- .../apache/samza/metrics/MetricsHelper.scala | 6 +- .../samza/metrics/MetricsRegistryMap.scala | 27 +++-- .../samza/metrics/reporter/JmxReporter.scala | 30 ++++-- .../reporter/MetricsSnapshotReporter.scala | 6 +- .../webapp/ApplicationMasterRestServlet.scala | 3 + 16 files changed, 321 insertions(+), 60 deletions(-) create mode 100644 samza-api/src/main/java/org/apache/samza/metrics/ListGauge.java create mode 100644 samza-api/src/test/java/org/apache/samza/metrics/TestListGauge.java diff --git a/samza-api/src/main/java/org/apache/samza/metrics/ListGauge.java b/samza-api/src/main/java/org/apache/samza/metrics/ListGauge.java new file mode 100644 index 0000000000..744acbdfc7 --- /dev/null +++ b/samza-api/src/main/java/org/apache/samza/metrics/ListGauge.java @@ -0,0 +1,90 @@ +/* + * 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; + +import java.util.Collection; +import java.util.LinkedList; + + +/** + * A ListGauge is a collection of {@link org.apache.samza.metrics.Gauge}. ListGauges are useful for maintaining, recording, + * or collecting specific Gauge values over time. It is implemented as a {@link org.apache.samza.metrics.Metric}. + * For example, the set of recent errors that have occurred. + * + * This current implementation uses a size-bound-policy and holds the N most-recent Gauge objects added to the list. + * This bound N is configurable at instantiation time. + * TODO: Support a time-based and size-and-time-based hybrid policy. + * TODO: Add a derived class to do compaction for errors using hash-based errorIDs and adding timestamp for errors to dedup + * on the read path. + * + * All public methods are thread-safe. + * + */ +public class ListGauge implements Metric { + private final String name; + private final Collection metricList; + private int nItems; + + /** + * Create a new ListGauge. + * @param name Name to be assigned + * @param nItems The number of items to hold in the list + */ + public ListGauge(String name, int nItems) { + this.name = name; + this.metricList = new LinkedList<>(); + this.nItems = nItems; + } + + /** + * Get the name assigned to the ListGauge + * @return the assigned name + */ + public String getName() { + return this.name; + } + + /** + * Add a gauge to the list + * @param value The Gauge value to be added + */ + public synchronized void add(Gauge value) { + if (this.metricList.size() == nItems) { + ((LinkedList) this.metricList).removeFirst(); + } + + this.metricList.add(value); + } + + /** + * Get the Collection of Gauge values currently in the list + * @return the collection of gauge values + */ + public synchronized Collection getValue() { + return this.metricList; + } + + /** + * {@inheritDoc} + */ + @Override + public synchronized void visit(MetricsVisitor visitor) { + visitor.listGauge(this); + } +} diff --git a/samza-api/src/main/java/org/apache/samza/metrics/MetricsRegistry.java b/samza-api/src/main/java/org/apache/samza/metrics/MetricsRegistry.java index 5a00d01868..8f2cfb4870 100644 --- a/samza-api/src/main/java/org/apache/samza/metrics/MetricsRegistry.java +++ b/samza-api/src/main/java/org/apache/samza/metrics/MetricsRegistry.java @@ -64,6 +64,15 @@ public interface MetricsRegistry { */ Gauge newGauge(String group, Gauge value); + /** + * Register a {@link org.apache.samza.metrics.ListGauge} + * @param group Group for this ListGauge + * @param listGauge the ListGauge to register + * @param Type of the ListGauge + * @return ListGauge registered + */ + ListGauge newListGauge(String group, ListGauge listGauge); + /** * Create and Register a new {@link org.apache.samza.metrics.Timer} * @param group Group for this Timer diff --git a/samza-api/src/main/java/org/apache/samza/metrics/MetricsVisitor.java b/samza-api/src/main/java/org/apache/samza/metrics/MetricsVisitor.java index 75abfe7078..354a9ae54d 100644 --- a/samza-api/src/main/java/org/apache/samza/metrics/MetricsVisitor.java +++ b/samza-api/src/main/java/org/apache/samza/metrics/MetricsVisitor.java @@ -31,8 +31,13 @@ public abstract class MetricsVisitor { public abstract void timer(Timer timer); + public abstract void listGauge(ListGauge listGauge); + public void visit(Metric metric) { - if (metric instanceof Counter) { + // Cast for metrics of type ListGauge + if (metric instanceof ListGauge) { + listGauge((ListGauge) metric); + } else if (metric instanceof Counter) { counter((Counter) metric); } else if (metric instanceof Gauge) { gauge((Gauge) metric); diff --git a/samza-api/src/main/java/org/apache/samza/metrics/ReadableMetricsRegistryListener.java b/samza-api/src/main/java/org/apache/samza/metrics/ReadableMetricsRegistryListener.java index 739d68f8ef..84225f5baf 100644 --- a/samza-api/src/main/java/org/apache/samza/metrics/ReadableMetricsRegistryListener.java +++ b/samza-api/src/main/java/org/apache/samza/metrics/ReadableMetricsRegistryListener.java @@ -24,5 +24,7 @@ public interface ReadableMetricsRegistryListener { void onGauge(String group, Gauge gauge); + void onListGauge(String group, ListGauge listGauge); + void onTimer(String group, Timer timer); } diff --git a/samza-api/src/main/java/org/apache/samza/util/NoOpMetricsRegistry.java b/samza-api/src/main/java/org/apache/samza/util/NoOpMetricsRegistry.java index 3df855c2f7..73653ccfca 100644 --- a/samza-api/src/main/java/org/apache/samza/util/NoOpMetricsRegistry.java +++ b/samza-api/src/main/java/org/apache/samza/util/NoOpMetricsRegistry.java @@ -21,9 +21,11 @@ import org.apache.samza.metrics.Counter; import org.apache.samza.metrics.Gauge; +import org.apache.samza.metrics.ListGauge; import org.apache.samza.metrics.MetricsRegistry; import org.apache.samza.metrics.Timer; + /** * {@link org.apache.samza.metrics.MetricsRegistry} implementation for when no actual metrics need to be * recorded but a registry is still required. @@ -49,6 +51,11 @@ public Gauge newGauge(String group, Gauge gauge) { return gauge; } + @Override + public ListGauge newListGauge(String group, ListGauge listGauge) { + return listGauge; + } + @Override public Timer newTimer(String group, String name) { return new Timer(name); diff --git a/samza-api/src/test/java/org/apache/samza/metrics/TestListGauge.java b/samza-api/src/test/java/org/apache/samza/metrics/TestListGauge.java new file mode 100644 index 0000000000..1747006f47 --- /dev/null +++ b/samza-api/src/test/java/org/apache/samza/metrics/TestListGauge.java @@ -0,0 +1,99 @@ +/* + * 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; + +import java.time.Duration; +import java.util.Collection; +import java.util.Iterator; +import org.junit.Assert; +import org.junit.Test; + + +/** + * Class to encapsulate test-cases for {@link org.apache.samza.metrics.ListGauge} + */ +public class TestListGauge { + + private final static Duration THREAD_TEST_TIMEOUT = Duration.ofSeconds(10); + + @Test + public void basicTest() { + ListGauge listGauge = new ListGauge("listGauge", 10); + Gauge sampleGauge = new Gauge("key", "value"); + listGauge.add(sampleGauge); + Assert.assertEquals("Names should be the same", listGauge.getName(), "listGauge"); + Assert.assertEquals("List sizes should match", listGauge.getValue().size(), 1); + Assert.assertEquals("ListGauge should contain sampleGauge", listGauge.getValue().contains(sampleGauge), true); + } + + @Test + public void testSizeEnforcement() { + ListGauge listGauge = new ListGauge("listGauge", 10); + for (int i = 15; i > 0; i--) { + Gauge sampleGauge = new Gauge("key", "v" + i); + listGauge.add(sampleGauge); + } + Assert.assertEquals("List sizes should be as configured at creation time", listGauge.getValue().size(), 10); + + int valueIndex = 10; + Collection currentList = listGauge.getValue(); + Iterator iterator = currentList.iterator(); + while (iterator.hasNext()) { + Gauge gauge = (Gauge) iterator.next(); + Assert.assertTrue(gauge.getName().equals("key")); + Assert.assertTrue(gauge.getValue().equals("v" + valueIndex)); + valueIndex--; + } + } + + @Test + public void testThreadSafety() throws InterruptedException { + ListGauge listGauge = new ListGauge("listGauge", 20); + + Thread thread1 = new Thread(new Runnable() { + @Override + public void run() { + for (int i = 1; i <= 100; i++) { + listGauge.add(new Gauge("thread1", i)); + } + } + }); + + Thread thread2 = new Thread(new Runnable() { + @Override + public void run() { + for (int i = 1; i <= 100; i++) { + listGauge.add(new Gauge("key", i)); + } + } + }); + + thread1.start(); + thread2.start(); + + thread1.join(THREAD_TEST_TIMEOUT.toMillis()); + thread2.join(THREAD_TEST_TIMEOUT.toMillis()); + + Assert.assertTrue("ListGauge should have the last 20 values", listGauge.getValue().size() == 20); + for (Gauge gauge : listGauge.getValue()) { + Assert.assertTrue("Values should have the last 20 range", + ((Integer) gauge.getValue()) <= 100 && ((Integer) gauge.getValue()) > 80); + } + } +} diff --git a/samza-api/src/test/java/org/apache/samza/metrics/TestTimer.java b/samza-api/src/test/java/org/apache/samza/metrics/TestTimer.java index 8076e02d61..b945636137 100644 --- a/samza-api/src/test/java/org/apache/samza/metrics/TestTimer.java +++ b/samza-api/src/test/java/org/apache/samza/metrics/TestTimer.java @@ -62,7 +62,7 @@ public void testTimerWithDifferentWindowSize() { assertTrue(snapshot.getValues().containsAll(Arrays.asList(1L, 2L, 3L))); assertEquals(3, snapshot.getValues().size()); - // The time is 500 for update(4L) because getSnapshot calls clock once + 3 + // The time is 500 for update(4L) because getValue calls clock once + 3 // updates that call clock 3 times timer.update(4L); Snapshot snapshot2 = timer.getSnapshot(); diff --git a/samza-azure/src/test/java/org/apache/samza/system/eventhub/TestMetricsRegistry.java b/samza-azure/src/test/java/org/apache/samza/system/eventhub/TestMetricsRegistry.java index d29b975578..110b58212c 100644 --- a/samza-azure/src/test/java/org/apache/samza/system/eventhub/TestMetricsRegistry.java +++ b/samza-azure/src/test/java/org/apache/samza/system/eventhub/TestMetricsRegistry.java @@ -1,27 +1,28 @@ /* -* 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. -*/ + * 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.system.eventhub; import org.apache.commons.collections4.map.HashedMap; import org.apache.samza.metrics.Counter; import org.apache.samza.metrics.Gauge; +import org.apache.samza.metrics.ListGauge; import org.apache.samza.metrics.MetricsRegistry; import org.apache.samza.metrics.Timer; @@ -29,10 +30,12 @@ import java.util.List; import java.util.Map; + public class TestMetricsRegistry implements MetricsRegistry { private Map> counters = new HashedMap<>(); private Map>> gauges = new HashedMap<>(); + private Map> listGauges = new HashedMap<>(); public List getCounters(String groupName) { return counters.get(groupName); @@ -78,6 +81,14 @@ public Gauge newGauge(String group, Gauge value) { return value; } + @Override + public ListGauge newListGauge(String group, ListGauge listGauge) { + listGauges.putIfAbsent(group, new ArrayList()); + ListGauge value = new ListGauge(group, 1000); + listGauges.get(group).add(value); + return value; + } + @Override public Timer newTimer(String group, String name) { return null; diff --git a/samza-core/src/main/java/org/apache/samza/metrics/MetricGroup.java b/samza-core/src/main/java/org/apache/samza/metrics/MetricGroup.java index 53526d86c4..43d901f5b9 100644 --- a/samza-core/src/main/java/org/apache/samza/metrics/MetricGroup.java +++ b/samza-core/src/main/java/org/apache/samza/metrics/MetricGroup.java @@ -1,21 +1,21 @@ /* -* 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. -*/ + * 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; @@ -48,6 +48,10 @@ public Gauge newGauge(String name, T value) { return registry.newGauge(groupName, new Gauge((prefix + name).toLowerCase(), value)); } + public ListGauge newListGauge(String name, int nItems) { + return registry.newListGauge(groupName, new ListGauge(name, nItems)); + } + /* * Specify a dynamic gauge that always returns the latest value when polled. * The value closure/object must be thread safe, since metrics reporters may access diff --git a/samza-core/src/main/scala/org/apache/samza/container/SamzaContainer.scala b/samza-core/src/main/scala/org/apache/samza/container/SamzaContainer.scala index d46363986d..87a59176bf 100644 --- a/samza-core/src/main/scala/org/apache/samza/container/SamzaContainer.scala +++ b/samza-core/src/main/scala/org/apache/samza/container/SamzaContainer.scala @@ -43,7 +43,7 @@ import org.apache.samza.container.disk.{DiskQuotaPolicyFactory, DiskSpaceMonitor import org.apache.samza.container.host.{StatisticsMonitorImpl, SystemMemoryStatistics, SystemStatisticsMonitor} import org.apache.samza.coordinator.stream.{CoordinatorStreamManager, CoordinatorStreamSystemProducer} import org.apache.samza.job.model.JobModel -import org.apache.samza.metrics.{JmxServer, JvmMetrics, MetricsRegistryMap, MetricsReporter} +import org.apache.samza.metrics._ import org.apache.samza.serializers._ import org.apache.samza.serializers.model.SamzaObjectMapper import org.apache.samza.storage.{StorageEngineFactory, TaskStorageManager} @@ -752,7 +752,9 @@ class SamzaContainer( } status = SamzaContainerStatus.FAILED exceptionSeen = e - metrics.exceptionAtShutdown.set(ExceptionUtils.getStackTrace(e)) + + // Adding a shutdownException to the exception ListGauge in SamzaContainerMetrics + metrics.exception.add(new Gauge[String]("shutdownException", ExceptionUtils.getStackTrace(e))) debug("Updated value of exceptionAtShutdown to %s" format ExceptionUtils.getStackTrace(e)) } diff --git a/samza-core/src/main/scala/org/apache/samza/container/SamzaContainerMetrics.scala b/samza-core/src/main/scala/org/apache/samza/container/SamzaContainerMetrics.scala index c08433cabd..d6cefb9162 100644 --- a/samza-core/src/main/scala/org/apache/samza/container/SamzaContainerMetrics.scala +++ b/samza-core/src/main/scala/org/apache/samza/container/SamzaContainerMetrics.scala @@ -48,8 +48,8 @@ class SamzaContainerMetrics( val taskStoreRestorationMetrics: util.Map[TaskName, Gauge[Long]] = new util.HashMap[TaskName, Gauge[Long]]() - // A string-gauge metric to capture unhandled exceptions at this container that lead to a container shutdown - val exceptionAtShutdown = newGauge[String]("exception", "") + // A string-gauge metric to capture the last 1000 exceptions at this container + val exception = newListGauge[String]("exception", 1000) def addStoreRestorationGauge(taskName: TaskName, storeName: String) { taskStoreRestorationMetrics.put(taskName, newGauge("%s-%s-restore-time" format(taskName.toString, storeName), -1L)) diff --git a/samza-core/src/main/scala/org/apache/samza/metrics/MetricsHelper.scala b/samza-core/src/main/scala/org/apache/samza/metrics/MetricsHelper.scala index 1520b0e62d..4342cd9eb1 100644 --- a/samza-core/src/main/scala/org/apache/samza/metrics/MetricsHelper.scala +++ b/samza-core/src/main/scala/org/apache/samza/metrics/MetricsHelper.scala @@ -40,9 +40,11 @@ trait MetricsHelper { def newGauge[T](name: String, value: T) = metricGroup.newGauge[T](name,value) + def newListGauge[T](name: String, listSize: Int) = metricGroup.newListGauge[T](name, listSize) + /** - * Specify a dynamic gauge that always returns the latest value when polled. - * The value closure must be thread safe, since metrics reporters may access + * Specify a dynamic gauge that always returns the latest value when polled. + * The value closure must be thread safe, since metrics reporters may access * it from another thread. */ def newGauge[T](name: String, value: () => T) = { diff --git a/samza-core/src/main/scala/org/apache/samza/metrics/MetricsRegistryMap.scala b/samza-core/src/main/scala/org/apache/samza/metrics/MetricsRegistryMap.scala index 40ffee2bfa..aec0898ee9 100644 --- a/samza-core/src/main/scala/org/apache/samza/metrics/MetricsRegistryMap.scala +++ b/samza-core/src/main/scala/org/apache/samza/metrics/MetricsRegistryMap.scala @@ -23,9 +23,9 @@ import org.apache.samza.util.Logging import java.util.concurrent.ConcurrentHashMap /** - * A class that holds all metrics registered with it. It can be registered - * with one or more MetricReporters to flush metrics. - */ + * A class that holds all metrics registered with it. It can be registered + * with one or more MetricReporters to flush metrics. + */ class MetricsRegistryMap(val name: String) extends ReadableMetricsRegistry with Logging { var listeners = Set[ReadableMetricsRegistryListener]() @@ -37,7 +37,7 @@ class MetricsRegistryMap(val name: String) extends ReadableMetricsRegistry with def this() = this("unknown") def newCounter(group: String, counter: Counter) = { - debug("Add new counter %s %s %s." format (group, counter.getName, counter)) + debug("Add new counter %s %s %s." format(group, counter.getName, counter)) putAndGetGroup(group).putIfAbsent(counter.getName, counter) val realCounter = metrics.get(group).get(counter.getName).asInstanceOf[Counter] listeners.foreach(_.onCounter(group, realCounter)) @@ -45,25 +45,33 @@ class MetricsRegistryMap(val name: String) extends ReadableMetricsRegistry with } def newCounter(group: String, name: String) = { - debug("Creating new counter %s %s." format (group, name)) + debug("Creating new counter %s %s." format(group, name)) newCounter(group, new Counter(name)) } def newGauge[T](group: String, gauge: Gauge[T]) = { - debug("Adding new gauge %s %s %s." format (group, gauge.getName, gauge)) + debug("Adding new gauge %s %s %s." format(group, gauge.getName, gauge)) putAndGetGroup(group).putIfAbsent(gauge.getName, gauge) val realGauge = metrics.get(group).get(gauge.getName).asInstanceOf[Gauge[T]] listeners.foreach(_.onGauge(group, realGauge)) realGauge } + def newListGauge[T](group: String, listGauge: ListGauge) = { + debug("Adding new listgauge %s %s %s." format(group, listGauge.getName, listGauge)) + putAndGetGroup(group).putIfAbsent(listGauge.getName, listGauge) + val realListGauge = metrics.get(group).get(listGauge.getName).asInstanceOf[ListGauge] + listeners.foreach(_.onListGauge(group, realListGauge)) + realListGauge + } + def newGauge[T](group: String, name: String, value: T) = { - debug("Creating new gauge %s %s %s." format (group, name, value)) + debug("Creating new gauge %s %s %s." format(group, name, value)) newGauge(group, new Gauge[T](name, value)) } def newTimer(group: String, timer: Timer) = { - debug("Add new timer %s %s %s." format (group, timer.getName, timer)) + debug("Add new timer %s %s %s." format(group, timer.getName, timer)) putAndGetGroup(group).putIfAbsent(timer.getName, timer) val realTimer = metrics.get(group).get(timer.getName).asInstanceOf[Timer] listeners.foreach(_.onTimer(group, realTimer)) @@ -71,7 +79,7 @@ class MetricsRegistryMap(val name: String) extends ReadableMetricsRegistry with } def newTimer(group: String, name: String) = { - debug("Creating new timer %s %s." format (group, name)) + debug("Creating new timer %s %s." format(group, name)) newTimer(group, new Timer(name)) } @@ -95,4 +103,5 @@ class MetricsRegistryMap(val name: String) extends ReadableMetricsRegistry with def unlisten(listener: ReadableMetricsRegistryListener) { listeners -= listener } + } diff --git a/samza-core/src/main/scala/org/apache/samza/metrics/reporter/JmxReporter.scala b/samza-core/src/main/scala/org/apache/samza/metrics/reporter/JmxReporter.scala index 7da8a9c52a..797d5c5f71 100644 --- a/samza-core/src/main/scala/org/apache/samza/metrics/reporter/JmxReporter.scala +++ b/samza-core/src/main/scala/org/apache/samza/metrics/reporter/JmxReporter.scala @@ -20,19 +20,15 @@ package org.apache.samza.metrics.reporter import java.lang.management.ManagementFactory + import org.apache.samza.util.Logging import javax.management.MBeanServer import javax.management.ObjectName + import org.apache.samza.config.Config -import org.apache.samza.metrics.Counter -import org.apache.samza.metrics.Gauge -import org.apache.samza.metrics.Timer -import org.apache.samza.metrics.MetricsReporter -import org.apache.samza.metrics.MetricsReporterFactory -import org.apache.samza.metrics.ReadableMetricsRegistry -import org.apache.samza.metrics.ReadableMetricsRegistryListener +import org.apache.samza.metrics._ + import scala.collection.JavaConverters._ -import org.apache.samza.metrics.MetricsVisitor import org.apache.samza.metrics.JmxUtil._ class JmxReporter(server: MBeanServer) extends MetricsReporter with Logging { @@ -49,9 +45,14 @@ class JmxReporter(server: MBeanServer) extends MetricsReporter with Logging { registry.getGroup(group).asScala.foreach { case (name, metric) => metric.visit(new MetricsVisitor { + def counter(counter: Counter) = registerBean(new JmxCounter(counter, getObjectName(group, name, sources(registry)))) + def gauge[T](gauge: Gauge[T]) = registerBean(new JmxGauge(gauge.asInstanceOf[Gauge[Object]], getObjectName(group, name, sources(registry)))) + def timer(timer: Timer) = registerBean(new JmxTimer(timer, getObjectName(group, name, sources(registry)))) + + def listGauge[T](listGauge: ListGauge) = registerBean(new JmxListGauge(listGauge.asInstanceOf[ListGauge], getObjectName(group, listGauge.getName, sources(registry)))) }) } }) @@ -73,6 +74,10 @@ class JmxReporter(server: MBeanServer) extends MetricsReporter with Logging { def onTimer(group: String, timer: Timer) { registerBean(new JmxTimer(timer, getObjectName(group, timer.getName, source))) } + + def onListGauge(group: String, listGauge: ListGauge) { + registerBean(new JmxListGauge(listGauge, getObjectName(group, listGauge.getName, source))) + } } } else { warn("Trying to re-register a registry for source %s. Ignoring." format source) @@ -107,6 +112,13 @@ trait JmxGaugeMBean extends MetricMBean { class JmxGauge(g: org.apache.samza.metrics.Gauge[Object], on: ObjectName) extends JmxGaugeMBean { def getValue = g.getValue + + def objectName = on +} + +class JmxListGauge(g: org.apache.samza.metrics.ListGauge, on: ObjectName) extends JmxGaugeMBean { + def getValue = g.getValue + def objectName = on } @@ -116,6 +128,7 @@ trait JmxCounterMBean extends MetricMBean { class JmxCounter(c: org.apache.samza.metrics.Counter, on: ObjectName) extends JmxCounterMBean { def getCount() = c.getCount() + def objectName = on } @@ -125,6 +138,7 @@ trait JmxTimerMBean extends MetricMBean { class JmxTimer(t: org.apache.samza.metrics.Timer, on: ObjectName) extends JmxTimerMBean { def getAverageTime() = t.getSnapshot().getAverage() + def objectName = on } diff --git a/samza-core/src/main/scala/org/apache/samza/metrics/reporter/MetricsSnapshotReporter.scala b/samza-core/src/main/scala/org/apache/samza/metrics/reporter/MetricsSnapshotReporter.scala index 221832a4ba..48b6472fe4 100644 --- a/samza-core/src/main/scala/org/apache/samza/metrics/reporter/MetricsSnapshotReporter.scala +++ b/samza-core/src/main/scala/org/apache/samza/metrics/reporter/MetricsSnapshotReporter.scala @@ -19,6 +19,8 @@ package org.apache.samza.metrics.reporter +import java.util + import com.google.common.util.concurrent.ThreadFactoryBuilder import org.apache.samza.metrics._ import org.apache.samza.serializers.Serializer @@ -26,7 +28,6 @@ import org.apache.samza.system.OutgoingMessageEnvelope import org.apache.samza.system.SystemProducer import org.apache.samza.system.SystemStream import org.apache.samza.util.Logging - import java.util.HashMap import java.util.Map import java.util.concurrent.Executors @@ -115,6 +116,8 @@ class MetricsSnapshotReporter( registry.getGroup(group).asScala.foreach { case (name, metric) => metric.visit(new MetricsVisitor { + // for listGauge the value is returned as a list, which gets serialized + def listGauge[T](listGauge: ListGauge) = {groupMsg.put(name, listGauge.getValue) } def counter(counter: Counter) = groupMsg.put(name, counter.getCount: java.lang.Long) def gauge[T](gauge: Gauge[T]) = groupMsg.put(name, gauge.getValue.asInstanceOf[Object]) def timer(timer: Timer) = groupMsg.put(name, timer.getSnapshot().getAverage(): java.lang.Double) @@ -147,6 +150,7 @@ class MetricsSnapshotReporter( } } + debug("Finished flushing metrics.") } } diff --git a/samza-yarn/src/main/scala/org/apache/samza/webapp/ApplicationMasterRestServlet.scala b/samza-yarn/src/main/scala/org/apache/samza/webapp/ApplicationMasterRestServlet.scala index 122a1dfaee..c4e72f1cdb 100644 --- a/samza-yarn/src/main/scala/org/apache/samza/webapp/ApplicationMasterRestServlet.scala +++ b/samza-yarn/src/main/scala/org/apache/samza/webapp/ApplicationMasterRestServlet.scala @@ -49,6 +49,9 @@ class ApplicationMasterRestServlet(samzaConfig: Config, samzaAppState: SamzaAppl registry.getGroup(group).asScala.foreach { case (name, metric) => metric.visit(new MetricsVisitor() { + def listGauge[T](listGauge: ListGauge) = + groupMap.put(name, listGauge.getValue) + def counter(counter: Counter) = groupMap.put(counter.getName, counter.getCount: java.lang.Long) From 75ef5d20aedf7f2dd0972ff9ddcc3101e28b095d Mon Sep 17 00:00:00 2001 From: Ray Matharu Date: Thu, 31 May 2018 15:33:07 -0700 Subject: [PATCH 03/33] Typifying ListGauge to ListGauge, adding an eviction policy, default policy to retain last N --- .../org/apache/samza/metrics/ListGauge.java | 71 +++++++++++-------- .../metrics/ListGaugeEvictionPolicy.java | 23 ++++++ .../apache/samza/metrics/MetricsRegistry.java | 2 +- .../apache/samza/metrics/MetricsVisitor.java | 6 +- .../ReadableMetricsRegistryListener.java | 2 +- .../samza/metrics/RetainLastNPolicy.java | 47 ++++++++++++ .../samza/util/NoOpMetricsRegistry.java | 2 +- .../apache/samza/metrics/TestListGauge.java | 34 +++++---- .../system/eventhub/TestMetricsRegistry.java | 2 +- .../org/apache/samza/metrics/MetricGroup.java | 4 +- .../samza/container/SamzaContainer.scala | 4 -- .../container/SamzaContainerMetrics.scala | 6 +- .../apache/samza/metrics/MetricsHelper.scala | 2 +- .../samza/metrics/MetricsRegistryMap.scala | 12 +++- .../samza/metrics/reporter/JmxReporter.scala | 9 +-- .../reporter/MetricsSnapshotReporter.scala | 2 +- .../webapp/ApplicationMasterRestServlet.scala | 2 +- 17 files changed, 158 insertions(+), 72 deletions(-) create mode 100644 samza-api/src/main/java/org/apache/samza/metrics/ListGaugeEvictionPolicy.java create mode 100644 samza-api/src/main/java/org/apache/samza/metrics/RetainLastNPolicy.java diff --git a/samza-api/src/main/java/org/apache/samza/metrics/ListGauge.java b/samza-api/src/main/java/org/apache/samza/metrics/ListGauge.java index 744acbdfc7..a7541b3fc4 100644 --- a/samza-api/src/main/java/org/apache/samza/metrics/ListGauge.java +++ b/samza-api/src/main/java/org/apache/samza/metrics/ListGauge.java @@ -18,42 +18,43 @@ */ package org.apache.samza.metrics; +import java.util.ArrayList; import java.util.Collection; +import java.util.Collections; import java.util.LinkedList; +import java.util.List; /** - * A ListGauge is a collection of {@link org.apache.samza.metrics.Gauge}. ListGauges are useful for maintaining, recording, - * or collecting specific Gauge values over time. It is implemented as a {@link org.apache.samza.metrics.Metric}. - * For example, the set of recent errors that have occurred. + * A {@link ListGauge} is a {@link org.apache.samza.metrics.Metric} that buffers multiple instances of a type T in a list. + * {@link ListGauge}s are useful for maintaining, recording, or collecting values over time. + * For example, a set of specific logging-events (e.g., errors). * - * This current implementation uses a size-bound-policy and holds the N most-recent Gauge objects added to the list. - * This bound N is configurable at instantiation time. - * TODO: Support a time-based and size-and-time-based hybrid policy. - * TODO: Add a derived class to do compaction for errors using hash-based errorIDs and adding timestamp for errors to dedup - * on the read path. + * Eviction from list is either done by consuming-code using the remove APIs or by specifying an eviction policy + * at creation time. * * All public methods are thread-safe. * */ -public class ListGauge implements Metric { +public class ListGauge implements Metric { private final String name; - private final Collection metricList; - private int nItems; + private final List metricList; + private ListGaugeEvictionPolicy listGaugeEvictionPolicy; + + private final static int DEFAULT_POLICY_NUM_RETAIN = 60; /** - * Create a new ListGauge. + * Create a new {@link ListGauge} with no auto eviction, callers can add/remove items as desired. * @param name Name to be assigned - * @param nItems The number of items to hold in the list */ - public ListGauge(String name, int nItems) { + public ListGauge(String name) { this.name = name; - this.metricList = new LinkedList<>(); - this.nItems = nItems; + this.metricList = new ArrayList(DEFAULT_POLICY_NUM_RETAIN); + this.listGaugeEvictionPolicy = new RetainLastNPolicy(this, DEFAULT_POLICY_NUM_RETAIN); } /** - * Get the name assigned to the ListGauge + * Get the name assigned to this {@link ListGauge} * @return the assigned name */ public String getName() { @@ -61,23 +62,37 @@ public String getName() { } /** - * Add a gauge to the list - * @param value The Gauge value to be added + * Get the Collection of Gauge values currently in the list, used when serializing this Gauge. + * @return the collection of gauge values */ - public synchronized void add(Gauge value) { - if (this.metricList.size() == nItems) { - ((LinkedList) this.metricList).removeFirst(); - } + public synchronized Collection getValue() { + return Collections.unmodifiableList(this.metricList); + } - this.metricList.add(value); + /** + * Package-private method to change the eviction policy + * @param listGaugeEvictionPolicy + */ + synchronized void setEvictionPolicy(ListGaugeEvictionPolicy listGaugeEvictionPolicy) { + this.listGaugeEvictionPolicy = listGaugeEvictionPolicy; } /** - * Get the Collection of Gauge values currently in the list - * @return the collection of gauge values + * Add a gauge to the list + * @param value The Gauge value to be added */ - public synchronized Collection getValue() { - return this.metricList; + public synchronized void add(T value) { + this.metricList.add(value); + + // notify the policy object (if one is present), for performing any eviction that may be needed. + // note: monitor is being held + if (this.listGaugeEvictionPolicy != null) { + this.listGaugeEvictionPolicy.elementAddedCallback(); + } + } + + public synchronized boolean remove(T value) { + return this.metricList.remove(value); } /** diff --git a/samza-api/src/main/java/org/apache/samza/metrics/ListGaugeEvictionPolicy.java b/samza-api/src/main/java/org/apache/samza/metrics/ListGaugeEvictionPolicy.java new file mode 100644 index 0000000000..966a54a10a --- /dev/null +++ b/samza-api/src/main/java/org/apache/samza/metrics/ListGaugeEvictionPolicy.java @@ -0,0 +1,23 @@ +/* + * 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; + +public interface ListGaugeEvictionPolicy { + void elementAddedCallback(); +} diff --git a/samza-api/src/main/java/org/apache/samza/metrics/MetricsRegistry.java b/samza-api/src/main/java/org/apache/samza/metrics/MetricsRegistry.java index 8f2cfb4870..fa0fd3984a 100644 --- a/samza-api/src/main/java/org/apache/samza/metrics/MetricsRegistry.java +++ b/samza-api/src/main/java/org/apache/samza/metrics/MetricsRegistry.java @@ -71,7 +71,7 @@ public interface MetricsRegistry { * @param Type of the ListGauge * @return ListGauge registered */ - ListGauge newListGauge(String group, ListGauge listGauge); + ListGauge newListGauge(String group, ListGauge listGauge); /** * Create and Register a new {@link org.apache.samza.metrics.Timer} diff --git a/samza-api/src/main/java/org/apache/samza/metrics/MetricsVisitor.java b/samza-api/src/main/java/org/apache/samza/metrics/MetricsVisitor.java index 354a9ae54d..49a0929515 100644 --- a/samza-api/src/main/java/org/apache/samza/metrics/MetricsVisitor.java +++ b/samza-api/src/main/java/org/apache/samza/metrics/MetricsVisitor.java @@ -31,12 +31,12 @@ public abstract class MetricsVisitor { public abstract void timer(Timer timer); - public abstract void listGauge(ListGauge listGauge); + public abstract void listGauge(ListGauge listGauge); public void visit(Metric metric) { // Cast for metrics of type ListGauge - if (metric instanceof ListGauge) { - listGauge((ListGauge) metric); + if (metric instanceof ListGauge) { + listGauge((ListGauge) metric); } else if (metric instanceof Counter) { counter((Counter) metric); } else if (metric instanceof Gauge) { diff --git a/samza-api/src/main/java/org/apache/samza/metrics/ReadableMetricsRegistryListener.java b/samza-api/src/main/java/org/apache/samza/metrics/ReadableMetricsRegistryListener.java index 84225f5baf..ba5b182828 100644 --- a/samza-api/src/main/java/org/apache/samza/metrics/ReadableMetricsRegistryListener.java +++ b/samza-api/src/main/java/org/apache/samza/metrics/ReadableMetricsRegistryListener.java @@ -24,7 +24,7 @@ public interface ReadableMetricsRegistryListener { void onGauge(String group, Gauge gauge); - void onListGauge(String group, ListGauge listGauge); + void onListGauge(String group, ListGauge listGauge); void onTimer(String group, Timer timer); } diff --git a/samza-api/src/main/java/org/apache/samza/metrics/RetainLastNPolicy.java b/samza-api/src/main/java/org/apache/samza/metrics/RetainLastNPolicy.java new file mode 100644 index 0000000000..03be581039 --- /dev/null +++ b/samza-api/src/main/java/org/apache/samza/metrics/RetainLastNPolicy.java @@ -0,0 +1,47 @@ +/* + * 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; + +import java.util.Collection; +import java.util.Iterator; + + +public class RetainLastNPolicy implements ListGaugeEvictionPolicy { + + private final ListGauge listGauge; + private final int nItems; + + public RetainLastNPolicy(ListGauge listGauge, int numItems) { + this.listGauge = listGauge; + this.nItems = numItems; + } + + @Override + public void elementAddedCallback() { + // get a snapshot of the list + Collection listGaugeCollection = this.listGauge.getValue(); + int numToEvict = listGaugeCollection.size() - nItems; + Iterator iterator = listGaugeCollection.iterator(); + while (numToEvict > 0 && iterator.hasNext()) { + // Remove in FIFO order to retain the last nItems + listGauge.remove(iterator.next()); + numToEvict--; + } + } +} diff --git a/samza-api/src/main/java/org/apache/samza/util/NoOpMetricsRegistry.java b/samza-api/src/main/java/org/apache/samza/util/NoOpMetricsRegistry.java index 73653ccfca..76b82164dc 100644 --- a/samza-api/src/main/java/org/apache/samza/util/NoOpMetricsRegistry.java +++ b/samza-api/src/main/java/org/apache/samza/util/NoOpMetricsRegistry.java @@ -52,7 +52,7 @@ public Gauge newGauge(String group, Gauge gauge) { } @Override - public ListGauge newListGauge(String group, ListGauge listGauge) { + public ListGauge newListGauge(String group, ListGauge listGauge) { return listGauge; } diff --git a/samza-api/src/test/java/org/apache/samza/metrics/TestListGauge.java b/samza-api/src/test/java/org/apache/samza/metrics/TestListGauge.java index 1747006f47..5c19440b3d 100644 --- a/samza-api/src/test/java/org/apache/samza/metrics/TestListGauge.java +++ b/samza-api/src/test/java/org/apache/samza/metrics/TestListGauge.java @@ -34,43 +34,42 @@ public class TestListGauge { @Test public void basicTest() { - ListGauge listGauge = new ListGauge("listGauge", 10); - Gauge sampleGauge = new Gauge("key", "value"); - listGauge.add(sampleGauge); - Assert.assertEquals("Names should be the same", listGauge.getName(), "listGauge"); + ListGauge listGauge = new ListGauge("sampleListGauge"); + listGauge.add("sampleValue"); + Assert.assertEquals("Names should be the same", listGauge.getName(), "sampleListGauge"); Assert.assertEquals("List sizes should match", listGauge.getValue().size(), 1); - Assert.assertEquals("ListGauge should contain sampleGauge", listGauge.getValue().contains(sampleGauge), true); + Assert.assertEquals("ListGauge should contain sampleGauge", listGauge.getValue().contains("sampleValue"), true); } @Test public void testSizeEnforcement() { - ListGauge listGauge = new ListGauge("listGauge", 10); + ListGauge listGauge = new ListGauge("listGauge"); + listGauge.setEvictionPolicy(new RetainLastNPolicy(listGauge, 10)); for (int i = 15; i > 0; i--) { - Gauge sampleGauge = new Gauge("key", "v" + i); - listGauge.add(sampleGauge); + listGauge.add("v" + i); } Assert.assertEquals("List sizes should be as configured at creation time", listGauge.getValue().size(), 10); int valueIndex = 10; - Collection currentList = listGauge.getValue(); + Collection currentList = listGauge.getValue(); Iterator iterator = currentList.iterator(); while (iterator.hasNext()) { - Gauge gauge = (Gauge) iterator.next(); - Assert.assertTrue(gauge.getName().equals("key")); - Assert.assertTrue(gauge.getValue().equals("v" + valueIndex)); + String gaugeValue = (String) iterator.next(); + Assert.assertTrue(gaugeValue.equals("v" + valueIndex)); valueIndex--; } } @Test public void testThreadSafety() throws InterruptedException { - ListGauge listGauge = new ListGauge("listGauge", 20); + ListGauge listGauge = new ListGauge("listGauge"); + listGauge.setEvictionPolicy(new RetainLastNPolicy(listGauge, 20)); Thread thread1 = new Thread(new Runnable() { @Override public void run() { for (int i = 1; i <= 100; i++) { - listGauge.add(new Gauge("thread1", i)); + listGauge.add(i); } } }); @@ -79,7 +78,7 @@ public void run() { @Override public void run() { for (int i = 1; i <= 100; i++) { - listGauge.add(new Gauge("key", i)); + listGauge.add(i); } } }); @@ -91,9 +90,8 @@ public void run() { thread2.join(THREAD_TEST_TIMEOUT.toMillis()); Assert.assertTrue("ListGauge should have the last 20 values", listGauge.getValue().size() == 20); - for (Gauge gauge : listGauge.getValue()) { - Assert.assertTrue("Values should have the last 20 range", - ((Integer) gauge.getValue()) <= 100 && ((Integer) gauge.getValue()) > 80); + for (Integer gaugeValue : listGauge.getValue()) { + Assert.assertTrue("Values should have the last 20 range", gaugeValue <= 100 && gaugeValue > 80); } } } diff --git a/samza-azure/src/test/java/org/apache/samza/system/eventhub/TestMetricsRegistry.java b/samza-azure/src/test/java/org/apache/samza/system/eventhub/TestMetricsRegistry.java index 110b58212c..312ad2e06a 100644 --- a/samza-azure/src/test/java/org/apache/samza/system/eventhub/TestMetricsRegistry.java +++ b/samza-azure/src/test/java/org/apache/samza/system/eventhub/TestMetricsRegistry.java @@ -84,7 +84,7 @@ public Gauge newGauge(String group, Gauge value) { @Override public ListGauge newListGauge(String group, ListGauge listGauge) { listGauges.putIfAbsent(group, new ArrayList()); - ListGauge value = new ListGauge(group, 1000); + ListGauge value = new ListGauge(group); listGauges.get(group).add(value); return value; } diff --git a/samza-core/src/main/java/org/apache/samza/metrics/MetricGroup.java b/samza-core/src/main/java/org/apache/samza/metrics/MetricGroup.java index 43d901f5b9..4dfbbc9acf 100644 --- a/samza-core/src/main/java/org/apache/samza/metrics/MetricGroup.java +++ b/samza-core/src/main/java/org/apache/samza/metrics/MetricGroup.java @@ -48,8 +48,8 @@ public Gauge newGauge(String name, T value) { return registry.newGauge(groupName, new Gauge((prefix + name).toLowerCase(), value)); } - public ListGauge newListGauge(String name, int nItems) { - return registry.newListGauge(groupName, new ListGauge(name, nItems)); + public ListGauge newListGauge(String name) { + return registry.newListGauge(groupName, new ListGauge(name)); } /* diff --git a/samza-core/src/main/scala/org/apache/samza/container/SamzaContainer.scala b/samza-core/src/main/scala/org/apache/samza/container/SamzaContainer.scala index 87a59176bf..ddc99ff6da 100644 --- a/samza-core/src/main/scala/org/apache/samza/container/SamzaContainer.scala +++ b/samza-core/src/main/scala/org/apache/samza/container/SamzaContainer.scala @@ -752,10 +752,6 @@ class SamzaContainer( } status = SamzaContainerStatus.FAILED exceptionSeen = e - - // Adding a shutdownException to the exception ListGauge in SamzaContainerMetrics - metrics.exception.add(new Gauge[String]("shutdownException", ExceptionUtils.getStackTrace(e))) - debug("Updated value of exceptionAtShutdown to %s" format ExceptionUtils.getStackTrace(e)) } try { diff --git a/samza-core/src/main/scala/org/apache/samza/container/SamzaContainerMetrics.scala b/samza-core/src/main/scala/org/apache/samza/container/SamzaContainerMetrics.scala index d6cefb9162..ae7776b996 100644 --- a/samza-core/src/main/scala/org/apache/samza/container/SamzaContainerMetrics.scala +++ b/samza-core/src/main/scala/org/apache/samza/container/SamzaContainerMetrics.scala @@ -21,7 +21,7 @@ package org.apache.samza.container import java.util -import org.apache.samza.metrics.{Gauge, ReadableMetricsRegistry, MetricsRegistryMap, MetricsHelper} +import org.apache.samza.metrics._ class SamzaContainerMetrics( val source: String = "unknown", @@ -48,8 +48,8 @@ class SamzaContainerMetrics( val taskStoreRestorationMetrics: util.Map[TaskName, Gauge[Long]] = new util.HashMap[TaskName, Gauge[Long]]() - // A string-gauge metric to capture the last 1000 exceptions at this container - val exception = newListGauge[String]("exception", 1000) + // A string-gauge metric to capture exceptions at this container + val exception = newListGauge[String]("exception") def addStoreRestorationGauge(taskName: TaskName, storeName: String) { taskStoreRestorationMetrics.put(taskName, newGauge("%s-%s-restore-time" format(taskName.toString, storeName), -1L)) diff --git a/samza-core/src/main/scala/org/apache/samza/metrics/MetricsHelper.scala b/samza-core/src/main/scala/org/apache/samza/metrics/MetricsHelper.scala index 4342cd9eb1..21ec7633f4 100644 --- a/samza-core/src/main/scala/org/apache/samza/metrics/MetricsHelper.scala +++ b/samza-core/src/main/scala/org/apache/samza/metrics/MetricsHelper.scala @@ -40,7 +40,7 @@ trait MetricsHelper { def newGauge[T](name: String, value: T) = metricGroup.newGauge[T](name,value) - def newListGauge[T](name: String, listSize: Int) = metricGroup.newListGauge[T](name, listSize) + def newListGauge[T](name: String) = metricGroup.newListGauge[T](name) /** * Specify a dynamic gauge that always returns the latest value when polled. diff --git a/samza-core/src/main/scala/org/apache/samza/metrics/MetricsRegistryMap.scala b/samza-core/src/main/scala/org/apache/samza/metrics/MetricsRegistryMap.scala index aec0898ee9..d12c53ca2c 100644 --- a/samza-core/src/main/scala/org/apache/samza/metrics/MetricsRegistryMap.scala +++ b/samza-core/src/main/scala/org/apache/samza/metrics/MetricsRegistryMap.scala @@ -57,10 +57,17 @@ class MetricsRegistryMap(val name: String) extends ReadableMetricsRegistry with realGauge } - def newListGauge[T](group: String, listGauge: ListGauge) = { + /** + * Register a {@link org.apache.samza.metrics.ListGauge} + * + * @param group Group for this ListGauge + * @param listGauge the ListGauge to register + * @tparam T the type of the list gauge + */ + def newListGauge[T](group: String, listGauge: ListGauge[T]) = { debug("Adding new listgauge %s %s %s." format(group, listGauge.getName, listGauge)) putAndGetGroup(group).putIfAbsent(listGauge.getName, listGauge) - val realListGauge = metrics.get(group).get(listGauge.getName).asInstanceOf[ListGauge] + val realListGauge = metrics.get(group).get(listGauge.getName).asInstanceOf[ListGauge[T]] listeners.foreach(_.onListGauge(group, realListGauge)) realListGauge } @@ -103,5 +110,4 @@ class MetricsRegistryMap(val name: String) extends ReadableMetricsRegistry with def unlisten(listener: ReadableMetricsRegistryListener) { listeners -= listener } - } diff --git a/samza-core/src/main/scala/org/apache/samza/metrics/reporter/JmxReporter.scala b/samza-core/src/main/scala/org/apache/samza/metrics/reporter/JmxReporter.scala index 797d5c5f71..32718ced2c 100644 --- a/samza-core/src/main/scala/org/apache/samza/metrics/reporter/JmxReporter.scala +++ b/samza-core/src/main/scala/org/apache/samza/metrics/reporter/JmxReporter.scala @@ -52,7 +52,8 @@ class JmxReporter(server: MBeanServer) extends MetricsReporter with Logging { def timer(timer: Timer) = registerBean(new JmxTimer(timer, getObjectName(group, name, sources(registry)))) - def listGauge[T](listGauge: ListGauge) = registerBean(new JmxListGauge(listGauge.asInstanceOf[ListGauge], getObjectName(group, listGauge.getName, sources(registry)))) + def listGauge[T](listGauge: ListGauge[T]) = registerBean(new JmxListGauge(listGauge.asInstanceOf[ListGauge[Object]], getObjectName(group, name, sources(registry)))) + }) } }) @@ -75,8 +76,8 @@ class JmxReporter(server: MBeanServer) extends MetricsReporter with Logging { registerBean(new JmxTimer(timer, getObjectName(group, timer.getName, source))) } - def onListGauge(group: String, listGauge: ListGauge) { - registerBean(new JmxListGauge(listGauge, getObjectName(group, listGauge.getName, source))) + def onListGauge(group: String, listGauge: ListGauge[_]) { + registerBean(new JmxListGauge(listGauge.asInstanceOf[ListGauge[Object]], getObjectName(group, listGauge.getName, source))) } } } else { @@ -116,7 +117,7 @@ class JmxGauge(g: org.apache.samza.metrics.Gauge[Object], on: ObjectName) extend def objectName = on } -class JmxListGauge(g: org.apache.samza.metrics.ListGauge, on: ObjectName) extends JmxGaugeMBean { +class JmxListGauge(g: org.apache.samza.metrics.ListGauge[Object], on: ObjectName) extends JmxGaugeMBean { def getValue = g.getValue def objectName = on diff --git a/samza-core/src/main/scala/org/apache/samza/metrics/reporter/MetricsSnapshotReporter.scala b/samza-core/src/main/scala/org/apache/samza/metrics/reporter/MetricsSnapshotReporter.scala index 48b6472fe4..b3e0eb7f8d 100644 --- a/samza-core/src/main/scala/org/apache/samza/metrics/reporter/MetricsSnapshotReporter.scala +++ b/samza-core/src/main/scala/org/apache/samza/metrics/reporter/MetricsSnapshotReporter.scala @@ -117,7 +117,7 @@ class MetricsSnapshotReporter( case (name, metric) => metric.visit(new MetricsVisitor { // for listGauge the value is returned as a list, which gets serialized - def listGauge[T](listGauge: ListGauge) = {groupMsg.put(name, listGauge.getValue) } + def listGauge[T](listGauge: ListGauge[T]) = {groupMsg.put(name, listGauge.getValue) } def counter(counter: Counter) = groupMsg.put(name, counter.getCount: java.lang.Long) def gauge[T](gauge: Gauge[T]) = groupMsg.put(name, gauge.getValue.asInstanceOf[Object]) def timer(timer: Timer) = groupMsg.put(name, timer.getSnapshot().getAverage(): java.lang.Double) diff --git a/samza-yarn/src/main/scala/org/apache/samza/webapp/ApplicationMasterRestServlet.scala b/samza-yarn/src/main/scala/org/apache/samza/webapp/ApplicationMasterRestServlet.scala index c4e72f1cdb..671a44910c 100644 --- a/samza-yarn/src/main/scala/org/apache/samza/webapp/ApplicationMasterRestServlet.scala +++ b/samza-yarn/src/main/scala/org/apache/samza/webapp/ApplicationMasterRestServlet.scala @@ -49,7 +49,7 @@ class ApplicationMasterRestServlet(samzaConfig: Config, samzaAppState: SamzaAppl registry.getGroup(group).asScala.foreach { case (name, metric) => metric.visit(new MetricsVisitor() { - def listGauge[T](listGauge: ListGauge) = + def listGauge[T](listGauge: ListGauge[T]) = groupMap.put(name, listGauge.getValue) def counter(counter: Counter) = From 1efed5a861fe2079827d81781041d1bebb52374d Mon Sep 17 00:00:00 2001 From: Ray Matharu Date: Thu, 10 May 2018 16:31:07 -0700 Subject: [PATCH 04/33] Adding metric type ListGauge Added exception metric (as string-guage), to be emitted using the snapshot reporter. Switched producer-shutdown logic to ensure metric-flush at shutdown-time Adding ListGauge and integration with SamzaContainerMetric Typifying ListGauge to ListGauge, adding an eviction policy, default policy to retain last N --- .../org/apache/samza/metrics/ListGauge.java | 105 ++++++++++++++++++ .../metrics/ListGaugeEvictionPolicy.java | 23 ++++ .../apache/samza/metrics/MetricsRegistry.java | 9 ++ .../apache/samza/metrics/MetricsVisitor.java | 7 +- .../ReadableMetricsRegistryListener.java | 2 + .../samza/metrics/RetainLastNPolicy.java | 47 ++++++++ .../samza/util/NoOpMetricsRegistry.java | 7 ++ .../apache/samza/metrics/TestListGauge.java | 97 ++++++++++++++++ .../org/apache/samza/metrics/TestTimer.java | 2 +- .../system/eventhub/TestMetricsRegistry.java | 45 +++++--- .../org/apache/samza/metrics/MetricGroup.java | 38 ++++--- .../samza/container/SamzaContainer.scala | 3 +- .../container/SamzaContainerMetrics.scala | 5 +- .../apache/samza/metrics/MetricsHelper.scala | 6 +- .../samza/metrics/MetricsRegistryMap.scala | 33 ++++-- .../samza/metrics/reporter/JmxReporter.scala | 31 ++++-- .../reporter/MetricsSnapshotReporter.scala | 26 +++-- .../webapp/ApplicationMasterRestServlet.scala | 3 + 18 files changed, 425 insertions(+), 64 deletions(-) create mode 100644 samza-api/src/main/java/org/apache/samza/metrics/ListGauge.java create mode 100644 samza-api/src/main/java/org/apache/samza/metrics/ListGaugeEvictionPolicy.java create mode 100644 samza-api/src/main/java/org/apache/samza/metrics/RetainLastNPolicy.java create mode 100644 samza-api/src/test/java/org/apache/samza/metrics/TestListGauge.java diff --git a/samza-api/src/main/java/org/apache/samza/metrics/ListGauge.java b/samza-api/src/main/java/org/apache/samza/metrics/ListGauge.java new file mode 100644 index 0000000000..a7541b3fc4 --- /dev/null +++ b/samza-api/src/main/java/org/apache/samza/metrics/ListGauge.java @@ -0,0 +1,105 @@ +/* + * 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; + +import java.util.ArrayList; +import java.util.Collection; +import java.util.Collections; +import java.util.LinkedList; +import java.util.List; + + +/** + * A {@link ListGauge} is a {@link org.apache.samza.metrics.Metric} that buffers multiple instances of a type T in a list. + * {@link ListGauge}s are useful for maintaining, recording, or collecting values over time. + * For example, a set of specific logging-events (e.g., errors). + * + * Eviction from list is either done by consuming-code using the remove APIs or by specifying an eviction policy + * at creation time. + * + * All public methods are thread-safe. + * + */ +public class ListGauge implements Metric { + private final String name; + private final List metricList; + private ListGaugeEvictionPolicy listGaugeEvictionPolicy; + + private final static int DEFAULT_POLICY_NUM_RETAIN = 60; + + /** + * Create a new {@link ListGauge} with no auto eviction, callers can add/remove items as desired. + * @param name Name to be assigned + */ + public ListGauge(String name) { + this.name = name; + this.metricList = new ArrayList(DEFAULT_POLICY_NUM_RETAIN); + this.listGaugeEvictionPolicy = new RetainLastNPolicy(this, DEFAULT_POLICY_NUM_RETAIN); + } + + /** + * Get the name assigned to this {@link ListGauge} + * @return the assigned name + */ + public String getName() { + return this.name; + } + + /** + * Get the Collection of Gauge values currently in the list, used when serializing this Gauge. + * @return the collection of gauge values + */ + public synchronized Collection getValue() { + return Collections.unmodifiableList(this.metricList); + } + + /** + * Package-private method to change the eviction policy + * @param listGaugeEvictionPolicy + */ + synchronized void setEvictionPolicy(ListGaugeEvictionPolicy listGaugeEvictionPolicy) { + this.listGaugeEvictionPolicy = listGaugeEvictionPolicy; + } + + /** + * Add a gauge to the list + * @param value The Gauge value to be added + */ + public synchronized void add(T value) { + this.metricList.add(value); + + // notify the policy object (if one is present), for performing any eviction that may be needed. + // note: monitor is being held + if (this.listGaugeEvictionPolicy != null) { + this.listGaugeEvictionPolicy.elementAddedCallback(); + } + } + + public synchronized boolean remove(T value) { + return this.metricList.remove(value); + } + + /** + * {@inheritDoc} + */ + @Override + public synchronized void visit(MetricsVisitor visitor) { + visitor.listGauge(this); + } +} diff --git a/samza-api/src/main/java/org/apache/samza/metrics/ListGaugeEvictionPolicy.java b/samza-api/src/main/java/org/apache/samza/metrics/ListGaugeEvictionPolicy.java new file mode 100644 index 0000000000..966a54a10a --- /dev/null +++ b/samza-api/src/main/java/org/apache/samza/metrics/ListGaugeEvictionPolicy.java @@ -0,0 +1,23 @@ +/* + * 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; + +public interface ListGaugeEvictionPolicy { + void elementAddedCallback(); +} diff --git a/samza-api/src/main/java/org/apache/samza/metrics/MetricsRegistry.java b/samza-api/src/main/java/org/apache/samza/metrics/MetricsRegistry.java index 5a00d01868..fa0fd3984a 100644 --- a/samza-api/src/main/java/org/apache/samza/metrics/MetricsRegistry.java +++ b/samza-api/src/main/java/org/apache/samza/metrics/MetricsRegistry.java @@ -64,6 +64,15 @@ public interface MetricsRegistry { */ Gauge newGauge(String group, Gauge value); + /** + * Register a {@link org.apache.samza.metrics.ListGauge} + * @param group Group for this ListGauge + * @param listGauge the ListGauge to register + * @param Type of the ListGauge + * @return ListGauge registered + */ + ListGauge newListGauge(String group, ListGauge listGauge); + /** * Create and Register a new {@link org.apache.samza.metrics.Timer} * @param group Group for this Timer diff --git a/samza-api/src/main/java/org/apache/samza/metrics/MetricsVisitor.java b/samza-api/src/main/java/org/apache/samza/metrics/MetricsVisitor.java index 75abfe7078..49a0929515 100644 --- a/samza-api/src/main/java/org/apache/samza/metrics/MetricsVisitor.java +++ b/samza-api/src/main/java/org/apache/samza/metrics/MetricsVisitor.java @@ -31,8 +31,13 @@ public abstract class MetricsVisitor { public abstract void timer(Timer timer); + public abstract void listGauge(ListGauge listGauge); + public void visit(Metric metric) { - if (metric instanceof Counter) { + // Cast for metrics of type ListGauge + if (metric instanceof ListGauge) { + listGauge((ListGauge) metric); + } else if (metric instanceof Counter) { counter((Counter) metric); } else if (metric instanceof Gauge) { gauge((Gauge) metric); diff --git a/samza-api/src/main/java/org/apache/samza/metrics/ReadableMetricsRegistryListener.java b/samza-api/src/main/java/org/apache/samza/metrics/ReadableMetricsRegistryListener.java index 739d68f8ef..ba5b182828 100644 --- a/samza-api/src/main/java/org/apache/samza/metrics/ReadableMetricsRegistryListener.java +++ b/samza-api/src/main/java/org/apache/samza/metrics/ReadableMetricsRegistryListener.java @@ -24,5 +24,7 @@ public interface ReadableMetricsRegistryListener { void onGauge(String group, Gauge gauge); + void onListGauge(String group, ListGauge listGauge); + void onTimer(String group, Timer timer); } diff --git a/samza-api/src/main/java/org/apache/samza/metrics/RetainLastNPolicy.java b/samza-api/src/main/java/org/apache/samza/metrics/RetainLastNPolicy.java new file mode 100644 index 0000000000..03be581039 --- /dev/null +++ b/samza-api/src/main/java/org/apache/samza/metrics/RetainLastNPolicy.java @@ -0,0 +1,47 @@ +/* + * 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; + +import java.util.Collection; +import java.util.Iterator; + + +public class RetainLastNPolicy implements ListGaugeEvictionPolicy { + + private final ListGauge listGauge; + private final int nItems; + + public RetainLastNPolicy(ListGauge listGauge, int numItems) { + this.listGauge = listGauge; + this.nItems = numItems; + } + + @Override + public void elementAddedCallback() { + // get a snapshot of the list + Collection listGaugeCollection = this.listGauge.getValue(); + int numToEvict = listGaugeCollection.size() - nItems; + Iterator iterator = listGaugeCollection.iterator(); + while (numToEvict > 0 && iterator.hasNext()) { + // Remove in FIFO order to retain the last nItems + listGauge.remove(iterator.next()); + numToEvict--; + } + } +} diff --git a/samza-api/src/main/java/org/apache/samza/util/NoOpMetricsRegistry.java b/samza-api/src/main/java/org/apache/samza/util/NoOpMetricsRegistry.java index 3df855c2f7..76b82164dc 100644 --- a/samza-api/src/main/java/org/apache/samza/util/NoOpMetricsRegistry.java +++ b/samza-api/src/main/java/org/apache/samza/util/NoOpMetricsRegistry.java @@ -21,9 +21,11 @@ import org.apache.samza.metrics.Counter; import org.apache.samza.metrics.Gauge; +import org.apache.samza.metrics.ListGauge; import org.apache.samza.metrics.MetricsRegistry; import org.apache.samza.metrics.Timer; + /** * {@link org.apache.samza.metrics.MetricsRegistry} implementation for when no actual metrics need to be * recorded but a registry is still required. @@ -49,6 +51,11 @@ public Gauge newGauge(String group, Gauge gauge) { return gauge; } + @Override + public ListGauge newListGauge(String group, ListGauge listGauge) { + return listGauge; + } + @Override public Timer newTimer(String group, String name) { return new Timer(name); diff --git a/samza-api/src/test/java/org/apache/samza/metrics/TestListGauge.java b/samza-api/src/test/java/org/apache/samza/metrics/TestListGauge.java new file mode 100644 index 0000000000..5c19440b3d --- /dev/null +++ b/samza-api/src/test/java/org/apache/samza/metrics/TestListGauge.java @@ -0,0 +1,97 @@ +/* + * 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; + +import java.time.Duration; +import java.util.Collection; +import java.util.Iterator; +import org.junit.Assert; +import org.junit.Test; + + +/** + * Class to encapsulate test-cases for {@link org.apache.samza.metrics.ListGauge} + */ +public class TestListGauge { + + private final static Duration THREAD_TEST_TIMEOUT = Duration.ofSeconds(10); + + @Test + public void basicTest() { + ListGauge listGauge = new ListGauge("sampleListGauge"); + listGauge.add("sampleValue"); + Assert.assertEquals("Names should be the same", listGauge.getName(), "sampleListGauge"); + Assert.assertEquals("List sizes should match", listGauge.getValue().size(), 1); + Assert.assertEquals("ListGauge should contain sampleGauge", listGauge.getValue().contains("sampleValue"), true); + } + + @Test + public void testSizeEnforcement() { + ListGauge listGauge = new ListGauge("listGauge"); + listGauge.setEvictionPolicy(new RetainLastNPolicy(listGauge, 10)); + for (int i = 15; i > 0; i--) { + listGauge.add("v" + i); + } + Assert.assertEquals("List sizes should be as configured at creation time", listGauge.getValue().size(), 10); + + int valueIndex = 10; + Collection currentList = listGauge.getValue(); + Iterator iterator = currentList.iterator(); + while (iterator.hasNext()) { + String gaugeValue = (String) iterator.next(); + Assert.assertTrue(gaugeValue.equals("v" + valueIndex)); + valueIndex--; + } + } + + @Test + public void testThreadSafety() throws InterruptedException { + ListGauge listGauge = new ListGauge("listGauge"); + listGauge.setEvictionPolicy(new RetainLastNPolicy(listGauge, 20)); + + Thread thread1 = new Thread(new Runnable() { + @Override + public void run() { + for (int i = 1; i <= 100; i++) { + listGauge.add(i); + } + } + }); + + Thread thread2 = new Thread(new Runnable() { + @Override + public void run() { + for (int i = 1; i <= 100; i++) { + listGauge.add(i); + } + } + }); + + thread1.start(); + thread2.start(); + + thread1.join(THREAD_TEST_TIMEOUT.toMillis()); + thread2.join(THREAD_TEST_TIMEOUT.toMillis()); + + Assert.assertTrue("ListGauge should have the last 20 values", listGauge.getValue().size() == 20); + for (Integer gaugeValue : listGauge.getValue()) { + Assert.assertTrue("Values should have the last 20 range", gaugeValue <= 100 && gaugeValue > 80); + } + } +} diff --git a/samza-api/src/test/java/org/apache/samza/metrics/TestTimer.java b/samza-api/src/test/java/org/apache/samza/metrics/TestTimer.java index 8076e02d61..b945636137 100644 --- a/samza-api/src/test/java/org/apache/samza/metrics/TestTimer.java +++ b/samza-api/src/test/java/org/apache/samza/metrics/TestTimer.java @@ -62,7 +62,7 @@ public void testTimerWithDifferentWindowSize() { assertTrue(snapshot.getValues().containsAll(Arrays.asList(1L, 2L, 3L))); assertEquals(3, snapshot.getValues().size()); - // The time is 500 for update(4L) because getSnapshot calls clock once + 3 + // The time is 500 for update(4L) because getValue calls clock once + 3 // updates that call clock 3 times timer.update(4L); Snapshot snapshot2 = timer.getSnapshot(); diff --git a/samza-azure/src/test/java/org/apache/samza/system/eventhub/TestMetricsRegistry.java b/samza-azure/src/test/java/org/apache/samza/system/eventhub/TestMetricsRegistry.java index d29b975578..312ad2e06a 100644 --- a/samza-azure/src/test/java/org/apache/samza/system/eventhub/TestMetricsRegistry.java +++ b/samza-azure/src/test/java/org/apache/samza/system/eventhub/TestMetricsRegistry.java @@ -1,27 +1,28 @@ /* -* 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. -*/ + * 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.system.eventhub; import org.apache.commons.collections4.map.HashedMap; import org.apache.samza.metrics.Counter; import org.apache.samza.metrics.Gauge; +import org.apache.samza.metrics.ListGauge; import org.apache.samza.metrics.MetricsRegistry; import org.apache.samza.metrics.Timer; @@ -29,10 +30,12 @@ import java.util.List; import java.util.Map; + public class TestMetricsRegistry implements MetricsRegistry { private Map> counters = new HashedMap<>(); private Map>> gauges = new HashedMap<>(); + private Map> listGauges = new HashedMap<>(); public List getCounters(String groupName) { return counters.get(groupName); @@ -78,6 +81,14 @@ public Gauge newGauge(String group, Gauge value) { return value; } + @Override + public ListGauge newListGauge(String group, ListGauge listGauge) { + listGauges.putIfAbsent(group, new ArrayList()); + ListGauge value = new ListGauge(group); + listGauges.get(group).add(value); + return value; + } + @Override public Timer newTimer(String group, String name) { return null; diff --git a/samza-core/src/main/java/org/apache/samza/metrics/MetricGroup.java b/samza-core/src/main/java/org/apache/samza/metrics/MetricGroup.java index 53526d86c4..4dfbbc9acf 100644 --- a/samza-core/src/main/java/org/apache/samza/metrics/MetricGroup.java +++ b/samza-core/src/main/java/org/apache/samza/metrics/MetricGroup.java @@ -1,21 +1,21 @@ /* -* 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. -*/ + * 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; @@ -48,6 +48,10 @@ public Gauge newGauge(String name, T value) { return registry.newGauge(groupName, new Gauge((prefix + name).toLowerCase(), value)); } + public ListGauge newListGauge(String name) { + return registry.newListGauge(groupName, new ListGauge(name)); + } + /* * Specify a dynamic gauge that always returns the latest value when polled. * The value closure/object must be thread safe, since metrics reporters may access diff --git a/samza-core/src/main/scala/org/apache/samza/container/SamzaContainer.scala b/samza-core/src/main/scala/org/apache/samza/container/SamzaContainer.scala index 5380fc98dc..ddc99ff6da 100644 --- a/samza-core/src/main/scala/org/apache/samza/container/SamzaContainer.scala +++ b/samza-core/src/main/scala/org/apache/samza/container/SamzaContainer.scala @@ -28,6 +28,7 @@ import java.util.Base64 import java.util.concurrent.{ExecutorService, Executors, ScheduledExecutorService, TimeUnit} import com.google.common.util.concurrent.ThreadFactoryBuilder +import org.apache.commons.lang3.exception.ExceptionUtils import org.apache.samza.checkpoint.{CheckpointListener, CheckpointManagerFactory, OffsetManager, OffsetManagerMetrics} import org.apache.samza.config.JobConfig.Config2Job import org.apache.samza.config.MetricsConfig.Config2Metrics @@ -42,7 +43,7 @@ import org.apache.samza.container.disk.{DiskQuotaPolicyFactory, DiskSpaceMonitor import org.apache.samza.container.host.{StatisticsMonitorImpl, SystemMemoryStatistics, SystemStatisticsMonitor} import org.apache.samza.coordinator.stream.{CoordinatorStreamManager, CoordinatorStreamSystemProducer} import org.apache.samza.job.model.JobModel -import org.apache.samza.metrics.{JmxServer, JvmMetrics, MetricsRegistryMap, MetricsReporter} +import org.apache.samza.metrics._ import org.apache.samza.serializers._ import org.apache.samza.serializers.model.SamzaObjectMapper import org.apache.samza.storage.{StorageEngineFactory, TaskStorageManager} diff --git a/samza-core/src/main/scala/org/apache/samza/container/SamzaContainerMetrics.scala b/samza-core/src/main/scala/org/apache/samza/container/SamzaContainerMetrics.scala index c1229565ad..ae7776b996 100644 --- a/samza-core/src/main/scala/org/apache/samza/container/SamzaContainerMetrics.scala +++ b/samza-core/src/main/scala/org/apache/samza/container/SamzaContainerMetrics.scala @@ -21,7 +21,7 @@ package org.apache.samza.container import java.util -import org.apache.samza.metrics.{Gauge, ReadableMetricsRegistry, MetricsRegistryMap, MetricsHelper} +import org.apache.samza.metrics._ class SamzaContainerMetrics( val source: String = "unknown", @@ -48,6 +48,9 @@ class SamzaContainerMetrics( val taskStoreRestorationMetrics: util.Map[TaskName, Gauge[Long]] = new util.HashMap[TaskName, Gauge[Long]]() + // A string-gauge metric to capture exceptions at this container + val exception = newListGauge[String]("exception") + def addStoreRestorationGauge(taskName: TaskName, storeName: String) { taskStoreRestorationMetrics.put(taskName, newGauge("%s-%s-restore-time" format(taskName.toString, storeName), -1L)) } diff --git a/samza-core/src/main/scala/org/apache/samza/metrics/MetricsHelper.scala b/samza-core/src/main/scala/org/apache/samza/metrics/MetricsHelper.scala index 1520b0e62d..21ec7633f4 100644 --- a/samza-core/src/main/scala/org/apache/samza/metrics/MetricsHelper.scala +++ b/samza-core/src/main/scala/org/apache/samza/metrics/MetricsHelper.scala @@ -40,9 +40,11 @@ trait MetricsHelper { def newGauge[T](name: String, value: T) = metricGroup.newGauge[T](name,value) + def newListGauge[T](name: String) = metricGroup.newListGauge[T](name) + /** - * Specify a dynamic gauge that always returns the latest value when polled. - * The value closure must be thread safe, since metrics reporters may access + * Specify a dynamic gauge that always returns the latest value when polled. + * The value closure must be thread safe, since metrics reporters may access * it from another thread. */ def newGauge[T](name: String, value: () => T) = { diff --git a/samza-core/src/main/scala/org/apache/samza/metrics/MetricsRegistryMap.scala b/samza-core/src/main/scala/org/apache/samza/metrics/MetricsRegistryMap.scala index 40ffee2bfa..d12c53ca2c 100644 --- a/samza-core/src/main/scala/org/apache/samza/metrics/MetricsRegistryMap.scala +++ b/samza-core/src/main/scala/org/apache/samza/metrics/MetricsRegistryMap.scala @@ -23,9 +23,9 @@ import org.apache.samza.util.Logging import java.util.concurrent.ConcurrentHashMap /** - * A class that holds all metrics registered with it. It can be registered - * with one or more MetricReporters to flush metrics. - */ + * A class that holds all metrics registered with it. It can be registered + * with one or more MetricReporters to flush metrics. + */ class MetricsRegistryMap(val name: String) extends ReadableMetricsRegistry with Logging { var listeners = Set[ReadableMetricsRegistryListener]() @@ -37,7 +37,7 @@ class MetricsRegistryMap(val name: String) extends ReadableMetricsRegistry with def this() = this("unknown") def newCounter(group: String, counter: Counter) = { - debug("Add new counter %s %s %s." format (group, counter.getName, counter)) + debug("Add new counter %s %s %s." format(group, counter.getName, counter)) putAndGetGroup(group).putIfAbsent(counter.getName, counter) val realCounter = metrics.get(group).get(counter.getName).asInstanceOf[Counter] listeners.foreach(_.onCounter(group, realCounter)) @@ -45,25 +45,40 @@ class MetricsRegistryMap(val name: String) extends ReadableMetricsRegistry with } def newCounter(group: String, name: String) = { - debug("Creating new counter %s %s." format (group, name)) + debug("Creating new counter %s %s." format(group, name)) newCounter(group, new Counter(name)) } def newGauge[T](group: String, gauge: Gauge[T]) = { - debug("Adding new gauge %s %s %s." format (group, gauge.getName, gauge)) + debug("Adding new gauge %s %s %s." format(group, gauge.getName, gauge)) putAndGetGroup(group).putIfAbsent(gauge.getName, gauge) val realGauge = metrics.get(group).get(gauge.getName).asInstanceOf[Gauge[T]] listeners.foreach(_.onGauge(group, realGauge)) realGauge } + /** + * Register a {@link org.apache.samza.metrics.ListGauge} + * + * @param group Group for this ListGauge + * @param listGauge the ListGauge to register + * @tparam T the type of the list gauge + */ + def newListGauge[T](group: String, listGauge: ListGauge[T]) = { + debug("Adding new listgauge %s %s %s." format(group, listGauge.getName, listGauge)) + putAndGetGroup(group).putIfAbsent(listGauge.getName, listGauge) + val realListGauge = metrics.get(group).get(listGauge.getName).asInstanceOf[ListGauge[T]] + listeners.foreach(_.onListGauge(group, realListGauge)) + realListGauge + } + def newGauge[T](group: String, name: String, value: T) = { - debug("Creating new gauge %s %s %s." format (group, name, value)) + debug("Creating new gauge %s %s %s." format(group, name, value)) newGauge(group, new Gauge[T](name, value)) } def newTimer(group: String, timer: Timer) = { - debug("Add new timer %s %s %s." format (group, timer.getName, timer)) + debug("Add new timer %s %s %s." format(group, timer.getName, timer)) putAndGetGroup(group).putIfAbsent(timer.getName, timer) val realTimer = metrics.get(group).get(timer.getName).asInstanceOf[Timer] listeners.foreach(_.onTimer(group, realTimer)) @@ -71,7 +86,7 @@ class MetricsRegistryMap(val name: String) extends ReadableMetricsRegistry with } def newTimer(group: String, name: String) = { - debug("Creating new timer %s %s." format (group, name)) + debug("Creating new timer %s %s." format(group, name)) newTimer(group, new Timer(name)) } diff --git a/samza-core/src/main/scala/org/apache/samza/metrics/reporter/JmxReporter.scala b/samza-core/src/main/scala/org/apache/samza/metrics/reporter/JmxReporter.scala index 7da8a9c52a..32718ced2c 100644 --- a/samza-core/src/main/scala/org/apache/samza/metrics/reporter/JmxReporter.scala +++ b/samza-core/src/main/scala/org/apache/samza/metrics/reporter/JmxReporter.scala @@ -20,19 +20,15 @@ package org.apache.samza.metrics.reporter import java.lang.management.ManagementFactory + import org.apache.samza.util.Logging import javax.management.MBeanServer import javax.management.ObjectName + import org.apache.samza.config.Config -import org.apache.samza.metrics.Counter -import org.apache.samza.metrics.Gauge -import org.apache.samza.metrics.Timer -import org.apache.samza.metrics.MetricsReporter -import org.apache.samza.metrics.MetricsReporterFactory -import org.apache.samza.metrics.ReadableMetricsRegistry -import org.apache.samza.metrics.ReadableMetricsRegistryListener +import org.apache.samza.metrics._ + import scala.collection.JavaConverters._ -import org.apache.samza.metrics.MetricsVisitor import org.apache.samza.metrics.JmxUtil._ class JmxReporter(server: MBeanServer) extends MetricsReporter with Logging { @@ -49,9 +45,15 @@ class JmxReporter(server: MBeanServer) extends MetricsReporter with Logging { registry.getGroup(group).asScala.foreach { case (name, metric) => metric.visit(new MetricsVisitor { + def counter(counter: Counter) = registerBean(new JmxCounter(counter, getObjectName(group, name, sources(registry)))) + def gauge[T](gauge: Gauge[T]) = registerBean(new JmxGauge(gauge.asInstanceOf[Gauge[Object]], getObjectName(group, name, sources(registry)))) + def timer(timer: Timer) = registerBean(new JmxTimer(timer, getObjectName(group, name, sources(registry)))) + + def listGauge[T](listGauge: ListGauge[T]) = registerBean(new JmxListGauge(listGauge.asInstanceOf[ListGauge[Object]], getObjectName(group, name, sources(registry)))) + }) } }) @@ -73,6 +75,10 @@ class JmxReporter(server: MBeanServer) extends MetricsReporter with Logging { def onTimer(group: String, timer: Timer) { registerBean(new JmxTimer(timer, getObjectName(group, timer.getName, source))) } + + def onListGauge(group: String, listGauge: ListGauge[_]) { + registerBean(new JmxListGauge(listGauge.asInstanceOf[ListGauge[Object]], getObjectName(group, listGauge.getName, source))) + } } } else { warn("Trying to re-register a registry for source %s. Ignoring." format source) @@ -107,6 +113,13 @@ trait JmxGaugeMBean extends MetricMBean { class JmxGauge(g: org.apache.samza.metrics.Gauge[Object], on: ObjectName) extends JmxGaugeMBean { def getValue = g.getValue + + def objectName = on +} + +class JmxListGauge(g: org.apache.samza.metrics.ListGauge[Object], on: ObjectName) extends JmxGaugeMBean { + def getValue = g.getValue + def objectName = on } @@ -116,6 +129,7 @@ trait JmxCounterMBean extends MetricMBean { class JmxCounter(c: org.apache.samza.metrics.Counter, on: ObjectName) extends JmxCounterMBean { def getCount() = c.getCount() + def objectName = on } @@ -125,6 +139,7 @@ trait JmxTimerMBean extends MetricMBean { class JmxTimer(t: org.apache.samza.metrics.Timer, on: ObjectName) extends JmxTimerMBean { def getAverageTime() = t.getSnapshot().getAverage() + def objectName = on } diff --git a/samza-core/src/main/scala/org/apache/samza/metrics/reporter/MetricsSnapshotReporter.scala b/samza-core/src/main/scala/org/apache/samza/metrics/reporter/MetricsSnapshotReporter.scala index 65ca49c2f3..b3e0eb7f8d 100644 --- a/samza-core/src/main/scala/org/apache/samza/metrics/reporter/MetricsSnapshotReporter.scala +++ b/samza-core/src/main/scala/org/apache/samza/metrics/reporter/MetricsSnapshotReporter.scala @@ -19,6 +19,8 @@ package org.apache.samza.metrics.reporter +import java.util + import com.google.common.util.concurrent.ThreadFactoryBuilder import org.apache.samza.metrics._ import org.apache.samza.serializers.Serializer @@ -26,7 +28,6 @@ import org.apache.samza.system.OutgoingMessageEnvelope import org.apache.samza.system.SystemProducer import org.apache.samza.system.SystemStream import org.apache.samza.util.Logging - import java.util.HashMap import java.util.Map import java.util.concurrent.Executors @@ -83,15 +84,18 @@ class MetricsSnapshotReporter( } def stop = { - info("Stopping producer.") - producer.stop + // Scheduling an event with 0 delay to ensure flushing of metrics one last time before shutdown + executor.schedule(this,0, TimeUnit.SECONDS) info("Stopping reporter timer.") - + // Allow the scheduled task above to finish, and block for termination (for max 60 seconds) executor.shutdown executor.awaitTermination(60, TimeUnit.SECONDS) + info("Stopping producer.") + producer.stop + if (!executor.isTerminated) { warn("Unable to shutdown reporter timer.") } @@ -112,6 +116,8 @@ class MetricsSnapshotReporter( registry.getGroup(group).asScala.foreach { case (name, metric) => metric.visit(new MetricsVisitor { + // for listGauge the value is returned as a list, which gets serialized + def listGauge[T](listGauge: ListGauge[T]) = {groupMsg.put(name, listGauge.getValue) } def counter(counter: Counter) = groupMsg.put(name, counter.getCount: java.lang.Long) def gauge[T](gauge: Gauge[T]) = groupMsg.put(name, gauge.getValue.asInstanceOf[Object]) def timer(timer: Timer) = groupMsg.put(name, timer.getSnapshot().getAverage(): java.lang.Double) @@ -133,12 +139,18 @@ class MetricsSnapshotReporter( metricsSnapshot } - producer.send(source, new OutgoingMessageEnvelope(out, host, null, maybeSerialized)) + try { + + producer.send(source, new OutgoingMessageEnvelope(out, host, null, maybeSerialized)) - // Always flush, since we don't want metrics to get batched up. - producer.flush(source) + // Always flush, since we don't want metrics to get batched up. + producer.flush(source) + } catch { + case e: Exception => error("Exception when flushing metrics for source %s " format(source), e) + } } + debug("Finished flushing metrics.") } } diff --git a/samza-yarn/src/main/scala/org/apache/samza/webapp/ApplicationMasterRestServlet.scala b/samza-yarn/src/main/scala/org/apache/samza/webapp/ApplicationMasterRestServlet.scala index 122a1dfaee..671a44910c 100644 --- a/samza-yarn/src/main/scala/org/apache/samza/webapp/ApplicationMasterRestServlet.scala +++ b/samza-yarn/src/main/scala/org/apache/samza/webapp/ApplicationMasterRestServlet.scala @@ -49,6 +49,9 @@ class ApplicationMasterRestServlet(samzaConfig: Config, samzaAppState: SamzaAppl registry.getGroup(group).asScala.foreach { case (name, metric) => metric.visit(new MetricsVisitor() { + def listGauge[T](listGauge: ListGauge[T]) = + groupMap.put(name, listGauge.getValue) + def counter(counter: Counter) = groupMap.put(counter.getName, counter.getCount: java.lang.Long) From 6602fa08b119df50022a7eaf56c5ad9be22f656d Mon Sep 17 00:00:00 2001 From: Ray Matharu Date: Fri, 1 Jun 2018 15:22:34 -0700 Subject: [PATCH 05/33] Adding DiagnosticsAppender, Populating ListGauge exception metric (samza container metric) using DiagnosticsAppender --- .../org/apache/samza/metrics/ListGauge.java | 1 - .../samza/container/SamzaContainer.scala | 15 ++++ .../container/SamzaContainerMetrics.scala | 3 +- .../diagnostics/DiagnosticsAppender.java | 84 ++++++++++++++++++ .../DiagnosticsExceptionEvent.java | 86 +++++++++++++++++++ 5 files changed, 187 insertions(+), 2 deletions(-) create mode 100644 samza-core/src/main/scala/org/apache/samza/diagnostics/DiagnosticsAppender.java create mode 100644 samza-core/src/main/scala/org/apache/samza/diagnostics/DiagnosticsExceptionEvent.java diff --git a/samza-api/src/main/java/org/apache/samza/metrics/ListGauge.java b/samza-api/src/main/java/org/apache/samza/metrics/ListGauge.java index a7541b3fc4..8c778f7fe7 100644 --- a/samza-api/src/main/java/org/apache/samza/metrics/ListGauge.java +++ b/samza-api/src/main/java/org/apache/samza/metrics/ListGauge.java @@ -21,7 +21,6 @@ import java.util.ArrayList; import java.util.Collection; import java.util.Collections; -import java.util.LinkedList; import java.util.List; diff --git a/samza-core/src/main/scala/org/apache/samza/container/SamzaContainer.scala b/samza-core/src/main/scala/org/apache/samza/container/SamzaContainer.scala index ddc99ff6da..623cf7ddce 100644 --- a/samza-core/src/main/scala/org/apache/samza/container/SamzaContainer.scala +++ b/samza-core/src/main/scala/org/apache/samza/container/SamzaContainer.scala @@ -54,6 +54,7 @@ import org.apache.samza.task._ import org.apache.samza.util.Util import org.apache.samza.util._ import org.apache.samza.{SamzaContainerStatus, SamzaException} +import org.apache.samza.diagnostics.DiagnosticsAppender; import scala.collection.JavaConverters._ @@ -723,6 +724,7 @@ class SamzaContainer( jmxServer = new JmxServer() startMetrics + startDiagnostics startAdmins startOffsetManager startLocalityManager @@ -870,6 +872,19 @@ class SamzaContainer( }) } + def startDiagnostics { + // TODO: where should this reside, MetricConfig? Log4jSystemConfig? or a new separate config? + val DIAGNOSTICS_APPENDER_ENABLE = "diagnostics.appender.enable" + if (containerContext.config.getBoolean(DIAGNOSTICS_APPENDER_ENABLE, false)) { + import org.apache.log4j.Logger + val rootLogger = Logger.getRootLogger + + info("Starting Diagnostics Appender.") + val diagnosticsAppender = new DiagnosticsAppender(this.metrics) + rootLogger.addAppender(diagnosticsAppender) + } + } + def startOffsetManager { info("Registering task instances with offsets.") diff --git a/samza-core/src/main/scala/org/apache/samza/container/SamzaContainerMetrics.scala b/samza-core/src/main/scala/org/apache/samza/container/SamzaContainerMetrics.scala index ae7776b996..b1b14647d0 100644 --- a/samza-core/src/main/scala/org/apache/samza/container/SamzaContainerMetrics.scala +++ b/samza-core/src/main/scala/org/apache/samza/container/SamzaContainerMetrics.scala @@ -21,6 +21,7 @@ package org.apache.samza.container import java.util +import org.apache.samza.diagnostics.DiagnosticsExceptionEvent import org.apache.samza.metrics._ class SamzaContainerMetrics( @@ -49,7 +50,7 @@ class SamzaContainerMetrics( val taskStoreRestorationMetrics: util.Map[TaskName, Gauge[Long]] = new util.HashMap[TaskName, Gauge[Long]]() // A string-gauge metric to capture exceptions at this container - val exception = newListGauge[String]("exception") + val exception = newListGauge[DiagnosticsExceptionEvent]("exception") def addStoreRestorationGauge(taskName: TaskName, storeName: String) { taskStoreRestorationMetrics.put(taskName, newGauge("%s-%s-restore-time" format(taskName.toString, storeName), -1L)) diff --git a/samza-core/src/main/scala/org/apache/samza/diagnostics/DiagnosticsAppender.java b/samza-core/src/main/scala/org/apache/samza/diagnostics/DiagnosticsAppender.java new file mode 100644 index 0000000000..345bbb1591 --- /dev/null +++ b/samza-core/src/main/scala/org/apache/samza/diagnostics/DiagnosticsAppender.java @@ -0,0 +1,84 @@ +/* + * 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.diagnostics; + +import java.util.Arrays; +import org.apache.log4j.AppenderSkeleton; +import org.apache.log4j.spi.LoggingEvent; +import org.apache.samza.container.SamzaContainerMetrics; +import org.apache.samza.metrics.ListGauge; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + + +/** + * Provides an in-memory appender that parses LoggingEvents to filter events relevant to diagnostics. + * Currently, filters exception related events and update an exception metric ({@link ListGauge}) in + * {@link SamzaContainerMetrics}. + * + * When used inconjunction with {@link org.apache.samza.metrics.reporter.MetricsSnapshotReporter} provides a + * stream of diagnostics-related events.s + */ +public class DiagnosticsAppender extends AppenderSkeleton { + + private final Logger logger = LoggerFactory.getLogger(this.getClass().getName()); + private final SamzaContainerMetrics samzaContainerMetrics; + + public DiagnosticsAppender(SamzaContainerMetrics samzaContainerMetrics) { + this.samzaContainerMetrics = samzaContainerMetrics; + } + + @Override + protected void append(LoggingEvent loggingEvent) { + + // if an event with a non-null throwable is received => exception event + if (loggingEvent.getThrowableInformation() != null) { + + DiagnosticsExceptionEvent diagnosticsExceptionEvent = + new DiagnosticsExceptionEvent(loggingEvent.timeStamp, loggingEvent.getMessage().toString(), + loggingEvent.getThreadName(), + Arrays.toString(loggingEvent.getThrowableInformation().getThrowableStrRep()), + getStackTraceIdentifier(loggingEvent.getThrowableInformation().getThrowable().getStackTrace())); + + ((ListGauge) samzaContainerMetrics.exception()).add(diagnosticsExceptionEvent); + logger.debug("Received DiagnosticsExceptionEvent " + diagnosticsExceptionEvent); + } else { + logger.debug("Received non-exception event with message " + loggingEvent.getMessage()); + } + } + + private int getStackTraceIdentifier(StackTraceElement[] stackTrace) { + return Arrays.hashCode(stackTrace); + } + + @Override + public void close() { + // Do nothing. + } + + /** + * Returns false since this appender requires no layout. + * @return false + */ + @Override + public boolean requiresLayout() { + return false; + } +} diff --git a/samza-core/src/main/scala/org/apache/samza/diagnostics/DiagnosticsExceptionEvent.java b/samza-core/src/main/scala/org/apache/samza/diagnostics/DiagnosticsExceptionEvent.java new file mode 100644 index 0000000000..97353cc584 --- /dev/null +++ b/samza-core/src/main/scala/org/apache/samza/diagnostics/DiagnosticsExceptionEvent.java @@ -0,0 +1,86 @@ +/* + * 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.diagnostics; + +/** + * This class encapsulates information related to an exception event that is useful for diagnostics. + * It used to define container, task, and other metrics as + * {@link org.apache.samza.metrics.ListGauge} of type {@link DiagnosticsExceptionEvent}. + */ +public class DiagnosticsExceptionEvent { + + private long timestamp; // the timestamp associated with this exception + private String message; // the string message associated with this exception + private String threadName; // the name of the thread on which this exception occurred + + // a compact string representation of this exception, to avoid serializing the entire stack trace + private String compactStackTrace; + + // a unique identifier computed to identify this stack trace + private Object stackTraceIdentifier; + + public DiagnosticsExceptionEvent(long timestampMillis, String message, String threadName, String compactStackTrace, + Object stackTraceIdentifier) { + this.message = message; + this.timestamp = timestampMillis; + this.threadName = threadName; + this.compactStackTrace = compactStackTrace; + this.stackTraceIdentifier = stackTraceIdentifier; + } + + public long getTimestamp() { + return timestamp; + } + + public void setTimestamp(long timestamp) { + this.timestamp = timestamp; + } + + public String getMessage() { + return message; + } + + public void setMessage(String message) { + this.message = message; + } + + public String getThreadName() { + return threadName; + } + + public void setThreadName(String threadName) { + this.threadName = threadName; + } + + public String getCompactStackTrace() { + return compactStackTrace; + } + + public void setCompactStackTrace(String compactStackTrace) { + this.compactStackTrace = compactStackTrace; + } + + public Object getStackTraceIdentifier() { + return stackTraceIdentifier; + } + + public void setStackTraceIdentifier(Object stackTraceIdentifier) { + this.stackTraceIdentifier = stackTraceIdentifier; + } +} \ No newline at end of file From 054d5078ad3ecbd98a85eaa67eeb57c78e2a0f70 Mon Sep 17 00:00:00 2001 From: Ray Matharu Date: Fri, 1 Jun 2018 18:01:27 -0700 Subject: [PATCH 06/33] Adding DiagnosticsExceptionEventEvictionPolicy to evict stale events from ListGauge --- .../org/apache/samza/metrics/ListGauge.java | 2 +- .../samza/metrics/RetainLastNPolicy.java | 4 +- .../diagnostics/DiagnosticsAppender.java | 16 ++++- ...agnosticsExceptionEventEvictionPolicy.java | 71 +++++++++++++++++++ 4 files changed, 87 insertions(+), 6 deletions(-) create mode 100644 samza-core/src/main/scala/org/apache/samza/diagnostics/DiagnosticsExceptionEventEvictionPolicy.java diff --git a/samza-api/src/main/java/org/apache/samza/metrics/ListGauge.java b/samza-api/src/main/java/org/apache/samza/metrics/ListGauge.java index 8c778f7fe7..3c88d04a49 100644 --- a/samza-api/src/main/java/org/apache/samza/metrics/ListGauge.java +++ b/samza-api/src/main/java/org/apache/samza/metrics/ListGauge.java @@ -72,7 +72,7 @@ public synchronized Collection getValue() { * Package-private method to change the eviction policy * @param listGaugeEvictionPolicy */ - synchronized void setEvictionPolicy(ListGaugeEvictionPolicy listGaugeEvictionPolicy) { + public synchronized void setEvictionPolicy(ListGaugeEvictionPolicy listGaugeEvictionPolicy) { this.listGaugeEvictionPolicy = listGaugeEvictionPolicy; } diff --git a/samza-api/src/main/java/org/apache/samza/metrics/RetainLastNPolicy.java b/samza-api/src/main/java/org/apache/samza/metrics/RetainLastNPolicy.java index 03be581039..6455ad40ca 100644 --- a/samza-api/src/main/java/org/apache/samza/metrics/RetainLastNPolicy.java +++ b/samza-api/src/main/java/org/apache/samza/metrics/RetainLastNPolicy.java @@ -24,8 +24,8 @@ public class RetainLastNPolicy implements ListGaugeEvictionPolicy { - private final ListGauge listGauge; - private final int nItems; + protected final ListGauge listGauge; + protected final int nItems; public RetainLastNPolicy(ListGauge listGauge, int numItems) { this.listGauge = listGauge; diff --git a/samza-core/src/main/scala/org/apache/samza/diagnostics/DiagnosticsAppender.java b/samza-core/src/main/scala/org/apache/samza/diagnostics/DiagnosticsAppender.java index 345bbb1591..aec2ec925e 100644 --- a/samza-core/src/main/scala/org/apache/samza/diagnostics/DiagnosticsAppender.java +++ b/samza-core/src/main/scala/org/apache/samza/diagnostics/DiagnosticsAppender.java @@ -19,6 +19,7 @@ package org.apache.samza.diagnostics; +import java.time.Duration; import java.util.Arrays; import org.apache.log4j.AppenderSkeleton; import org.apache.log4j.spi.LoggingEvent; @@ -39,10 +40,19 @@ public class DiagnosticsAppender extends AppenderSkeleton { private final Logger logger = LoggerFactory.getLogger(this.getClass().getName()); - private final SamzaContainerMetrics samzaContainerMetrics; + private final ListGauge samzaContainerExceptionMetric; + + private final static Duration DEFAULT_EVICTION_DURATION = Duration.ofMinutes(60); //one hour + private final static Duration DEFAULT_EVICTION_PERIOD = Duration.ofMinutes(1); //one minute + private final static int DEFAULT_MAX_NITEMS = 1000; // based on max kafka size of 1 MB public DiagnosticsAppender(SamzaContainerMetrics samzaContainerMetrics) { - this.samzaContainerMetrics = samzaContainerMetrics; + this.samzaContainerExceptionMetric = (ListGauge) samzaContainerMetrics.exception(); + DiagnosticsExceptionEventEvictionPolicy diagnosticsExceptionEventEvictionPolicy = + new DiagnosticsExceptionEventEvictionPolicy(samzaContainerExceptionMetric, DEFAULT_MAX_NITEMS, + DEFAULT_EVICTION_DURATION, DEFAULT_EVICTION_PERIOD); + ((ListGauge) samzaContainerMetrics.exception()).setEvictionPolicy( + diagnosticsExceptionEventEvictionPolicy); } @Override @@ -57,7 +67,7 @@ protected void append(LoggingEvent loggingEvent) { Arrays.toString(loggingEvent.getThrowableInformation().getThrowableStrRep()), getStackTraceIdentifier(loggingEvent.getThrowableInformation().getThrowable().getStackTrace())); - ((ListGauge) samzaContainerMetrics.exception()).add(diagnosticsExceptionEvent); + samzaContainerExceptionMetric.add(diagnosticsExceptionEvent); logger.debug("Received DiagnosticsExceptionEvent " + diagnosticsExceptionEvent); } else { logger.debug("Received non-exception event with message " + loggingEvent.getMessage()); diff --git a/samza-core/src/main/scala/org/apache/samza/diagnostics/DiagnosticsExceptionEventEvictionPolicy.java b/samza-core/src/main/scala/org/apache/samza/diagnostics/DiagnosticsExceptionEventEvictionPolicy.java new file mode 100644 index 0000000000..24520ae77a --- /dev/null +++ b/samza-core/src/main/scala/org/apache/samza/diagnostics/DiagnosticsExceptionEventEvictionPolicy.java @@ -0,0 +1,71 @@ +/* + * 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.diagnostics; + +import java.time.Duration; +import java.util.concurrent.Executors; +import java.util.concurrent.ScheduledExecutorService; +import java.util.concurrent.TimeUnit; +import java.util.stream.Stream; +import org.apache.samza.metrics.ListGauge; +import org.apache.samza.metrics.RetainLastNPolicy; + + +/** + * Provides an eviction policy that evicts entries from the ListGauge if + * a.) There are more elements in the listGauge than the specified maxNumberOfItems (removal in FIFO order), or + * b.) There are elements which have timestamps which are stale as compared to currentTime (the staleness bound is + * specified as maxStaleness). + * + * This naive implementation uses a periodic thread with a configurable period + * TODO: an event-schedule based implementation that schedules an "eviction" inside onElementAdded callback. + */ +public class DiagnosticsExceptionEventEvictionPolicy extends RetainLastNPolicy { + + private final Duration durationThreshold; + private final ScheduledExecutorService scheduledExecutorService; + + /** + * Construct a {@link DiagnosticsExceptionEventEvictionPolicy} + * @param listGauge The list gauge that is to be used with this policy + * @param maxNumberOfItems The max number of items that can remain in the listgauge + * @param maxStaleness The max staleness of items permitted in the listgauge + * @param period The periodicity with which the listGauge would be checked for stale values + */ + public DiagnosticsExceptionEventEvictionPolicy(ListGauge listGauge, int maxNumberOfItems, + Duration maxStaleness, Duration period) { + super(listGauge, maxNumberOfItems); + this.durationThreshold = maxStaleness; + this.scheduledExecutorService = Executors.newSingleThreadScheduledExecutor(); + this.scheduledExecutorService.schedule(new EvictionRunnable(), period.toMillis(), TimeUnit.MILLISECONDS); + } + + private class EvictionRunnable implements Runnable { + @Override + public void run() { + long currentTimestamp = System.currentTimeMillis(); + + // compute the list of events tso remove based on their timestamps, current time, and durationThreshold + Stream itemsToRemove = + listGauge.getValue().stream().filter(x -> currentTimestamp - x.getTimestamp() > durationThreshold.toMillis()); + + itemsToRemove.forEach(x -> listGauge.remove(x)); + } + } +} From 1f08eae481b78fac812238476acd0b7cfd41ddfe Mon Sep 17 00:00:00 2001 From: Ray Matharu Date: Mon, 4 Jun 2018 13:03:28 -0700 Subject: [PATCH 07/33] Removing remove() and policy-configurability from ListGauge, exposing only top-level params --- .../DefaultListGaugeEvictionPolicy.java | 86 +++++++++++++++++++ .../org/apache/samza/metrics/ListGauge.java | 83 +++++++++++------- .../samza/metrics/RetainLastNPolicy.java | 47 ---------- .../apache/samza/metrics/TestListGauge.java | 16 ++-- 4 files changed, 145 insertions(+), 87 deletions(-) create mode 100644 samza-api/src/main/java/org/apache/samza/metrics/DefaultListGaugeEvictionPolicy.java delete mode 100644 samza-api/src/main/java/org/apache/samza/metrics/RetainLastNPolicy.java diff --git a/samza-api/src/main/java/org/apache/samza/metrics/DefaultListGaugeEvictionPolicy.java b/samza-api/src/main/java/org/apache/samza/metrics/DefaultListGaugeEvictionPolicy.java new file mode 100644 index 0000000000..bc3763919c --- /dev/null +++ b/samza-api/src/main/java/org/apache/samza/metrics/DefaultListGaugeEvictionPolicy.java @@ -0,0 +1,86 @@ +/* + * 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; + +import java.time.Duration; +import java.time.Instant; +import java.util.Queue; +import java.util.concurrent.Executors; +import java.util.concurrent.ScheduledExecutorService; +import java.util.concurrent.TimeUnit; + + +/** + * Provides an eviction policy that evicts entries from the valueList if + * a.) There are more elements in the valueList than the specified maxNumberOfItems (removal in FIFO order), or + * b.) There are elements which have timestamps which are stale as compared to currentTime (the staleness bound is + * specified as maxStaleness). + * + * This naive implementation uses a periodic thread with a configurable period. + */ +public class DefaultListGaugeEvictionPolicy implements ListGaugeEvictionPolicy { + + private final Queue> valueList; + private final int nItems; + private final Duration durationThreshold; + private final ScheduledExecutorService scheduledExecutorService; + + public DefaultListGaugeEvictionPolicy(Queue> valueList, int maxNumberOfItems, + Duration maxStaleness, Duration period) { + this.valueList = valueList; + this.nItems = maxNumberOfItems; + this.durationThreshold = maxStaleness; + this.scheduledExecutorService = Executors.newSingleThreadScheduledExecutor(); + this.scheduledExecutorService.schedule(new EvictionRunnable(), period.toMillis(), TimeUnit.MILLISECONDS); + } + + @Override + public void elementAddedCallback() { + + // need to synchronize here because this thread could be concurrent with the runnable thread and can + // cause two vals to be removed (wrong eviction) even if a threadsafe queue was used. + synchronized (this.valueList) { + int numToEvict = this.valueList.size() - nItems; + + while (numToEvict > 0) { + this.valueList.poll(); // remove head + numToEvict--; + } + } + } + + private class EvictionRunnable implements Runnable { + + @Override + public void run() { + Instant currentTimestamp = Instant.now(); + + synchronized (valueList) { + ListGauge.ValueInfo valueInfo = valueList.peek(); + + // continue remove-head if currenttimestamp - head-element's timestamp > durationThreshold + while (valueInfo != null + && Duration.between(valueInfo.insertTimestamp, currentTimestamp).compareTo(durationThreshold) > 0) { + valueList.poll(); + valueInfo = valueList.peek(); + } + } + } + } +} diff --git a/samza-api/src/main/java/org/apache/samza/metrics/ListGauge.java b/samza-api/src/main/java/org/apache/samza/metrics/ListGauge.java index a7541b3fc4..42f047efa3 100644 --- a/samza-api/src/main/java/org/apache/samza/metrics/ListGauge.java +++ b/samza-api/src/main/java/org/apache/samza/metrics/ListGauge.java @@ -18,11 +18,13 @@ */ package org.apache.samza.metrics; -import java.util.ArrayList; +import java.time.Duration; +import java.time.Instant; import java.util.Collection; import java.util.Collections; -import java.util.LinkedList; -import java.util.List; +import java.util.Queue; +import java.util.concurrent.ConcurrentLinkedQueue; +import java.util.stream.Collectors; /** @@ -38,19 +40,34 @@ */ public class ListGauge implements Metric { private final String name; - private final List metricList; - private ListGaugeEvictionPolicy listGaugeEvictionPolicy; + private final Queue> metricList; + private final ListGaugeEvictionPolicy listGaugeEvictionPolicy; - private final static int DEFAULT_POLICY_NUM_RETAIN = 60; + private final static int DEFAULT_MAX_NITEMS = 1000; + private final static Duration DEFAULT_MAX_STALENESS = Duration.ofMinutes(60); + private final static Duration DEFAULT_EVICTION_CHECK_PERIOD = Duration.ofMinutes(1); /** - * Create a new {@link ListGauge} with no auto eviction, callers can add/remove items as desired. + * Create a new {@link ListGauge} that auto evicts based on the given maxNumberOfItems, maxStaleness, and period parameters. + * * @param name Name to be assigned + * @param maxNumberOfItems The max number of items that can remain in the listgauge + * @param maxStaleness The max staleness of items permitted in the listgauge + * @param period The periodicity with which the listGauge would be checked for stale values. */ - public ListGauge(String name) { + public ListGauge(String name, int maxNumberOfItems, Duration maxStaleness, Duration period) { this.name = name; - this.metricList = new ArrayList(DEFAULT_POLICY_NUM_RETAIN); - this.listGaugeEvictionPolicy = new RetainLastNPolicy(this, DEFAULT_POLICY_NUM_RETAIN); + this.metricList = new ConcurrentLinkedQueue>(); + this.listGaugeEvictionPolicy = + new DefaultListGaugeEvictionPolicy(this.metricList, maxNumberOfItems, maxStaleness, period); + } + + /** + * Create a new {@link ListGauge} that auto evicts upto a max of 100 items and a max-staleness of 60 minutes. + * @param name Name to be assigned + */ + public ListGauge(String name) { + this(name, DEFAULT_MAX_NITEMS, DEFAULT_MAX_STALENESS, DEFAULT_EVICTION_CHECK_PERIOD); } /** @@ -65,41 +82,41 @@ public String getName() { * Get the Collection of Gauge values currently in the list, used when serializing this Gauge. * @return the collection of gauge values */ - public synchronized Collection getValue() { - return Collections.unmodifiableList(this.metricList); - } - - /** - * Package-private method to change the eviction policy - * @param listGaugeEvictionPolicy - */ - synchronized void setEvictionPolicy(ListGaugeEvictionPolicy listGaugeEvictionPolicy) { - this.listGaugeEvictionPolicy = listGaugeEvictionPolicy; + public Collection getValue() { + return Collections.unmodifiableList(this.metricList.stream().map(x -> x.value).collect(Collectors.toList())); } /** - * Add a gauge to the list + * Add a value to the list. + * (Timestamp assigned to this value is the current timestamp.) * @param value The Gauge value to be added */ - public synchronized void add(T value) { - this.metricList.add(value); + public void add(T value) { + this.metricList.add(new ValueInfo(Instant.now(), value)); - // notify the policy object (if one is present), for performing any eviction that may be needed. - // note: monitor is being held - if (this.listGaugeEvictionPolicy != null) { - this.listGaugeEvictionPolicy.elementAddedCallback(); - } - } - - public synchronized boolean remove(T value) { - return this.metricList.remove(value); + // notify the policy object for performing any eviction that may be needed. + this.listGaugeEvictionPolicy.elementAddedCallback(); } /** * {@inheritDoc} */ @Override - public synchronized void visit(MetricsVisitor visitor) { + public void visit(MetricsVisitor visitor) { visitor.listGauge(this); } + + /** + * This class is used for bookkeeping of values added to the ListGauge. + * @param + */ + public static class ValueInfo { + public final Instant insertTimestamp; + public final T value; + + public ValueInfo(Instant insertTimestamp, T value) { + this.insertTimestamp = insertTimestamp; + this.value = value; + } + } } diff --git a/samza-api/src/main/java/org/apache/samza/metrics/RetainLastNPolicy.java b/samza-api/src/main/java/org/apache/samza/metrics/RetainLastNPolicy.java deleted file mode 100644 index 03be581039..0000000000 --- a/samza-api/src/main/java/org/apache/samza/metrics/RetainLastNPolicy.java +++ /dev/null @@ -1,47 +0,0 @@ -/* - * 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; - -import java.util.Collection; -import java.util.Iterator; - - -public class RetainLastNPolicy implements ListGaugeEvictionPolicy { - - private final ListGauge listGauge; - private final int nItems; - - public RetainLastNPolicy(ListGauge listGauge, int numItems) { - this.listGauge = listGauge; - this.nItems = numItems; - } - - @Override - public void elementAddedCallback() { - // get a snapshot of the list - Collection listGaugeCollection = this.listGauge.getValue(); - int numToEvict = listGaugeCollection.size() - nItems; - Iterator iterator = listGaugeCollection.iterator(); - while (numToEvict > 0 && iterator.hasNext()) { - // Remove in FIFO order to retain the last nItems - listGauge.remove(iterator.next()); - numToEvict--; - } - } -} diff --git a/samza-api/src/test/java/org/apache/samza/metrics/TestListGauge.java b/samza-api/src/test/java/org/apache/samza/metrics/TestListGauge.java index 5c19440b3d..e53bd6f88c 100644 --- a/samza-api/src/test/java/org/apache/samza/metrics/TestListGauge.java +++ b/samza-api/src/test/java/org/apache/samza/metrics/TestListGauge.java @@ -32,9 +32,13 @@ public class TestListGauge { private final static Duration THREAD_TEST_TIMEOUT = Duration.ofSeconds(10); + private ListGauge getListGaugeForTest() { + return new ListGauge("sampleListGauge", 10, Duration.ofSeconds(60), Duration.ofSeconds(60)); + } + @Test public void basicTest() { - ListGauge listGauge = new ListGauge("sampleListGauge"); + ListGauge listGauge = getListGaugeForTest(); listGauge.add("sampleValue"); Assert.assertEquals("Names should be the same", listGauge.getName(), "sampleListGauge"); Assert.assertEquals("List sizes should match", listGauge.getValue().size(), 1); @@ -43,8 +47,7 @@ public void basicTest() { @Test public void testSizeEnforcement() { - ListGauge listGauge = new ListGauge("listGauge"); - listGauge.setEvictionPolicy(new RetainLastNPolicy(listGauge, 10)); + ListGauge listGauge = getListGaugeForTest(); for (int i = 15; i > 0; i--) { listGauge.add("v" + i); } @@ -62,8 +65,7 @@ public void testSizeEnforcement() { @Test public void testThreadSafety() throws InterruptedException { - ListGauge listGauge = new ListGauge("listGauge"); - listGauge.setEvictionPolicy(new RetainLastNPolicy(listGauge, 20)); + ListGauge listGauge = getListGaugeForTest(); Thread thread1 = new Thread(new Runnable() { @Override @@ -89,9 +91,9 @@ public void run() { thread1.join(THREAD_TEST_TIMEOUT.toMillis()); thread2.join(THREAD_TEST_TIMEOUT.toMillis()); - Assert.assertTrue("ListGauge should have the last 20 values", listGauge.getValue().size() == 20); + Assert.assertTrue("ListGauge should have the last 10 values", listGauge.getValue().size() == 10); for (Integer gaugeValue : listGauge.getValue()) { - Assert.assertTrue("Values should have the last 20 range", gaugeValue <= 100 && gaugeValue > 80); + Assert.assertTrue("Values should have the last 10 range", gaugeValue <= 100 && gaugeValue > 90); } } } From 6f19ba0adc40fd2dd377b76ef564449ed266f387 Mon Sep 17 00:00:00 2001 From: Ray Matharu Date: Mon, 4 Jun 2018 13:23:13 -0700 Subject: [PATCH 08/33] Renaming valueList to elements, and getValue to getValues --- .../DefaultListGaugeEvictionPolicy.java | 24 +++++++++---------- .../org/apache/samza/metrics/ListGauge.java | 12 +++++----- .../apache/samza/metrics/TestListGauge.java | 12 +++++----- .../org/apache/samza/metrics/TestTimer.java | 2 +- .../samza/metrics/reporter/JmxReporter.scala | 2 +- .../reporter/MetricsSnapshotReporter.scala | 2 +- .../webapp/ApplicationMasterRestServlet.scala | 2 +- 7 files changed, 28 insertions(+), 28 deletions(-) diff --git a/samza-api/src/main/java/org/apache/samza/metrics/DefaultListGaugeEvictionPolicy.java b/samza-api/src/main/java/org/apache/samza/metrics/DefaultListGaugeEvictionPolicy.java index bc3763919c..6d0779b2c9 100644 --- a/samza-api/src/main/java/org/apache/samza/metrics/DefaultListGaugeEvictionPolicy.java +++ b/samza-api/src/main/java/org/apache/samza/metrics/DefaultListGaugeEvictionPolicy.java @@ -27,8 +27,8 @@ /** - * Provides an eviction policy that evicts entries from the valueList if - * a.) There are more elements in the valueList than the specified maxNumberOfItems (removal in FIFO order), or + * Provides an eviction policy that evicts entries from the elements if + * a.) There are more elements in the elements than the specified maxNumberOfItems (removal in FIFO order), or * b.) There are elements which have timestamps which are stale as compared to currentTime (the staleness bound is * specified as maxStaleness). * @@ -36,14 +36,14 @@ */ public class DefaultListGaugeEvictionPolicy implements ListGaugeEvictionPolicy { - private final Queue> valueList; + private final Queue> elements; private final int nItems; private final Duration durationThreshold; private final ScheduledExecutorService scheduledExecutorService; - public DefaultListGaugeEvictionPolicy(Queue> valueList, int maxNumberOfItems, + public DefaultListGaugeEvictionPolicy(Queue> elements, int maxNumberOfItems, Duration maxStaleness, Duration period) { - this.valueList = valueList; + this.elements = elements; this.nItems = maxNumberOfItems; this.durationThreshold = maxStaleness; this.scheduledExecutorService = Executors.newSingleThreadScheduledExecutor(); @@ -55,11 +55,11 @@ public void elementAddedCallback() { // need to synchronize here because this thread could be concurrent with the runnable thread and can // cause two vals to be removed (wrong eviction) even if a threadsafe queue was used. - synchronized (this.valueList) { - int numToEvict = this.valueList.size() - nItems; + synchronized (this.elements) { + int numToEvict = this.elements.size() - nItems; while (numToEvict > 0) { - this.valueList.poll(); // remove head + this.elements.poll(); // remove head numToEvict--; } } @@ -71,14 +71,14 @@ private class EvictionRunnable implements Runnable { public void run() { Instant currentTimestamp = Instant.now(); - synchronized (valueList) { - ListGauge.ValueInfo valueInfo = valueList.peek(); + synchronized (elements) { + ListGauge.ValueInfo valueInfo = elements.peek(); // continue remove-head if currenttimestamp - head-element's timestamp > durationThreshold while (valueInfo != null && Duration.between(valueInfo.insertTimestamp, currentTimestamp).compareTo(durationThreshold) > 0) { - valueList.poll(); - valueInfo = valueList.peek(); + elements.poll(); + valueInfo = elements.peek(); } } } diff --git a/samza-api/src/main/java/org/apache/samza/metrics/ListGauge.java b/samza-api/src/main/java/org/apache/samza/metrics/ListGauge.java index 42f047efa3..ae93955432 100644 --- a/samza-api/src/main/java/org/apache/samza/metrics/ListGauge.java +++ b/samza-api/src/main/java/org/apache/samza/metrics/ListGauge.java @@ -40,7 +40,7 @@ */ public class ListGauge implements Metric { private final String name; - private final Queue> metricList; + private final Queue> elements; private final ListGaugeEvictionPolicy listGaugeEvictionPolicy; private final static int DEFAULT_MAX_NITEMS = 1000; @@ -57,9 +57,9 @@ public class ListGauge implements Metric { */ public ListGauge(String name, int maxNumberOfItems, Duration maxStaleness, Duration period) { this.name = name; - this.metricList = new ConcurrentLinkedQueue>(); + this.elements = new ConcurrentLinkedQueue>(); this.listGaugeEvictionPolicy = - new DefaultListGaugeEvictionPolicy(this.metricList, maxNumberOfItems, maxStaleness, period); + new DefaultListGaugeEvictionPolicy(this.elements, maxNumberOfItems, maxStaleness, period); } /** @@ -82,8 +82,8 @@ public String getName() { * Get the Collection of Gauge values currently in the list, used when serializing this Gauge. * @return the collection of gauge values */ - public Collection getValue() { - return Collections.unmodifiableList(this.metricList.stream().map(x -> x.value).collect(Collectors.toList())); + public Collection getValues() { + return Collections.unmodifiableList(this.elements.stream().map(x -> x.value).collect(Collectors.toList())); } /** @@ -92,7 +92,7 @@ public Collection getValue() { * @param value The Gauge value to be added */ public void add(T value) { - this.metricList.add(new ValueInfo(Instant.now(), value)); + this.elements.add(new ValueInfo(Instant.now(), value)); // notify the policy object for performing any eviction that may be needed. this.listGaugeEvictionPolicy.elementAddedCallback(); diff --git a/samza-api/src/test/java/org/apache/samza/metrics/TestListGauge.java b/samza-api/src/test/java/org/apache/samza/metrics/TestListGauge.java index e53bd6f88c..74e1fd02a8 100644 --- a/samza-api/src/test/java/org/apache/samza/metrics/TestListGauge.java +++ b/samza-api/src/test/java/org/apache/samza/metrics/TestListGauge.java @@ -41,8 +41,8 @@ public void basicTest() { ListGauge listGauge = getListGaugeForTest(); listGauge.add("sampleValue"); Assert.assertEquals("Names should be the same", listGauge.getName(), "sampleListGauge"); - Assert.assertEquals("List sizes should match", listGauge.getValue().size(), 1); - Assert.assertEquals("ListGauge should contain sampleGauge", listGauge.getValue().contains("sampleValue"), true); + Assert.assertEquals("List sizes should match", listGauge.getValues().size(), 1); + Assert.assertEquals("ListGauge should contain sampleGauge", listGauge.getValues().contains("sampleValue"), true); } @Test @@ -51,10 +51,10 @@ public void testSizeEnforcement() { for (int i = 15; i > 0; i--) { listGauge.add("v" + i); } - Assert.assertEquals("List sizes should be as configured at creation time", listGauge.getValue().size(), 10); + Assert.assertEquals("List sizes should be as configured at creation time", listGauge.getValues().size(), 10); int valueIndex = 10; - Collection currentList = listGauge.getValue(); + Collection currentList = listGauge.getValues(); Iterator iterator = currentList.iterator(); while (iterator.hasNext()) { String gaugeValue = (String) iterator.next(); @@ -91,8 +91,8 @@ public void run() { thread1.join(THREAD_TEST_TIMEOUT.toMillis()); thread2.join(THREAD_TEST_TIMEOUT.toMillis()); - Assert.assertTrue("ListGauge should have the last 10 values", listGauge.getValue().size() == 10); - for (Integer gaugeValue : listGauge.getValue()) { + Assert.assertTrue("ListGauge should have the last 10 values", listGauge.getValues().size() == 10); + for (Integer gaugeValue : listGauge.getValues()) { Assert.assertTrue("Values should have the last 10 range", gaugeValue <= 100 && gaugeValue > 90); } } diff --git a/samza-api/src/test/java/org/apache/samza/metrics/TestTimer.java b/samza-api/src/test/java/org/apache/samza/metrics/TestTimer.java index b945636137..6fdca549c5 100644 --- a/samza-api/src/test/java/org/apache/samza/metrics/TestTimer.java +++ b/samza-api/src/test/java/org/apache/samza/metrics/TestTimer.java @@ -62,7 +62,7 @@ public void testTimerWithDifferentWindowSize() { assertTrue(snapshot.getValues().containsAll(Arrays.asList(1L, 2L, 3L))); assertEquals(3, snapshot.getValues().size()); - // The time is 500 for update(4L) because getValue calls clock once + 3 + // The time is 500 for update(4L) because getValues calls clock once + 3 // updates that call clock 3 times timer.update(4L); Snapshot snapshot2 = timer.getSnapshot(); diff --git a/samza-core/src/main/scala/org/apache/samza/metrics/reporter/JmxReporter.scala b/samza-core/src/main/scala/org/apache/samza/metrics/reporter/JmxReporter.scala index 32718ced2c..5aa363ad4c 100644 --- a/samza-core/src/main/scala/org/apache/samza/metrics/reporter/JmxReporter.scala +++ b/samza-core/src/main/scala/org/apache/samza/metrics/reporter/JmxReporter.scala @@ -118,7 +118,7 @@ class JmxGauge(g: org.apache.samza.metrics.Gauge[Object], on: ObjectName) extend } class JmxListGauge(g: org.apache.samza.metrics.ListGauge[Object], on: ObjectName) extends JmxGaugeMBean { - def getValue = g.getValue + def getValue = g.getValues def objectName = on } diff --git a/samza-core/src/main/scala/org/apache/samza/metrics/reporter/MetricsSnapshotReporter.scala b/samza-core/src/main/scala/org/apache/samza/metrics/reporter/MetricsSnapshotReporter.scala index b3e0eb7f8d..1f22cff7e4 100644 --- a/samza-core/src/main/scala/org/apache/samza/metrics/reporter/MetricsSnapshotReporter.scala +++ b/samza-core/src/main/scala/org/apache/samza/metrics/reporter/MetricsSnapshotReporter.scala @@ -117,7 +117,7 @@ class MetricsSnapshotReporter( case (name, metric) => metric.visit(new MetricsVisitor { // for listGauge the value is returned as a list, which gets serialized - def listGauge[T](listGauge: ListGauge[T]) = {groupMsg.put(name, listGauge.getValue) } + def listGauge[T](listGauge: ListGauge[T]) = {groupMsg.put(name, listGauge.getValues) } def counter(counter: Counter) = groupMsg.put(name, counter.getCount: java.lang.Long) def gauge[T](gauge: Gauge[T]) = groupMsg.put(name, gauge.getValue.asInstanceOf[Object]) def timer(timer: Timer) = groupMsg.put(name, timer.getSnapshot().getAverage(): java.lang.Double) diff --git a/samza-yarn/src/main/scala/org/apache/samza/webapp/ApplicationMasterRestServlet.scala b/samza-yarn/src/main/scala/org/apache/samza/webapp/ApplicationMasterRestServlet.scala index 671a44910c..f60e021a75 100644 --- a/samza-yarn/src/main/scala/org/apache/samza/webapp/ApplicationMasterRestServlet.scala +++ b/samza-yarn/src/main/scala/org/apache/samza/webapp/ApplicationMasterRestServlet.scala @@ -50,7 +50,7 @@ class ApplicationMasterRestServlet(samzaConfig: Config, samzaAppState: SamzaAppl case (name, metric) => metric.visit(new MetricsVisitor() { def listGauge[T](listGauge: ListGauge[T]) = - groupMap.put(name, listGauge.getValue) + groupMap.put(name, listGauge.getValues) def counter(counter: Counter) = groupMap.put(counter.getName, counter.getCount: java.lang.Long) From 4f08cd196ba2a7b364d2a49642fa8ef8d57c51dc Mon Sep 17 00:00:00 2001 From: Ray Matharu Date: Mon, 4 Jun 2018 17:28:02 -0700 Subject: [PATCH 09/33] Removing the interface for ListGaugeEvictionPolicy --- .../DefaultListGaugeEvictionPolicy.java | 3 +-- .../org/apache/samza/metrics/ListGauge.java | 2 +- .../metrics/ListGaugeEvictionPolicy.java | 23 ------------------- 3 files changed, 2 insertions(+), 26 deletions(-) delete mode 100644 samza-api/src/main/java/org/apache/samza/metrics/ListGaugeEvictionPolicy.java diff --git a/samza-api/src/main/java/org/apache/samza/metrics/DefaultListGaugeEvictionPolicy.java b/samza-api/src/main/java/org/apache/samza/metrics/DefaultListGaugeEvictionPolicy.java index 6d0779b2c9..0139f12f93 100644 --- a/samza-api/src/main/java/org/apache/samza/metrics/DefaultListGaugeEvictionPolicy.java +++ b/samza-api/src/main/java/org/apache/samza/metrics/DefaultListGaugeEvictionPolicy.java @@ -34,7 +34,7 @@ * * This naive implementation uses a periodic thread with a configurable period. */ -public class DefaultListGaugeEvictionPolicy implements ListGaugeEvictionPolicy { +public class DefaultListGaugeEvictionPolicy { private final Queue> elements; private final int nItems; @@ -50,7 +50,6 @@ public DefaultListGaugeEvictionPolicy(Queue> elements, in this.scheduledExecutorService.schedule(new EvictionRunnable(), period.toMillis(), TimeUnit.MILLISECONDS); } - @Override public void elementAddedCallback() { // need to synchronize here because this thread could be concurrent with the runnable thread and can diff --git a/samza-api/src/main/java/org/apache/samza/metrics/ListGauge.java b/samza-api/src/main/java/org/apache/samza/metrics/ListGauge.java index ae93955432..634c2e8fc5 100644 --- a/samza-api/src/main/java/org/apache/samza/metrics/ListGauge.java +++ b/samza-api/src/main/java/org/apache/samza/metrics/ListGauge.java @@ -41,7 +41,7 @@ public class ListGauge implements Metric { private final String name; private final Queue> elements; - private final ListGaugeEvictionPolicy listGaugeEvictionPolicy; + private final DefaultListGaugeEvictionPolicy listGaugeEvictionPolicy; private final static int DEFAULT_MAX_NITEMS = 1000; private final static Duration DEFAULT_MAX_STALENESS = Duration.ofMinutes(60); diff --git a/samza-api/src/main/java/org/apache/samza/metrics/ListGaugeEvictionPolicy.java b/samza-api/src/main/java/org/apache/samza/metrics/ListGaugeEvictionPolicy.java deleted file mode 100644 index 966a54a10a..0000000000 --- a/samza-api/src/main/java/org/apache/samza/metrics/ListGaugeEvictionPolicy.java +++ /dev/null @@ -1,23 +0,0 @@ -/* - * 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; - -public interface ListGaugeEvictionPolicy { - void elementAddedCallback(); -} From 906afebca5ba3fc8500f1b925966cf14caa23d61 Mon Sep 17 00:00:00 2001 From: Ray Matharu Date: Mon, 4 Jun 2018 18:12:49 -0700 Subject: [PATCH 10/33] Removing ValueInfo class and moving TimestampedValue to samza-api util --- .../DefaultListGaugeEvictionPolicy.java | 9 +-- .../org/apache/samza/metrics/ListGauge.java | 21 ++----- .../functions/PartialJoinFunction.java | 2 +- .../operators/impl/OperatorImplGraph.java | 2 +- .../impl/PartialJoinOperatorImpl.java | 2 +- .../operators/impl/WindowOperatorImpl.java | 2 +- .../operators/impl/store/TimeSeriesStore.java | 2 + .../impl/store/TimeSeriesStoreImpl.java | 1 + .../impl/store/TimestampedValue.java | 61 ------------------- .../impl/store/TimestampedValueSerde.java | 1 + .../operators/spec/JoinOperatorSpec.java | 2 +- .../operators/impl/TestOperatorImplGraph.java | 2 +- .../impl/store/TestTimeSeriesStoreImpl.java | 1 + .../impl/store/TestTimestampedValueSerde.java | 1 + 14 files changed, 22 insertions(+), 87 deletions(-) delete mode 100644 samza-core/src/main/java/org/apache/samza/operators/impl/store/TimestampedValue.java diff --git a/samza-api/src/main/java/org/apache/samza/metrics/DefaultListGaugeEvictionPolicy.java b/samza-api/src/main/java/org/apache/samza/metrics/DefaultListGaugeEvictionPolicy.java index 0139f12f93..84fdcf102f 100644 --- a/samza-api/src/main/java/org/apache/samza/metrics/DefaultListGaugeEvictionPolicy.java +++ b/samza-api/src/main/java/org/apache/samza/metrics/DefaultListGaugeEvictionPolicy.java @@ -24,6 +24,7 @@ import java.util.concurrent.Executors; import java.util.concurrent.ScheduledExecutorService; import java.util.concurrent.TimeUnit; +import org.apache.samza.util.TimestampedValue; /** @@ -36,12 +37,12 @@ */ public class DefaultListGaugeEvictionPolicy { - private final Queue> elements; + private final Queue> elements; private final int nItems; private final Duration durationThreshold; private final ScheduledExecutorService scheduledExecutorService; - public DefaultListGaugeEvictionPolicy(Queue> elements, int maxNumberOfItems, + public DefaultListGaugeEvictionPolicy(Queue> elements, int maxNumberOfItems, Duration maxStaleness, Duration period) { this.elements = elements; this.nItems = maxNumberOfItems; @@ -71,11 +72,11 @@ public void run() { Instant currentTimestamp = Instant.now(); synchronized (elements) { - ListGauge.ValueInfo valueInfo = elements.peek(); + TimestampedValue valueInfo = elements.peek(); // continue remove-head if currenttimestamp - head-element's timestamp > durationThreshold while (valueInfo != null - && Duration.between(valueInfo.insertTimestamp, currentTimestamp).compareTo(durationThreshold) > 0) { + && currentTimestamp.toEpochMilli() - valueInfo.getTimestamp() > durationThreshold.toMillis()) { elements.poll(); valueInfo = elements.peek(); } diff --git a/samza-api/src/main/java/org/apache/samza/metrics/ListGauge.java b/samza-api/src/main/java/org/apache/samza/metrics/ListGauge.java index 634c2e8fc5..65ddeba5fc 100644 --- a/samza-api/src/main/java/org/apache/samza/metrics/ListGauge.java +++ b/samza-api/src/main/java/org/apache/samza/metrics/ListGauge.java @@ -25,6 +25,7 @@ import java.util.Queue; import java.util.concurrent.ConcurrentLinkedQueue; import java.util.stream.Collectors; +import org.apache.samza.util.TimestampedValue; /** @@ -40,7 +41,7 @@ */ public class ListGauge implements Metric { private final String name; - private final Queue> elements; + private final Queue> elements; private final DefaultListGaugeEvictionPolicy listGaugeEvictionPolicy; private final static int DEFAULT_MAX_NITEMS = 1000; @@ -57,7 +58,7 @@ public class ListGauge implements Metric { */ public ListGauge(String name, int maxNumberOfItems, Duration maxStaleness, Duration period) { this.name = name; - this.elements = new ConcurrentLinkedQueue>(); + this.elements = new ConcurrentLinkedQueue>(); this.listGaugeEvictionPolicy = new DefaultListGaugeEvictionPolicy(this.elements, maxNumberOfItems, maxStaleness, period); } @@ -83,7 +84,7 @@ public String getName() { * @return the collection of gauge values */ public Collection getValues() { - return Collections.unmodifiableList(this.elements.stream().map(x -> x.value).collect(Collectors.toList())); + return Collections.unmodifiableList(this.elements.stream().map(x -> x.getValue()).collect(Collectors.toList())); } /** @@ -92,7 +93,7 @@ public Collection getValues() { * @param value The Gauge value to be added */ public void add(T value) { - this.elements.add(new ValueInfo(Instant.now(), value)); + this.elements.add(new TimestampedValue(value, Instant.now().toEpochMilli())); // notify the policy object for performing any eviction that may be needed. this.listGaugeEvictionPolicy.elementAddedCallback(); @@ -106,17 +107,5 @@ public void visit(MetricsVisitor visitor) { visitor.listGauge(this); } - /** - * This class is used for bookkeeping of values added to the ListGauge. - * @param - */ - public static class ValueInfo { - public final Instant insertTimestamp; - public final T value; - public ValueInfo(Instant insertTimestamp, T value) { - this.insertTimestamp = insertTimestamp; - this.value = value; - } - } } diff --git a/samza-core/src/main/java/org/apache/samza/operators/functions/PartialJoinFunction.java b/samza-core/src/main/java/org/apache/samza/operators/functions/PartialJoinFunction.java index 5ede5e8f14..038abba9d0 100644 --- a/samza-core/src/main/java/org/apache/samza/operators/functions/PartialJoinFunction.java +++ b/samza-core/src/main/java/org/apache/samza/operators/functions/PartialJoinFunction.java @@ -18,7 +18,7 @@ */ package org.apache.samza.operators.functions; -import org.apache.samza.operators.impl.store.TimestampedValue; +import org.apache.samza.util.TimestampedValue; import org.apache.samza.storage.kv.KeyValueStore; /** diff --git a/samza-core/src/main/java/org/apache/samza/operators/impl/OperatorImplGraph.java b/samza-core/src/main/java/org/apache/samza/operators/impl/OperatorImplGraph.java index bbc878364b..d675822df0 100644 --- a/samza-core/src/main/java/org/apache/samza/operators/impl/OperatorImplGraph.java +++ b/samza-core/src/main/java/org/apache/samza/operators/impl/OperatorImplGraph.java @@ -29,7 +29,7 @@ import org.apache.samza.operators.TimerRegistry; import org.apache.samza.operators.functions.JoinFunction; import org.apache.samza.operators.functions.PartialJoinFunction; -import org.apache.samza.operators.impl.store.TimestampedValue; +import org.apache.samza.util.TimestampedValue; import org.apache.samza.operators.spec.BroadcastOperatorSpec; import org.apache.samza.operators.spec.InputOperatorSpec; import org.apache.samza.operators.spec.JoinOperatorSpec; diff --git a/samza-core/src/main/java/org/apache/samza/operators/impl/PartialJoinOperatorImpl.java b/samza-core/src/main/java/org/apache/samza/operators/impl/PartialJoinOperatorImpl.java index 90a71a0eff..0cdde490f6 100644 --- a/samza-core/src/main/java/org/apache/samza/operators/impl/PartialJoinOperatorImpl.java +++ b/samza-core/src/main/java/org/apache/samza/operators/impl/PartialJoinOperatorImpl.java @@ -21,7 +21,7 @@ import org.apache.samza.SamzaException; import org.apache.samza.config.Config; import org.apache.samza.operators.functions.PartialJoinFunction; -import org.apache.samza.operators.impl.store.TimestampedValue; +import org.apache.samza.util.TimestampedValue; import org.apache.samza.operators.spec.JoinOperatorSpec; import org.apache.samza.operators.spec.OperatorSpec; import org.apache.samza.storage.kv.KeyValueStore; diff --git a/samza-core/src/main/java/org/apache/samza/operators/impl/WindowOperatorImpl.java b/samza-core/src/main/java/org/apache/samza/operators/impl/WindowOperatorImpl.java index 32406cbcf0..32b6e5dd49 100644 --- a/samza-core/src/main/java/org/apache/samza/operators/impl/WindowOperatorImpl.java +++ b/samza-core/src/main/java/org/apache/samza/operators/impl/WindowOperatorImpl.java @@ -26,7 +26,7 @@ import org.apache.samza.operators.impl.store.TimeSeriesKey; import org.apache.samza.operators.impl.store.TimeSeriesStore; import org.apache.samza.operators.impl.store.TimeSeriesStoreImpl; -import org.apache.samza.operators.impl.store.TimestampedValue; +import org.apache.samza.util.TimestampedValue; import org.apache.samza.operators.spec.OperatorSpec; import org.apache.samza.operators.spec.WindowOperatorSpec; import org.apache.samza.operators.triggers.FiringType; diff --git a/samza-core/src/main/java/org/apache/samza/operators/impl/store/TimeSeriesStore.java b/samza-core/src/main/java/org/apache/samza/operators/impl/store/TimeSeriesStore.java index f3d694824a..c9b694d37f 100644 --- a/samza-core/src/main/java/org/apache/samza/operators/impl/store/TimeSeriesStore.java +++ b/samza-core/src/main/java/org/apache/samza/operators/impl/store/TimeSeriesStore.java @@ -21,6 +21,8 @@ package org.apache.samza.operators.impl.store; import org.apache.samza.storage.kv.ClosableIterator; +import org.apache.samza.util.TimestampedValue; + /** * A key-value store that allows entries to be queried and stored based on time ranges. diff --git a/samza-core/src/main/java/org/apache/samza/operators/impl/store/TimeSeriesStoreImpl.java b/samza-core/src/main/java/org/apache/samza/operators/impl/store/TimeSeriesStoreImpl.java index f03d3969f9..10a59679bb 100644 --- a/samza-core/src/main/java/org/apache/samza/operators/impl/store/TimeSeriesStoreImpl.java +++ b/samza-core/src/main/java/org/apache/samza/operators/impl/store/TimeSeriesStoreImpl.java @@ -23,6 +23,7 @@ import org.apache.samza.storage.kv.Entry; import org.apache.samza.storage.kv.KeyValueIterator; import org.apache.samza.storage.kv.KeyValueStore; +import org.apache.samza.util.TimestampedValue; import org.slf4j.Logger; import org.slf4j.LoggerFactory; diff --git a/samza-core/src/main/java/org/apache/samza/operators/impl/store/TimestampedValue.java b/samza-core/src/main/java/org/apache/samza/operators/impl/store/TimestampedValue.java deleted file mode 100644 index 5e451485f3..0000000000 --- a/samza-core/src/main/java/org/apache/samza/operators/impl/store/TimestampedValue.java +++ /dev/null @@ -1,61 +0,0 @@ -/* - * 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.operators.impl.store; - -/** - * An immutable pair of a value, and its corresponding timestamp. - * - * @param the type of the value - */ -public class TimestampedValue { - private final V value; - private final long timestamp; - - public TimestampedValue(V v, long timestamp) { - this.value = v; - this.timestamp = timestamp; - } - - public V getValue() { - return value; - } - - public long getTimestamp() { - return timestamp; - } - - @Override - public boolean equals(Object o) { - if (this == o) return true; - if (o == null || !getClass().equals(o.getClass())) return false; - - TimestampedValue that = (TimestampedValue) o; - - if (timestamp != that.timestamp) return false; - return value != null ? value.equals(that.value) : (that.value == null); - } - - @Override - public int hashCode() { - int result = value != null ? value.hashCode() : 0; - result = 31 * result + (int) (timestamp ^ (timestamp >>> 32)); - return result; - } -} diff --git a/samza-core/src/main/java/org/apache/samza/operators/impl/store/TimestampedValueSerde.java b/samza-core/src/main/java/org/apache/samza/operators/impl/store/TimestampedValueSerde.java index b14f8a4933..5b0cdac625 100644 --- a/samza-core/src/main/java/org/apache/samza/operators/impl/store/TimestampedValueSerde.java +++ b/samza-core/src/main/java/org/apache/samza/operators/impl/store/TimestampedValueSerde.java @@ -21,6 +21,7 @@ import org.apache.samza.serializers.Serde; import java.nio.ByteBuffer; +import org.apache.samza.util.TimestampedValue; public class TimestampedValueSerde implements Serde> { diff --git a/samza-core/src/main/java/org/apache/samza/operators/spec/JoinOperatorSpec.java b/samza-core/src/main/java/org/apache/samza/operators/spec/JoinOperatorSpec.java index 9e058ff424..fe74479d91 100644 --- a/samza-core/src/main/java/org/apache/samza/operators/spec/JoinOperatorSpec.java +++ b/samza-core/src/main/java/org/apache/samza/operators/spec/JoinOperatorSpec.java @@ -23,7 +23,7 @@ import org.apache.samza.operators.functions.TimerFunction; import org.apache.samza.operators.functions.WatermarkFunction; import org.apache.samza.operators.impl.store.TimestampedValueSerde; -import org.apache.samza.operators.impl.store.TimestampedValue; +import org.apache.samza.util.TimestampedValue; import org.apache.samza.serializers.Serde; import java.util.Arrays; diff --git a/samza-core/src/test/java/org/apache/samza/operators/impl/TestOperatorImplGraph.java b/samza-core/src/test/java/org/apache/samza/operators/impl/TestOperatorImplGraph.java index 2d8d1eb57c..7ce75bc6d0 100644 --- a/samza-core/src/test/java/org/apache/samza/operators/impl/TestOperatorImplGraph.java +++ b/samza-core/src/test/java/org/apache/samza/operators/impl/TestOperatorImplGraph.java @@ -44,7 +44,7 @@ import org.apache.samza.operators.functions.FilterFunction; import org.apache.samza.operators.functions.JoinFunction; import org.apache.samza.operators.functions.MapFunction; -import org.apache.samza.operators.impl.store.TimestampedValue; +import org.apache.samza.util.TimestampedValue; import org.apache.samza.operators.spec.OperatorSpec.OpCode; import org.apache.samza.runtime.ApplicationRunner; import org.apache.samza.serializers.IntegerSerde; diff --git a/samza-core/src/test/java/org/apache/samza/operators/impl/store/TestTimeSeriesStoreImpl.java b/samza-core/src/test/java/org/apache/samza/operators/impl/store/TestTimeSeriesStoreImpl.java index 0315a20212..94e171a8ca 100644 --- a/samza-core/src/test/java/org/apache/samza/operators/impl/store/TestTimeSeriesStoreImpl.java +++ b/samza-core/src/test/java/org/apache/samza/operators/impl/store/TestTimeSeriesStoreImpl.java @@ -24,6 +24,7 @@ import org.apache.samza.serializers.StringSerde; import org.apache.samza.storage.kv.ClosableIterator; import org.apache.samza.storage.kv.KeyValueStore; +import org.apache.samza.util.TimestampedValue; import org.junit.Assert; import org.junit.Test; diff --git a/samza-core/src/test/java/org/apache/samza/operators/impl/store/TestTimestampedValueSerde.java b/samza-core/src/test/java/org/apache/samza/operators/impl/store/TestTimestampedValueSerde.java index 40015ec6ee..1621e73df7 100644 --- a/samza-core/src/test/java/org/apache/samza/operators/impl/store/TestTimestampedValueSerde.java +++ b/samza-core/src/test/java/org/apache/samza/operators/impl/store/TestTimestampedValueSerde.java @@ -20,6 +20,7 @@ import org.apache.samza.serializers.ByteSerde; import org.apache.samza.serializers.IntegerSerde; +import org.apache.samza.util.TimestampedValue; import org.junit.Test; import java.nio.ByteBuffer; From bdc85d41c7844897544e74547eddcfa8d4f1a12b Mon Sep 17 00:00:00 2001 From: Ray Matharu Date: Mon, 4 Jun 2018 18:12:49 -0700 Subject: [PATCH 11/33] Removing ValueInfo class and moving TimestampedValue to samza-api util --- .../DefaultListGaugeEvictionPolicy.java | 9 +-- .../org/apache/samza/metrics/ListGauge.java | 21 ++----- .../functions/PartialJoinFunction.java | 2 +- .../operators/impl/OperatorImplGraph.java | 2 +- .../impl/PartialJoinOperatorImpl.java | 2 +- .../operators/impl/WindowOperatorImpl.java | 2 +- .../operators/impl/store/TimeSeriesStore.java | 2 + .../impl/store/TimeSeriesStoreImpl.java | 1 + .../impl/store/TimestampedValue.java | 61 ------------------- .../impl/store/TimestampedValueSerde.java | 1 + .../operators/spec/JoinOperatorSpec.java | 2 +- .../operators/impl/TestOperatorImplGraph.java | 2 +- .../impl/store/TestTimeSeriesStoreImpl.java | 1 + .../impl/store/TestTimestampedValueSerde.java | 1 + 14 files changed, 22 insertions(+), 87 deletions(-) delete mode 100644 samza-core/src/main/java/org/apache/samza/operators/impl/store/TimestampedValue.java diff --git a/samza-api/src/main/java/org/apache/samza/metrics/DefaultListGaugeEvictionPolicy.java b/samza-api/src/main/java/org/apache/samza/metrics/DefaultListGaugeEvictionPolicy.java index 0139f12f93..84fdcf102f 100644 --- a/samza-api/src/main/java/org/apache/samza/metrics/DefaultListGaugeEvictionPolicy.java +++ b/samza-api/src/main/java/org/apache/samza/metrics/DefaultListGaugeEvictionPolicy.java @@ -24,6 +24,7 @@ import java.util.concurrent.Executors; import java.util.concurrent.ScheduledExecutorService; import java.util.concurrent.TimeUnit; +import org.apache.samza.util.TimestampedValue; /** @@ -36,12 +37,12 @@ */ public class DefaultListGaugeEvictionPolicy { - private final Queue> elements; + private final Queue> elements; private final int nItems; private final Duration durationThreshold; private final ScheduledExecutorService scheduledExecutorService; - public DefaultListGaugeEvictionPolicy(Queue> elements, int maxNumberOfItems, + public DefaultListGaugeEvictionPolicy(Queue> elements, int maxNumberOfItems, Duration maxStaleness, Duration period) { this.elements = elements; this.nItems = maxNumberOfItems; @@ -71,11 +72,11 @@ public void run() { Instant currentTimestamp = Instant.now(); synchronized (elements) { - ListGauge.ValueInfo valueInfo = elements.peek(); + TimestampedValue valueInfo = elements.peek(); // continue remove-head if currenttimestamp - head-element's timestamp > durationThreshold while (valueInfo != null - && Duration.between(valueInfo.insertTimestamp, currentTimestamp).compareTo(durationThreshold) > 0) { + && currentTimestamp.toEpochMilli() - valueInfo.getTimestamp() > durationThreshold.toMillis()) { elements.poll(); valueInfo = elements.peek(); } diff --git a/samza-api/src/main/java/org/apache/samza/metrics/ListGauge.java b/samza-api/src/main/java/org/apache/samza/metrics/ListGauge.java index 634c2e8fc5..65ddeba5fc 100644 --- a/samza-api/src/main/java/org/apache/samza/metrics/ListGauge.java +++ b/samza-api/src/main/java/org/apache/samza/metrics/ListGauge.java @@ -25,6 +25,7 @@ import java.util.Queue; import java.util.concurrent.ConcurrentLinkedQueue; import java.util.stream.Collectors; +import org.apache.samza.util.TimestampedValue; /** @@ -40,7 +41,7 @@ */ public class ListGauge implements Metric { private final String name; - private final Queue> elements; + private final Queue> elements; private final DefaultListGaugeEvictionPolicy listGaugeEvictionPolicy; private final static int DEFAULT_MAX_NITEMS = 1000; @@ -57,7 +58,7 @@ public class ListGauge implements Metric { */ public ListGauge(String name, int maxNumberOfItems, Duration maxStaleness, Duration period) { this.name = name; - this.elements = new ConcurrentLinkedQueue>(); + this.elements = new ConcurrentLinkedQueue>(); this.listGaugeEvictionPolicy = new DefaultListGaugeEvictionPolicy(this.elements, maxNumberOfItems, maxStaleness, period); } @@ -83,7 +84,7 @@ public String getName() { * @return the collection of gauge values */ public Collection getValues() { - return Collections.unmodifiableList(this.elements.stream().map(x -> x.value).collect(Collectors.toList())); + return Collections.unmodifiableList(this.elements.stream().map(x -> x.getValue()).collect(Collectors.toList())); } /** @@ -92,7 +93,7 @@ public Collection getValues() { * @param value The Gauge value to be added */ public void add(T value) { - this.elements.add(new ValueInfo(Instant.now(), value)); + this.elements.add(new TimestampedValue(value, Instant.now().toEpochMilli())); // notify the policy object for performing any eviction that may be needed. this.listGaugeEvictionPolicy.elementAddedCallback(); @@ -106,17 +107,5 @@ public void visit(MetricsVisitor visitor) { visitor.listGauge(this); } - /** - * This class is used for bookkeeping of values added to the ListGauge. - * @param - */ - public static class ValueInfo { - public final Instant insertTimestamp; - public final T value; - public ValueInfo(Instant insertTimestamp, T value) { - this.insertTimestamp = insertTimestamp; - this.value = value; - } - } } diff --git a/samza-core/src/main/java/org/apache/samza/operators/functions/PartialJoinFunction.java b/samza-core/src/main/java/org/apache/samza/operators/functions/PartialJoinFunction.java index 5ede5e8f14..038abba9d0 100644 --- a/samza-core/src/main/java/org/apache/samza/operators/functions/PartialJoinFunction.java +++ b/samza-core/src/main/java/org/apache/samza/operators/functions/PartialJoinFunction.java @@ -18,7 +18,7 @@ */ package org.apache.samza.operators.functions; -import org.apache.samza.operators.impl.store.TimestampedValue; +import org.apache.samza.util.TimestampedValue; import org.apache.samza.storage.kv.KeyValueStore; /** diff --git a/samza-core/src/main/java/org/apache/samza/operators/impl/OperatorImplGraph.java b/samza-core/src/main/java/org/apache/samza/operators/impl/OperatorImplGraph.java index bbc878364b..d675822df0 100644 --- a/samza-core/src/main/java/org/apache/samza/operators/impl/OperatorImplGraph.java +++ b/samza-core/src/main/java/org/apache/samza/operators/impl/OperatorImplGraph.java @@ -29,7 +29,7 @@ import org.apache.samza.operators.TimerRegistry; import org.apache.samza.operators.functions.JoinFunction; import org.apache.samza.operators.functions.PartialJoinFunction; -import org.apache.samza.operators.impl.store.TimestampedValue; +import org.apache.samza.util.TimestampedValue; import org.apache.samza.operators.spec.BroadcastOperatorSpec; import org.apache.samza.operators.spec.InputOperatorSpec; import org.apache.samza.operators.spec.JoinOperatorSpec; diff --git a/samza-core/src/main/java/org/apache/samza/operators/impl/PartialJoinOperatorImpl.java b/samza-core/src/main/java/org/apache/samza/operators/impl/PartialJoinOperatorImpl.java index 90a71a0eff..0cdde490f6 100644 --- a/samza-core/src/main/java/org/apache/samza/operators/impl/PartialJoinOperatorImpl.java +++ b/samza-core/src/main/java/org/apache/samza/operators/impl/PartialJoinOperatorImpl.java @@ -21,7 +21,7 @@ import org.apache.samza.SamzaException; import org.apache.samza.config.Config; import org.apache.samza.operators.functions.PartialJoinFunction; -import org.apache.samza.operators.impl.store.TimestampedValue; +import org.apache.samza.util.TimestampedValue; import org.apache.samza.operators.spec.JoinOperatorSpec; import org.apache.samza.operators.spec.OperatorSpec; import org.apache.samza.storage.kv.KeyValueStore; diff --git a/samza-core/src/main/java/org/apache/samza/operators/impl/WindowOperatorImpl.java b/samza-core/src/main/java/org/apache/samza/operators/impl/WindowOperatorImpl.java index 32406cbcf0..32b6e5dd49 100644 --- a/samza-core/src/main/java/org/apache/samza/operators/impl/WindowOperatorImpl.java +++ b/samza-core/src/main/java/org/apache/samza/operators/impl/WindowOperatorImpl.java @@ -26,7 +26,7 @@ import org.apache.samza.operators.impl.store.TimeSeriesKey; import org.apache.samza.operators.impl.store.TimeSeriesStore; import org.apache.samza.operators.impl.store.TimeSeriesStoreImpl; -import org.apache.samza.operators.impl.store.TimestampedValue; +import org.apache.samza.util.TimestampedValue; import org.apache.samza.operators.spec.OperatorSpec; import org.apache.samza.operators.spec.WindowOperatorSpec; import org.apache.samza.operators.triggers.FiringType; diff --git a/samza-core/src/main/java/org/apache/samza/operators/impl/store/TimeSeriesStore.java b/samza-core/src/main/java/org/apache/samza/operators/impl/store/TimeSeriesStore.java index f3d694824a..c9b694d37f 100644 --- a/samza-core/src/main/java/org/apache/samza/operators/impl/store/TimeSeriesStore.java +++ b/samza-core/src/main/java/org/apache/samza/operators/impl/store/TimeSeriesStore.java @@ -21,6 +21,8 @@ package org.apache.samza.operators.impl.store; import org.apache.samza.storage.kv.ClosableIterator; +import org.apache.samza.util.TimestampedValue; + /** * A key-value store that allows entries to be queried and stored based on time ranges. diff --git a/samza-core/src/main/java/org/apache/samza/operators/impl/store/TimeSeriesStoreImpl.java b/samza-core/src/main/java/org/apache/samza/operators/impl/store/TimeSeriesStoreImpl.java index f03d3969f9..10a59679bb 100644 --- a/samza-core/src/main/java/org/apache/samza/operators/impl/store/TimeSeriesStoreImpl.java +++ b/samza-core/src/main/java/org/apache/samza/operators/impl/store/TimeSeriesStoreImpl.java @@ -23,6 +23,7 @@ import org.apache.samza.storage.kv.Entry; import org.apache.samza.storage.kv.KeyValueIterator; import org.apache.samza.storage.kv.KeyValueStore; +import org.apache.samza.util.TimestampedValue; import org.slf4j.Logger; import org.slf4j.LoggerFactory; diff --git a/samza-core/src/main/java/org/apache/samza/operators/impl/store/TimestampedValue.java b/samza-core/src/main/java/org/apache/samza/operators/impl/store/TimestampedValue.java deleted file mode 100644 index 5e451485f3..0000000000 --- a/samza-core/src/main/java/org/apache/samza/operators/impl/store/TimestampedValue.java +++ /dev/null @@ -1,61 +0,0 @@ -/* - * 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.operators.impl.store; - -/** - * An immutable pair of a value, and its corresponding timestamp. - * - * @param the type of the value - */ -public class TimestampedValue { - private final V value; - private final long timestamp; - - public TimestampedValue(V v, long timestamp) { - this.value = v; - this.timestamp = timestamp; - } - - public V getValue() { - return value; - } - - public long getTimestamp() { - return timestamp; - } - - @Override - public boolean equals(Object o) { - if (this == o) return true; - if (o == null || !getClass().equals(o.getClass())) return false; - - TimestampedValue that = (TimestampedValue) o; - - if (timestamp != that.timestamp) return false; - return value != null ? value.equals(that.value) : (that.value == null); - } - - @Override - public int hashCode() { - int result = value != null ? value.hashCode() : 0; - result = 31 * result + (int) (timestamp ^ (timestamp >>> 32)); - return result; - } -} diff --git a/samza-core/src/main/java/org/apache/samza/operators/impl/store/TimestampedValueSerde.java b/samza-core/src/main/java/org/apache/samza/operators/impl/store/TimestampedValueSerde.java index b14f8a4933..5b0cdac625 100644 --- a/samza-core/src/main/java/org/apache/samza/operators/impl/store/TimestampedValueSerde.java +++ b/samza-core/src/main/java/org/apache/samza/operators/impl/store/TimestampedValueSerde.java @@ -21,6 +21,7 @@ import org.apache.samza.serializers.Serde; import java.nio.ByteBuffer; +import org.apache.samza.util.TimestampedValue; public class TimestampedValueSerde implements Serde> { diff --git a/samza-core/src/main/java/org/apache/samza/operators/spec/JoinOperatorSpec.java b/samza-core/src/main/java/org/apache/samza/operators/spec/JoinOperatorSpec.java index 9e058ff424..fe74479d91 100644 --- a/samza-core/src/main/java/org/apache/samza/operators/spec/JoinOperatorSpec.java +++ b/samza-core/src/main/java/org/apache/samza/operators/spec/JoinOperatorSpec.java @@ -23,7 +23,7 @@ import org.apache.samza.operators.functions.TimerFunction; import org.apache.samza.operators.functions.WatermarkFunction; import org.apache.samza.operators.impl.store.TimestampedValueSerde; -import org.apache.samza.operators.impl.store.TimestampedValue; +import org.apache.samza.util.TimestampedValue; import org.apache.samza.serializers.Serde; import java.util.Arrays; diff --git a/samza-core/src/test/java/org/apache/samza/operators/impl/TestOperatorImplGraph.java b/samza-core/src/test/java/org/apache/samza/operators/impl/TestOperatorImplGraph.java index 2d8d1eb57c..7ce75bc6d0 100644 --- a/samza-core/src/test/java/org/apache/samza/operators/impl/TestOperatorImplGraph.java +++ b/samza-core/src/test/java/org/apache/samza/operators/impl/TestOperatorImplGraph.java @@ -44,7 +44,7 @@ import org.apache.samza.operators.functions.FilterFunction; import org.apache.samza.operators.functions.JoinFunction; import org.apache.samza.operators.functions.MapFunction; -import org.apache.samza.operators.impl.store.TimestampedValue; +import org.apache.samza.util.TimestampedValue; import org.apache.samza.operators.spec.OperatorSpec.OpCode; import org.apache.samza.runtime.ApplicationRunner; import org.apache.samza.serializers.IntegerSerde; diff --git a/samza-core/src/test/java/org/apache/samza/operators/impl/store/TestTimeSeriesStoreImpl.java b/samza-core/src/test/java/org/apache/samza/operators/impl/store/TestTimeSeriesStoreImpl.java index 0315a20212..94e171a8ca 100644 --- a/samza-core/src/test/java/org/apache/samza/operators/impl/store/TestTimeSeriesStoreImpl.java +++ b/samza-core/src/test/java/org/apache/samza/operators/impl/store/TestTimeSeriesStoreImpl.java @@ -24,6 +24,7 @@ import org.apache.samza.serializers.StringSerde; import org.apache.samza.storage.kv.ClosableIterator; import org.apache.samza.storage.kv.KeyValueStore; +import org.apache.samza.util.TimestampedValue; import org.junit.Assert; import org.junit.Test; diff --git a/samza-core/src/test/java/org/apache/samza/operators/impl/store/TestTimestampedValueSerde.java b/samza-core/src/test/java/org/apache/samza/operators/impl/store/TestTimestampedValueSerde.java index 40015ec6ee..1621e73df7 100644 --- a/samza-core/src/test/java/org/apache/samza/operators/impl/store/TestTimestampedValueSerde.java +++ b/samza-core/src/test/java/org/apache/samza/operators/impl/store/TestTimestampedValueSerde.java @@ -20,6 +20,7 @@ import org.apache.samza.serializers.ByteSerde; import org.apache.samza.serializers.IntegerSerde; +import org.apache.samza.util.TimestampedValue; import org.junit.Test; import java.nio.ByteBuffer; From 5c1ed0a40d1f9d35a75dae7e065be036d3fae027 Mon Sep 17 00:00:00 2001 From: Ray Matharu Date: Mon, 4 Jun 2018 18:12:49 -0700 Subject: [PATCH 12/33] Removing ValueInfo class and moving TimestampedValue to samza-api util Removing ValueInfo class and moving TimestampedValue to samza-api util Adding TimestampedValue.java in core --- .../DefaultListGaugeEvictionPolicy.java | 9 ++++---- .../org/apache/samza/metrics/ListGauge.java | 21 +++++-------------- .../apache/samza/util}/TimestampedValue.java | 2 +- .../functions/PartialJoinFunction.java | 2 +- .../operators/impl/OperatorImplGraph.java | 2 +- .../impl/PartialJoinOperatorImpl.java | 2 +- .../operators/impl/WindowOperatorImpl.java | 2 +- .../operators/impl/store/TimeSeriesStore.java | 2 ++ .../impl/store/TimeSeriesStoreImpl.java | 1 + .../impl/store/TimestampedValueSerde.java | 1 + .../operators/spec/JoinOperatorSpec.java | 2 +- .../operators/impl/TestOperatorImplGraph.java | 2 +- .../impl/store/TestTimeSeriesStoreImpl.java | 1 + .../impl/store/TestTimestampedValueSerde.java | 1 + 14 files changed, 23 insertions(+), 27 deletions(-) rename {samza-core/src/main/java/org/apache/samza/operators/impl/store => samza-api/src/main/java/org/apache/samza/util}/TimestampedValue.java (97%) diff --git a/samza-api/src/main/java/org/apache/samza/metrics/DefaultListGaugeEvictionPolicy.java b/samza-api/src/main/java/org/apache/samza/metrics/DefaultListGaugeEvictionPolicy.java index 0139f12f93..84fdcf102f 100644 --- a/samza-api/src/main/java/org/apache/samza/metrics/DefaultListGaugeEvictionPolicy.java +++ b/samza-api/src/main/java/org/apache/samza/metrics/DefaultListGaugeEvictionPolicy.java @@ -24,6 +24,7 @@ import java.util.concurrent.Executors; import java.util.concurrent.ScheduledExecutorService; import java.util.concurrent.TimeUnit; +import org.apache.samza.util.TimestampedValue; /** @@ -36,12 +37,12 @@ */ public class DefaultListGaugeEvictionPolicy { - private final Queue> elements; + private final Queue> elements; private final int nItems; private final Duration durationThreshold; private final ScheduledExecutorService scheduledExecutorService; - public DefaultListGaugeEvictionPolicy(Queue> elements, int maxNumberOfItems, + public DefaultListGaugeEvictionPolicy(Queue> elements, int maxNumberOfItems, Duration maxStaleness, Duration period) { this.elements = elements; this.nItems = maxNumberOfItems; @@ -71,11 +72,11 @@ public void run() { Instant currentTimestamp = Instant.now(); synchronized (elements) { - ListGauge.ValueInfo valueInfo = elements.peek(); + TimestampedValue valueInfo = elements.peek(); // continue remove-head if currenttimestamp - head-element's timestamp > durationThreshold while (valueInfo != null - && Duration.between(valueInfo.insertTimestamp, currentTimestamp).compareTo(durationThreshold) > 0) { + && currentTimestamp.toEpochMilli() - valueInfo.getTimestamp() > durationThreshold.toMillis()) { elements.poll(); valueInfo = elements.peek(); } diff --git a/samza-api/src/main/java/org/apache/samza/metrics/ListGauge.java b/samza-api/src/main/java/org/apache/samza/metrics/ListGauge.java index 634c2e8fc5..65ddeba5fc 100644 --- a/samza-api/src/main/java/org/apache/samza/metrics/ListGauge.java +++ b/samza-api/src/main/java/org/apache/samza/metrics/ListGauge.java @@ -25,6 +25,7 @@ import java.util.Queue; import java.util.concurrent.ConcurrentLinkedQueue; import java.util.stream.Collectors; +import org.apache.samza.util.TimestampedValue; /** @@ -40,7 +41,7 @@ */ public class ListGauge implements Metric { private final String name; - private final Queue> elements; + private final Queue> elements; private final DefaultListGaugeEvictionPolicy listGaugeEvictionPolicy; private final static int DEFAULT_MAX_NITEMS = 1000; @@ -57,7 +58,7 @@ public class ListGauge implements Metric { */ public ListGauge(String name, int maxNumberOfItems, Duration maxStaleness, Duration period) { this.name = name; - this.elements = new ConcurrentLinkedQueue>(); + this.elements = new ConcurrentLinkedQueue>(); this.listGaugeEvictionPolicy = new DefaultListGaugeEvictionPolicy(this.elements, maxNumberOfItems, maxStaleness, period); } @@ -83,7 +84,7 @@ public String getName() { * @return the collection of gauge values */ public Collection getValues() { - return Collections.unmodifiableList(this.elements.stream().map(x -> x.value).collect(Collectors.toList())); + return Collections.unmodifiableList(this.elements.stream().map(x -> x.getValue()).collect(Collectors.toList())); } /** @@ -92,7 +93,7 @@ public Collection getValues() { * @param value The Gauge value to be added */ public void add(T value) { - this.elements.add(new ValueInfo(Instant.now(), value)); + this.elements.add(new TimestampedValue(value, Instant.now().toEpochMilli())); // notify the policy object for performing any eviction that may be needed. this.listGaugeEvictionPolicy.elementAddedCallback(); @@ -106,17 +107,5 @@ public void visit(MetricsVisitor visitor) { visitor.listGauge(this); } - /** - * This class is used for bookkeeping of values added to the ListGauge. - * @param - */ - public static class ValueInfo { - public final Instant insertTimestamp; - public final T value; - public ValueInfo(Instant insertTimestamp, T value) { - this.insertTimestamp = insertTimestamp; - this.value = value; - } - } } diff --git a/samza-core/src/main/java/org/apache/samza/operators/impl/store/TimestampedValue.java b/samza-api/src/main/java/org/apache/samza/util/TimestampedValue.java similarity index 97% rename from samza-core/src/main/java/org/apache/samza/operators/impl/store/TimestampedValue.java rename to samza-api/src/main/java/org/apache/samza/util/TimestampedValue.java index 5e451485f3..e767918eb7 100644 --- a/samza-core/src/main/java/org/apache/samza/operators/impl/store/TimestampedValue.java +++ b/samza-api/src/main/java/org/apache/samza/util/TimestampedValue.java @@ -17,7 +17,7 @@ * under the License. * */ -package org.apache.samza.operators.impl.store; +package org.apache.samza.util; /** * An immutable pair of a value, and its corresponding timestamp. diff --git a/samza-core/src/main/java/org/apache/samza/operators/functions/PartialJoinFunction.java b/samza-core/src/main/java/org/apache/samza/operators/functions/PartialJoinFunction.java index 5ede5e8f14..038abba9d0 100644 --- a/samza-core/src/main/java/org/apache/samza/operators/functions/PartialJoinFunction.java +++ b/samza-core/src/main/java/org/apache/samza/operators/functions/PartialJoinFunction.java @@ -18,7 +18,7 @@ */ package org.apache.samza.operators.functions; -import org.apache.samza.operators.impl.store.TimestampedValue; +import org.apache.samza.util.TimestampedValue; import org.apache.samza.storage.kv.KeyValueStore; /** diff --git a/samza-core/src/main/java/org/apache/samza/operators/impl/OperatorImplGraph.java b/samza-core/src/main/java/org/apache/samza/operators/impl/OperatorImplGraph.java index bbc878364b..d675822df0 100644 --- a/samza-core/src/main/java/org/apache/samza/operators/impl/OperatorImplGraph.java +++ b/samza-core/src/main/java/org/apache/samza/operators/impl/OperatorImplGraph.java @@ -29,7 +29,7 @@ import org.apache.samza.operators.TimerRegistry; import org.apache.samza.operators.functions.JoinFunction; import org.apache.samza.operators.functions.PartialJoinFunction; -import org.apache.samza.operators.impl.store.TimestampedValue; +import org.apache.samza.util.TimestampedValue; import org.apache.samza.operators.spec.BroadcastOperatorSpec; import org.apache.samza.operators.spec.InputOperatorSpec; import org.apache.samza.operators.spec.JoinOperatorSpec; diff --git a/samza-core/src/main/java/org/apache/samza/operators/impl/PartialJoinOperatorImpl.java b/samza-core/src/main/java/org/apache/samza/operators/impl/PartialJoinOperatorImpl.java index 90a71a0eff..0cdde490f6 100644 --- a/samza-core/src/main/java/org/apache/samza/operators/impl/PartialJoinOperatorImpl.java +++ b/samza-core/src/main/java/org/apache/samza/operators/impl/PartialJoinOperatorImpl.java @@ -21,7 +21,7 @@ import org.apache.samza.SamzaException; import org.apache.samza.config.Config; import org.apache.samza.operators.functions.PartialJoinFunction; -import org.apache.samza.operators.impl.store.TimestampedValue; +import org.apache.samza.util.TimestampedValue; import org.apache.samza.operators.spec.JoinOperatorSpec; import org.apache.samza.operators.spec.OperatorSpec; import org.apache.samza.storage.kv.KeyValueStore; diff --git a/samza-core/src/main/java/org/apache/samza/operators/impl/WindowOperatorImpl.java b/samza-core/src/main/java/org/apache/samza/operators/impl/WindowOperatorImpl.java index 32406cbcf0..32b6e5dd49 100644 --- a/samza-core/src/main/java/org/apache/samza/operators/impl/WindowOperatorImpl.java +++ b/samza-core/src/main/java/org/apache/samza/operators/impl/WindowOperatorImpl.java @@ -26,7 +26,7 @@ import org.apache.samza.operators.impl.store.TimeSeriesKey; import org.apache.samza.operators.impl.store.TimeSeriesStore; import org.apache.samza.operators.impl.store.TimeSeriesStoreImpl; -import org.apache.samza.operators.impl.store.TimestampedValue; +import org.apache.samza.util.TimestampedValue; import org.apache.samza.operators.spec.OperatorSpec; import org.apache.samza.operators.spec.WindowOperatorSpec; import org.apache.samza.operators.triggers.FiringType; diff --git a/samza-core/src/main/java/org/apache/samza/operators/impl/store/TimeSeriesStore.java b/samza-core/src/main/java/org/apache/samza/operators/impl/store/TimeSeriesStore.java index f3d694824a..c9b694d37f 100644 --- a/samza-core/src/main/java/org/apache/samza/operators/impl/store/TimeSeriesStore.java +++ b/samza-core/src/main/java/org/apache/samza/operators/impl/store/TimeSeriesStore.java @@ -21,6 +21,8 @@ package org.apache.samza.operators.impl.store; import org.apache.samza.storage.kv.ClosableIterator; +import org.apache.samza.util.TimestampedValue; + /** * A key-value store that allows entries to be queried and stored based on time ranges. diff --git a/samza-core/src/main/java/org/apache/samza/operators/impl/store/TimeSeriesStoreImpl.java b/samza-core/src/main/java/org/apache/samza/operators/impl/store/TimeSeriesStoreImpl.java index f03d3969f9..10a59679bb 100644 --- a/samza-core/src/main/java/org/apache/samza/operators/impl/store/TimeSeriesStoreImpl.java +++ b/samza-core/src/main/java/org/apache/samza/operators/impl/store/TimeSeriesStoreImpl.java @@ -23,6 +23,7 @@ import org.apache.samza.storage.kv.Entry; import org.apache.samza.storage.kv.KeyValueIterator; import org.apache.samza.storage.kv.KeyValueStore; +import org.apache.samza.util.TimestampedValue; import org.slf4j.Logger; import org.slf4j.LoggerFactory; diff --git a/samza-core/src/main/java/org/apache/samza/operators/impl/store/TimestampedValueSerde.java b/samza-core/src/main/java/org/apache/samza/operators/impl/store/TimestampedValueSerde.java index b14f8a4933..5b0cdac625 100644 --- a/samza-core/src/main/java/org/apache/samza/operators/impl/store/TimestampedValueSerde.java +++ b/samza-core/src/main/java/org/apache/samza/operators/impl/store/TimestampedValueSerde.java @@ -21,6 +21,7 @@ import org.apache.samza.serializers.Serde; import java.nio.ByteBuffer; +import org.apache.samza.util.TimestampedValue; public class TimestampedValueSerde implements Serde> { diff --git a/samza-core/src/main/java/org/apache/samza/operators/spec/JoinOperatorSpec.java b/samza-core/src/main/java/org/apache/samza/operators/spec/JoinOperatorSpec.java index 9e058ff424..fe74479d91 100644 --- a/samza-core/src/main/java/org/apache/samza/operators/spec/JoinOperatorSpec.java +++ b/samza-core/src/main/java/org/apache/samza/operators/spec/JoinOperatorSpec.java @@ -23,7 +23,7 @@ import org.apache.samza.operators.functions.TimerFunction; import org.apache.samza.operators.functions.WatermarkFunction; import org.apache.samza.operators.impl.store.TimestampedValueSerde; -import org.apache.samza.operators.impl.store.TimestampedValue; +import org.apache.samza.util.TimestampedValue; import org.apache.samza.serializers.Serde; import java.util.Arrays; diff --git a/samza-core/src/test/java/org/apache/samza/operators/impl/TestOperatorImplGraph.java b/samza-core/src/test/java/org/apache/samza/operators/impl/TestOperatorImplGraph.java index 2d8d1eb57c..7ce75bc6d0 100644 --- a/samza-core/src/test/java/org/apache/samza/operators/impl/TestOperatorImplGraph.java +++ b/samza-core/src/test/java/org/apache/samza/operators/impl/TestOperatorImplGraph.java @@ -44,7 +44,7 @@ import org.apache.samza.operators.functions.FilterFunction; import org.apache.samza.operators.functions.JoinFunction; import org.apache.samza.operators.functions.MapFunction; -import org.apache.samza.operators.impl.store.TimestampedValue; +import org.apache.samza.util.TimestampedValue; import org.apache.samza.operators.spec.OperatorSpec.OpCode; import org.apache.samza.runtime.ApplicationRunner; import org.apache.samza.serializers.IntegerSerde; diff --git a/samza-core/src/test/java/org/apache/samza/operators/impl/store/TestTimeSeriesStoreImpl.java b/samza-core/src/test/java/org/apache/samza/operators/impl/store/TestTimeSeriesStoreImpl.java index 0315a20212..94e171a8ca 100644 --- a/samza-core/src/test/java/org/apache/samza/operators/impl/store/TestTimeSeriesStoreImpl.java +++ b/samza-core/src/test/java/org/apache/samza/operators/impl/store/TestTimeSeriesStoreImpl.java @@ -24,6 +24,7 @@ import org.apache.samza.serializers.StringSerde; import org.apache.samza.storage.kv.ClosableIterator; import org.apache.samza.storage.kv.KeyValueStore; +import org.apache.samza.util.TimestampedValue; import org.junit.Assert; import org.junit.Test; diff --git a/samza-core/src/test/java/org/apache/samza/operators/impl/store/TestTimestampedValueSerde.java b/samza-core/src/test/java/org/apache/samza/operators/impl/store/TestTimestampedValueSerde.java index 40015ec6ee..1621e73df7 100644 --- a/samza-core/src/test/java/org/apache/samza/operators/impl/store/TestTimestampedValueSerde.java +++ b/samza-core/src/test/java/org/apache/samza/operators/impl/store/TestTimestampedValueSerde.java @@ -20,6 +20,7 @@ import org.apache.samza.serializers.ByteSerde; import org.apache.samza.serializers.IntegerSerde; +import org.apache.samza.util.TimestampedValue; import org.junit.Test; import java.nio.ByteBuffer; From 508b6b33042dd476610c29bcf92961bca26ab6b9 Mon Sep 17 00:00:00 2001 From: Ray Matharu Date: Mon, 4 Jun 2018 18:41:55 -0700 Subject: [PATCH 13/33] Renaming diagnostics.appender.enable to samza.diagnostics.enabled --- .../main/scala/org/apache/samza/container/SamzaContainer.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/samza-core/src/main/scala/org/apache/samza/container/SamzaContainer.scala b/samza-core/src/main/scala/org/apache/samza/container/SamzaContainer.scala index 623cf7ddce..5c1c2f05cc 100644 --- a/samza-core/src/main/scala/org/apache/samza/container/SamzaContainer.scala +++ b/samza-core/src/main/scala/org/apache/samza/container/SamzaContainer.scala @@ -874,7 +874,7 @@ class SamzaContainer( def startDiagnostics { // TODO: where should this reside, MetricConfig? Log4jSystemConfig? or a new separate config? - val DIAGNOSTICS_APPENDER_ENABLE = "diagnostics.appender.enable" + val DIAGNOSTICS_APPENDER_ENABLE = "samza.diagnostics.enabled" if (containerContext.config.getBoolean(DIAGNOSTICS_APPENDER_ENABLE, false)) { import org.apache.log4j.Logger val rootLogger = Logger.getRootLogger From c7821baea18f7bd44ed9280dfa0f29cca98f67bf Mon Sep 17 00:00:00 2001 From: Ray Matharu Date: Tue, 5 Jun 2018 11:42:03 -0700 Subject: [PATCH 14/33] Removing periodic pruning from Listgauge, pruning on add and get in ListGauge --- .../DefaultListGaugeEvictionPolicy.java | 71 ++++++++----------- .../org/apache/samza/metrics/ListGauge.java | 18 +++-- 2 files changed, 37 insertions(+), 52 deletions(-) diff --git a/samza-api/src/main/java/org/apache/samza/metrics/DefaultListGaugeEvictionPolicy.java b/samza-api/src/main/java/org/apache/samza/metrics/DefaultListGaugeEvictionPolicy.java index 84fdcf102f..67b7c6410d 100644 --- a/samza-api/src/main/java/org/apache/samza/metrics/DefaultListGaugeEvictionPolicy.java +++ b/samza-api/src/main/java/org/apache/samza/metrics/DefaultListGaugeEvictionPolicy.java @@ -21,9 +21,6 @@ import java.time.Duration; import java.time.Instant; import java.util.Queue; -import java.util.concurrent.Executors; -import java.util.concurrent.ScheduledExecutorService; -import java.util.concurrent.TimeUnit; import org.apache.samza.util.TimestampedValue; @@ -33,54 +30,44 @@ * b.) There are elements which have timestamps which are stale as compared to currentTime (the staleness bound is * specified as maxStaleness). * - * This naive implementation uses a periodic thread with a configurable period. */ public class DefaultListGaugeEvictionPolicy { - private final Queue> elements; - private final int nItems; - private final Duration durationThreshold; - private final ScheduledExecutorService scheduledExecutorService; - - public DefaultListGaugeEvictionPolicy(Queue> elements, int maxNumberOfItems, - Duration maxStaleness, Duration period) { - this.elements = elements; - this.nItems = maxNumberOfItems; - this.durationThreshold = maxStaleness; - this.scheduledExecutorService = Executors.newSingleThreadScheduledExecutor(); - this.scheduledExecutorService.schedule(new EvictionRunnable(), period.toMillis(), TimeUnit.MILLISECONDS); + /** + * Evicts entries from the elements list, based on the given item-size and durationThreshold. + * Callers are responsible for thread-safety. + */ + public void evict(Queue> elements, int maxNumberOfItems, Duration maxStaleness) { + this.evictBasedOnSize(elements, maxNumberOfItems); + this.evictBasedOnTimestamp(elements, maxStaleness); } - public void elementAddedCallback() { - - // need to synchronize here because this thread could be concurrent with the runnable thread and can - // cause two vals to be removed (wrong eviction) even if a threadsafe queue was used. - synchronized (this.elements) { - int numToEvict = this.elements.size() - nItems; - - while (numToEvict > 0) { - this.elements.poll(); // remove head - numToEvict--; - } + /** + * Evicts entries from elements in FIFO order until it has maxNumberOfItems + * @param elements queue to evict elements from + * @param maxNumberOfItems max number of items to be left in the queue + */ + private void evictBasedOnSize(Queue> elements, int maxNumberOfItems) { + int numToEvict = elements.size() - maxNumberOfItems; + while (numToEvict > 0) { + elements.poll(); // remove head + numToEvict--; } } - private class EvictionRunnable implements Runnable { - - @Override - public void run() { - Instant currentTimestamp = Instant.now(); - - synchronized (elements) { - TimestampedValue valueInfo = elements.peek(); + /** + * Removes entries from elements to ensure no element has a timestamp more than maxStaleness before current timestamp. + * @param elements the queue to evict elements from + * @param maxStaleness max staleness permitted in elements + */ + private void evictBasedOnTimestamp(Queue> elements, Duration maxStaleness) { + Instant currentTimestamp = Instant.now(); + TimestampedValue valueInfo = elements.peek(); - // continue remove-head if currenttimestamp - head-element's timestamp > durationThreshold - while (valueInfo != null - && currentTimestamp.toEpochMilli() - valueInfo.getTimestamp() > durationThreshold.toMillis()) { - elements.poll(); - valueInfo = elements.peek(); - } - } + // continue remove-head if currenttimestamp - head-element's timestamp > durationThreshold + while (valueInfo != null && currentTimestamp.toEpochMilli() - valueInfo.getTimestamp() > maxStaleness.toMillis()) { + elements.poll(); + valueInfo = elements.peek(); } } } diff --git a/samza-api/src/main/java/org/apache/samza/metrics/ListGauge.java b/samza-api/src/main/java/org/apache/samza/metrics/ListGauge.java index 65ddeba5fc..e334756a7a 100644 --- a/samza-api/src/main/java/org/apache/samza/metrics/ListGauge.java +++ b/samza-api/src/main/java/org/apache/samza/metrics/ListGauge.java @@ -42,11 +42,10 @@ public class ListGauge implements Metric { private final String name; private final Queue> elements; - private final DefaultListGaugeEvictionPolicy listGaugeEvictionPolicy; + private final DefaultListGaugeEvictionPolicy listGaugeEvictionPolicy; private final static int DEFAULT_MAX_NITEMS = 1000; private final static Duration DEFAULT_MAX_STALENESS = Duration.ofMinutes(60); - private final static Duration DEFAULT_EVICTION_CHECK_PERIOD = Duration.ofMinutes(1); /** * Create a new {@link ListGauge} that auto evicts based on the given maxNumberOfItems, maxStaleness, and period parameters. @@ -54,13 +53,11 @@ public class ListGauge implements Metric { * @param name Name to be assigned * @param maxNumberOfItems The max number of items that can remain in the listgauge * @param maxStaleness The max staleness of items permitted in the listgauge - * @param period The periodicity with which the listGauge would be checked for stale values. */ - public ListGauge(String name, int maxNumberOfItems, Duration maxStaleness, Duration period) { + public ListGauge(String name, int maxNumberOfItems, Duration maxStaleness) { this.name = name; this.elements = new ConcurrentLinkedQueue>(); - this.listGaugeEvictionPolicy = - new DefaultListGaugeEvictionPolicy(this.elements, maxNumberOfItems, maxStaleness, period); + this.listGaugeEvictionPolicy = new DefaultListGaugeEvictionPolicy(); } /** @@ -68,7 +65,7 @@ public ListGauge(String name, int maxNumberOfItems, Duration maxStaleness, Durat * @param name Name to be assigned */ public ListGauge(String name) { - this(name, DEFAULT_MAX_NITEMS, DEFAULT_MAX_STALENESS, DEFAULT_EVICTION_CHECK_PERIOD); + this(name, DEFAULT_MAX_NITEMS, DEFAULT_MAX_STALENESS); } /** @@ -81,9 +78,12 @@ public String getName() { /** * Get the Collection of Gauge values currently in the list, used when serializing this Gauge. + * Also evicts values based on the configured maxItems and maxStaleness. * @return the collection of gauge values */ public Collection getValues() { + // notify the policy object for performing any eviction that may be needed. + this.listGaugeEvictionPolicy.evict(this.elements, DEFAULT_MAX_NITEMS, DEFAULT_MAX_STALENESS); return Collections.unmodifiableList(this.elements.stream().map(x -> x.getValue()).collect(Collectors.toList())); } @@ -96,7 +96,7 @@ public void add(T value) { this.elements.add(new TimestampedValue(value, Instant.now().toEpochMilli())); // notify the policy object for performing any eviction that may be needed. - this.listGaugeEvictionPolicy.elementAddedCallback(); + this.listGaugeEvictionPolicy.evict(this.elements, DEFAULT_MAX_NITEMS, DEFAULT_MAX_STALENESS); } /** @@ -106,6 +106,4 @@ public void add(T value) { public void visit(MetricsVisitor visitor) { visitor.listGauge(this); } - - } From 9ba8e5b3bdc6324440e7d42f601bd3f7f460b702 Mon Sep 17 00:00:00 2001 From: Ray Matharu Date: Tue, 5 Jun 2018 12:36:09 -0700 Subject: [PATCH 15/33] Updating testListGauge for new listgauge --- .../src/main/java/org/apache/samza/metrics/ListGauge.java | 8 ++++++-- .../test/java/org/apache/samza/metrics/TestListGauge.java | 2 +- 2 files changed, 7 insertions(+), 3 deletions(-) diff --git a/samza-api/src/main/java/org/apache/samza/metrics/ListGauge.java b/samza-api/src/main/java/org/apache/samza/metrics/ListGauge.java index e334756a7a..27631e1190 100644 --- a/samza-api/src/main/java/org/apache/samza/metrics/ListGauge.java +++ b/samza-api/src/main/java/org/apache/samza/metrics/ListGauge.java @@ -44,6 +44,8 @@ public class ListGauge implements Metric { private final Queue> elements; private final DefaultListGaugeEvictionPolicy listGaugeEvictionPolicy; + private final int maxNumberOfItems; + private final Duration maxStaleness; private final static int DEFAULT_MAX_NITEMS = 1000; private final static Duration DEFAULT_MAX_STALENESS = Duration.ofMinutes(60); @@ -57,6 +59,8 @@ public class ListGauge implements Metric { public ListGauge(String name, int maxNumberOfItems, Duration maxStaleness) { this.name = name; this.elements = new ConcurrentLinkedQueue>(); + this.maxNumberOfItems = maxNumberOfItems; + this.maxStaleness = maxStaleness; this.listGaugeEvictionPolicy = new DefaultListGaugeEvictionPolicy(); } @@ -83,7 +87,7 @@ public String getName() { */ public Collection getValues() { // notify the policy object for performing any eviction that may be needed. - this.listGaugeEvictionPolicy.evict(this.elements, DEFAULT_MAX_NITEMS, DEFAULT_MAX_STALENESS); + this.listGaugeEvictionPolicy.evict(this.elements, this.maxNumberOfItems, this.maxStaleness); return Collections.unmodifiableList(this.elements.stream().map(x -> x.getValue()).collect(Collectors.toList())); } @@ -96,7 +100,7 @@ public void add(T value) { this.elements.add(new TimestampedValue(value, Instant.now().toEpochMilli())); // notify the policy object for performing any eviction that may be needed. - this.listGaugeEvictionPolicy.evict(this.elements, DEFAULT_MAX_NITEMS, DEFAULT_MAX_STALENESS); + this.listGaugeEvictionPolicy.evict(this.elements, this.maxNumberOfItems, this.maxStaleness); } /** diff --git a/samza-api/src/test/java/org/apache/samza/metrics/TestListGauge.java b/samza-api/src/test/java/org/apache/samza/metrics/TestListGauge.java index 74e1fd02a8..eb9101238c 100644 --- a/samza-api/src/test/java/org/apache/samza/metrics/TestListGauge.java +++ b/samza-api/src/test/java/org/apache/samza/metrics/TestListGauge.java @@ -33,7 +33,7 @@ public class TestListGauge { private final static Duration THREAD_TEST_TIMEOUT = Duration.ofSeconds(10); private ListGauge getListGaugeForTest() { - return new ListGauge("sampleListGauge", 10, Duration.ofSeconds(60), Duration.ofSeconds(60)); + return new ListGauge("sampleListGauge", 10, Duration.ofSeconds(60)); } @Test From fd77c5315b72979b176021b7aa544ed110a914a7 Mon Sep 17 00:00:00 2001 From: Ray Matharu Date: Wed, 6 Jun 2018 14:38:07 -0700 Subject: [PATCH 16/33] Complete removing evictionpolicy separation, merging all eviction logic into listgauge --- .../DefaultListGaugeEvictionPolicy.java | 73 ------------------- .../org/apache/samza/metrics/ListGauge.java | 44 ++++++++++- 2 files changed, 40 insertions(+), 77 deletions(-) delete mode 100644 samza-api/src/main/java/org/apache/samza/metrics/DefaultListGaugeEvictionPolicy.java diff --git a/samza-api/src/main/java/org/apache/samza/metrics/DefaultListGaugeEvictionPolicy.java b/samza-api/src/main/java/org/apache/samza/metrics/DefaultListGaugeEvictionPolicy.java deleted file mode 100644 index 67b7c6410d..0000000000 --- a/samza-api/src/main/java/org/apache/samza/metrics/DefaultListGaugeEvictionPolicy.java +++ /dev/null @@ -1,73 +0,0 @@ -/* - * 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; - -import java.time.Duration; -import java.time.Instant; -import java.util.Queue; -import org.apache.samza.util.TimestampedValue; - - -/** - * Provides an eviction policy that evicts entries from the elements if - * a.) There are more elements in the elements than the specified maxNumberOfItems (removal in FIFO order), or - * b.) There are elements which have timestamps which are stale as compared to currentTime (the staleness bound is - * specified as maxStaleness). - * - */ -public class DefaultListGaugeEvictionPolicy { - - /** - * Evicts entries from the elements list, based on the given item-size and durationThreshold. - * Callers are responsible for thread-safety. - */ - public void evict(Queue> elements, int maxNumberOfItems, Duration maxStaleness) { - this.evictBasedOnSize(elements, maxNumberOfItems); - this.evictBasedOnTimestamp(elements, maxStaleness); - } - - /** - * Evicts entries from elements in FIFO order until it has maxNumberOfItems - * @param elements queue to evict elements from - * @param maxNumberOfItems max number of items to be left in the queue - */ - private void evictBasedOnSize(Queue> elements, int maxNumberOfItems) { - int numToEvict = elements.size() - maxNumberOfItems; - while (numToEvict > 0) { - elements.poll(); // remove head - numToEvict--; - } - } - - /** - * Removes entries from elements to ensure no element has a timestamp more than maxStaleness before current timestamp. - * @param elements the queue to evict elements from - * @param maxStaleness max staleness permitted in elements - */ - private void evictBasedOnTimestamp(Queue> elements, Duration maxStaleness) { - Instant currentTimestamp = Instant.now(); - TimestampedValue valueInfo = elements.peek(); - - // continue remove-head if currenttimestamp - head-element's timestamp > durationThreshold - while (valueInfo != null && currentTimestamp.toEpochMilli() - valueInfo.getTimestamp() > maxStaleness.toMillis()) { - elements.poll(); - valueInfo = elements.peek(); - } - } -} diff --git a/samza-api/src/main/java/org/apache/samza/metrics/ListGauge.java b/samza-api/src/main/java/org/apache/samza/metrics/ListGauge.java index 27631e1190..d25934bc61 100644 --- a/samza-api/src/main/java/org/apache/samza/metrics/ListGauge.java +++ b/samza-api/src/main/java/org/apache/samza/metrics/ListGauge.java @@ -42,7 +42,6 @@ public class ListGauge implements Metric { private final String name; private final Queue> elements; - private final DefaultListGaugeEvictionPolicy listGaugeEvictionPolicy; private final int maxNumberOfItems; private final Duration maxStaleness; @@ -61,7 +60,6 @@ public ListGauge(String name, int maxNumberOfItems, Duration maxStaleness) { this.elements = new ConcurrentLinkedQueue>(); this.maxNumberOfItems = maxNumberOfItems; this.maxStaleness = maxStaleness; - this.listGaugeEvictionPolicy = new DefaultListGaugeEvictionPolicy(); } /** @@ -87,7 +85,7 @@ public String getName() { */ public Collection getValues() { // notify the policy object for performing any eviction that may be needed. - this.listGaugeEvictionPolicy.evict(this.elements, this.maxNumberOfItems, this.maxStaleness); + this.evict(this.elements, this.maxNumberOfItems, this.maxStaleness); return Collections.unmodifiableList(this.elements.stream().map(x -> x.getValue()).collect(Collectors.toList())); } @@ -100,7 +98,7 @@ public void add(T value) { this.elements.add(new TimestampedValue(value, Instant.now().toEpochMilli())); // notify the policy object for performing any eviction that may be needed. - this.listGaugeEvictionPolicy.evict(this.elements, this.maxNumberOfItems, this.maxStaleness); + this.evict(this.elements, this.maxNumberOfItems, this.maxStaleness); } /** @@ -110,4 +108,42 @@ public void add(T value) { public void visit(MetricsVisitor visitor) { visitor.listGauge(this); } + + /** + * Evicts entries from the elements list, based on the given item-size and durationThreshold. + * Callers are responsible for thread-safety. + */ + public void evict(Queue> elements, int maxNumberOfItems, Duration maxStaleness) { + this.evictBasedOnSize(elements, maxNumberOfItems); + this.evictBasedOnTimestamp(elements, maxStaleness); + } + + /** + * Evicts entries from elements in FIFO order until it has maxNumberOfItems + * @param elements queue to evict elements from + * @param maxNumberOfItems max number of items to be left in the queue + */ + private void evictBasedOnSize(Queue> elements, int maxNumberOfItems) { + int numToEvict = elements.size() - maxNumberOfItems; + while (numToEvict > 0) { + elements.poll(); // remove head + numToEvict--; + } + } + + /** + * Removes entries from elements to ensure no element has a timestamp more than maxStaleness before current timestamp. + * @param elements the queue to evict elements from + * @param maxStaleness max staleness permitted in elements + */ + private void evictBasedOnTimestamp(Queue> elements, Duration maxStaleness) { + Instant currentTimestamp = Instant.now(); + TimestampedValue valueInfo = elements.peek(); + + // continue remove-head if currenttimestamp - head-element's timestamp > durationThreshold + while (valueInfo != null && currentTimestamp.toEpochMilli() - valueInfo.getTimestamp() > maxStaleness.toMillis()) { + elements.poll(); + valueInfo = elements.peek(); + } + } } From cc4d881e905acc92de2749c92e6e2e0881a3b401 Mon Sep 17 00:00:00 2001 From: Ray Matharu Date: Fri, 8 Jun 2018 13:36:44 -0700 Subject: [PATCH 17/33] Reverting superflous comment changes, indentation changes --- .../org/apache/samza/metrics/ListGauge.java | 44 ++++++-------- .../org/apache/samza/metrics/TestTimer.java | 7 +-- .../system/eventhub/TestMetricsRegistry.java | 58 +++++++++---------- .../org/apache/samza/metrics/MetricGroup.java | 42 +++++++------- .../samza/container/SamzaContainer.scala | 2 +- .../container/SamzaContainerMetrics.scala | 2 +- .../samza/metrics/MetricsRegistryMap.scala | 48 +++++++-------- .../samza/metrics/reporter/JmxReporter.scala | 11 ---- .../reporter/MetricsSnapshotReporter.scala | 2 - 9 files changed, 97 insertions(+), 119 deletions(-) diff --git a/samza-api/src/main/java/org/apache/samza/metrics/ListGauge.java b/samza-api/src/main/java/org/apache/samza/metrics/ListGauge.java index d25934bc61..a09d8c835e 100644 --- a/samza-api/src/main/java/org/apache/samza/metrics/ListGauge.java +++ b/samza-api/src/main/java/org/apache/samza/metrics/ListGauge.java @@ -33,8 +33,8 @@ * {@link ListGauge}s are useful for maintaining, recording, or collecting values over time. * For example, a set of specific logging-events (e.g., errors). * - * Eviction from list is either done by consuming-code using the remove APIs or by specifying an eviction policy - * at creation time. + * Eviction is controlled by parameters (maxNumberOfItems and maxStaleness), which are set during instantiation. + * Eviction happens during element addition or during reads of the ListGauge (getValues). * * All public methods are thread-safe. * @@ -79,13 +79,11 @@ public String getName() { } /** - * Get the Collection of Gauge values currently in the list, used when serializing this Gauge. - * Also evicts values based on the configured maxItems and maxStaleness. - * @return the collection of gauge values + * Get the Collection of values currently in the list. + * @return the collection of values */ public Collection getValues() { - // notify the policy object for performing any eviction that may be needed. - this.evict(this.elements, this.maxNumberOfItems, this.maxStaleness); + this.evict(); return Collections.unmodifiableList(this.elements.stream().map(x -> x.getValue()).collect(Collectors.toList())); } @@ -97,8 +95,8 @@ public Collection getValues() { public void add(T value) { this.elements.add(new TimestampedValue(value, Instant.now().toEpochMilli())); - // notify the policy object for performing any eviction that may be needed. - this.evict(this.elements, this.maxNumberOfItems, this.maxStaleness); + // perform any evictions that may be needed. + this.evict(); } /** @@ -111,39 +109,35 @@ public void visit(MetricsVisitor visitor) { /** * Evicts entries from the elements list, based on the given item-size and durationThreshold. - * Callers are responsible for thread-safety. */ - public void evict(Queue> elements, int maxNumberOfItems, Duration maxStaleness) { - this.evictBasedOnSize(elements, maxNumberOfItems); - this.evictBasedOnTimestamp(elements, maxStaleness); + private void evict() { + this.evictBasedOnSize(); + this.evictBasedOnTimestamp(); } /** * Evicts entries from elements in FIFO order until it has maxNumberOfItems - * @param elements queue to evict elements from - * @param maxNumberOfItems max number of items to be left in the queue */ - private void evictBasedOnSize(Queue> elements, int maxNumberOfItems) { - int numToEvict = elements.size() - maxNumberOfItems; + private void evictBasedOnSize() { + int numToEvict = this.elements.size() - this.maxNumberOfItems; while (numToEvict > 0) { - elements.poll(); // remove head + this.elements.poll(); // remove head numToEvict--; } } /** * Removes entries from elements to ensure no element has a timestamp more than maxStaleness before current timestamp. - * @param elements the queue to evict elements from - * @param maxStaleness max staleness permitted in elements */ - private void evictBasedOnTimestamp(Queue> elements, Duration maxStaleness) { + private void evictBasedOnTimestamp() { Instant currentTimestamp = Instant.now(); - TimestampedValue valueInfo = elements.peek(); + TimestampedValue valueInfo = this.elements.peek(); // continue remove-head if currenttimestamp - head-element's timestamp > durationThreshold - while (valueInfo != null && currentTimestamp.toEpochMilli() - valueInfo.getTimestamp() > maxStaleness.toMillis()) { - elements.poll(); - valueInfo = elements.peek(); + while (valueInfo != null + && currentTimestamp.toEpochMilli() - valueInfo.getTimestamp() > this.maxStaleness.toMillis()) { + this.elements.poll(); + valueInfo = this.elements.peek(); } } } diff --git a/samza-api/src/test/java/org/apache/samza/metrics/TestTimer.java b/samza-api/src/test/java/org/apache/samza/metrics/TestTimer.java index 6fdca549c5..c694d3ffc8 100644 --- a/samza-api/src/test/java/org/apache/samza/metrics/TestTimer.java +++ b/samza-api/src/test/java/org/apache/samza/metrics/TestTimer.java @@ -19,13 +19,12 @@ package org.apache.samza.metrics; -import static org.junit.Assert.*; - import java.util.Arrays; - import org.apache.samza.util.Clock; import org.junit.Test; +import static org.junit.Assert.*; + public class TestTimer { /* @@ -62,7 +61,7 @@ public void testTimerWithDifferentWindowSize() { assertTrue(snapshot.getValues().containsAll(Arrays.asList(1L, 2L, 3L))); assertEquals(3, snapshot.getValues().size()); - // The time is 500 for update(4L) because getValues calls clock once + 3 + // The time is 500 for update(4L) because getSnapshot calls clock once + 3 // updates that call clock 3 times timer.update(4L); Snapshot snapshot2 = timer.getSnapshot(); diff --git a/samza-azure/src/test/java/org/apache/samza/system/eventhub/TestMetricsRegistry.java b/samza-azure/src/test/java/org/apache/samza/system/eventhub/TestMetricsRegistry.java index 312ad2e06a..3d913c32f2 100644 --- a/samza-azure/src/test/java/org/apache/samza/system/eventhub/TestMetricsRegistry.java +++ b/samza-azure/src/test/java/org/apache/samza/system/eventhub/TestMetricsRegistry.java @@ -1,24 +1,27 @@ /* - * 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. - */ +* 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.system.eventhub; +import java.util.ArrayList; +import java.util.List; +import java.util.Map; import org.apache.commons.collections4.map.HashedMap; import org.apache.samza.metrics.Counter; import org.apache.samza.metrics.Gauge; @@ -26,11 +29,6 @@ import org.apache.samza.metrics.MetricsRegistry; import org.apache.samza.metrics.Timer; -import java.util.ArrayList; -import java.util.List; -import java.util.Map; - - public class TestMetricsRegistry implements MetricsRegistry { private Map> counters = new HashedMap<>(); @@ -71,6 +69,14 @@ public Counter newCounter(String group, Counter counter) { return null; } + @Override + public ListGauge newListGauge(String group, ListGauge listGauge) { + listGauges.putIfAbsent(group, new ArrayList()); + ListGauge value = new ListGauge(group); + listGauges.get(group).add(value); + return value; + } + @Override public Gauge newGauge(String group, Gauge value) { if (!gauges.containsKey(group)) { @@ -81,14 +87,6 @@ public Gauge newGauge(String group, Gauge value) { return value; } - @Override - public ListGauge newListGauge(String group, ListGauge listGauge) { - listGauges.putIfAbsent(group, new ArrayList()); - ListGauge value = new ListGauge(group); - listGauges.get(group).add(value); - return value; - } - @Override public Timer newTimer(String group, String name) { return null; diff --git a/samza-core/src/main/java/org/apache/samza/metrics/MetricGroup.java b/samza-core/src/main/java/org/apache/samza/metrics/MetricGroup.java index 4dfbbc9acf..d614293955 100644 --- a/samza-core/src/main/java/org/apache/samza/metrics/MetricGroup.java +++ b/samza-core/src/main/java/org/apache/samza/metrics/MetricGroup.java @@ -1,21 +1,21 @@ /* - * 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. - */ +* 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; @@ -44,14 +44,14 @@ public Counter newCounter(String name) { return registry.newCounter(groupName, (prefix + name).toLowerCase()); } - public Gauge newGauge(String name, T value) { - return registry.newGauge(groupName, new Gauge((prefix + name).toLowerCase(), value)); - } - public ListGauge newListGauge(String name) { return registry.newListGauge(groupName, new ListGauge(name)); } + public Gauge newGauge(String name, T value) { + return registry.newGauge(groupName, new Gauge((prefix + name).toLowerCase(), value)); + } + /* * Specify a dynamic gauge that always returns the latest value when polled. * The value closure/object must be thread safe, since metrics reporters may access diff --git a/samza-core/src/main/scala/org/apache/samza/container/SamzaContainer.scala b/samza-core/src/main/scala/org/apache/samza/container/SamzaContainer.scala index ddc99ff6da..8d886669b3 100644 --- a/samza-core/src/main/scala/org/apache/samza/container/SamzaContainer.scala +++ b/samza-core/src/main/scala/org/apache/samza/container/SamzaContainer.scala @@ -43,7 +43,7 @@ import org.apache.samza.container.disk.{DiskQuotaPolicyFactory, DiskSpaceMonitor import org.apache.samza.container.host.{StatisticsMonitorImpl, SystemMemoryStatistics, SystemStatisticsMonitor} import org.apache.samza.coordinator.stream.{CoordinatorStreamManager, CoordinatorStreamSystemProducer} import org.apache.samza.job.model.JobModel -import org.apache.samza.metrics._ +import org.apache.samza.metrics.{JmxServer, JvmMetrics, MetricsRegistryMap, MetricsReporter} import org.apache.samza.serializers._ import org.apache.samza.serializers.model.SamzaObjectMapper import org.apache.samza.storage.{StorageEngineFactory, TaskStorageManager} diff --git a/samza-core/src/main/scala/org/apache/samza/container/SamzaContainerMetrics.scala b/samza-core/src/main/scala/org/apache/samza/container/SamzaContainerMetrics.scala index ae7776b996..ad7b4d2994 100644 --- a/samza-core/src/main/scala/org/apache/samza/container/SamzaContainerMetrics.scala +++ b/samza-core/src/main/scala/org/apache/samza/container/SamzaContainerMetrics.scala @@ -21,7 +21,7 @@ package org.apache.samza.container import java.util -import org.apache.samza.metrics._ +import org.apache.samza.metrics.{Gauge, ReadableMetricsRegistry, MetricsRegistryMap, MetricsHelper} class SamzaContainerMetrics( val source: String = "unknown", diff --git a/samza-core/src/main/scala/org/apache/samza/metrics/MetricsRegistryMap.scala b/samza-core/src/main/scala/org/apache/samza/metrics/MetricsRegistryMap.scala index d12c53ca2c..75ed6aa9b4 100644 --- a/samza-core/src/main/scala/org/apache/samza/metrics/MetricsRegistryMap.scala +++ b/samza-core/src/main/scala/org/apache/samza/metrics/MetricsRegistryMap.scala @@ -23,9 +23,9 @@ import org.apache.samza.util.Logging import java.util.concurrent.ConcurrentHashMap /** - * A class that holds all metrics registered with it. It can be registered - * with one or more MetricReporters to flush metrics. - */ + * A class that holds all metrics registered with it. It can be registered + * with one or more MetricReporters to flush metrics. + */ class MetricsRegistryMap(val name: String) extends ReadableMetricsRegistry with Logging { var listeners = Set[ReadableMetricsRegistryListener]() @@ -37,7 +37,7 @@ class MetricsRegistryMap(val name: String) extends ReadableMetricsRegistry with def this() = this("unknown") def newCounter(group: String, counter: Counter) = { - debug("Add new counter %s %s %s." format(group, counter.getName, counter)) + debug("Add new counter %s %s %s." format (group, counter.getName, counter)) putAndGetGroup(group).putIfAbsent(counter.getName, counter) val realCounter = metrics.get(group).get(counter.getName).asInstanceOf[Counter] listeners.foreach(_.onCounter(group, realCounter)) @@ -45,40 +45,25 @@ class MetricsRegistryMap(val name: String) extends ReadableMetricsRegistry with } def newCounter(group: String, name: String) = { - debug("Creating new counter %s %s." format(group, name)) + debug("Creating new counter %s %s." format (group, name)) newCounter(group, new Counter(name)) } def newGauge[T](group: String, gauge: Gauge[T]) = { - debug("Adding new gauge %s %s %s." format(group, gauge.getName, gauge)) + debug("Adding new gauge %s %s %s." format (group, gauge.getName, gauge)) putAndGetGroup(group).putIfAbsent(gauge.getName, gauge) val realGauge = metrics.get(group).get(gauge.getName).asInstanceOf[Gauge[T]] listeners.foreach(_.onGauge(group, realGauge)) realGauge } - /** - * Register a {@link org.apache.samza.metrics.ListGauge} - * - * @param group Group for this ListGauge - * @param listGauge the ListGauge to register - * @tparam T the type of the list gauge - */ - def newListGauge[T](group: String, listGauge: ListGauge[T]) = { - debug("Adding new listgauge %s %s %s." format(group, listGauge.getName, listGauge)) - putAndGetGroup(group).putIfAbsent(listGauge.getName, listGauge) - val realListGauge = metrics.get(group).get(listGauge.getName).asInstanceOf[ListGauge[T]] - listeners.foreach(_.onListGauge(group, realListGauge)) - realListGauge - } - def newGauge[T](group: String, name: String, value: T) = { - debug("Creating new gauge %s %s %s." format(group, name, value)) + debug("Creating new gauge %s %s %s." format (group, name, value)) newGauge(group, new Gauge[T](name, value)) } def newTimer(group: String, timer: Timer) = { - debug("Add new timer %s %s %s." format(group, timer.getName, timer)) + debug("Add new timer %s %s %s." format (group, timer.getName, timer)) putAndGetGroup(group).putIfAbsent(timer.getName, timer) val realTimer = metrics.get(group).get(timer.getName).asInstanceOf[Timer] listeners.foreach(_.onTimer(group, realTimer)) @@ -86,10 +71,25 @@ class MetricsRegistryMap(val name: String) extends ReadableMetricsRegistry with } def newTimer(group: String, name: String) = { - debug("Creating new timer %s %s." format(group, name)) + debug("Creating new timer %s %s." format (group, name)) newTimer(group, new Timer(name)) } + /** + * Register a {@link org.apache.samza.metrics.ListGauge} + * + * @param group Group for this ListGauge + * @param listGauge the ListGauge to register + * @tparam T the type of the list gauge + */ + def newListGauge[T](group: String, listGauge: ListGauge[T]) = { + debug("Adding new listgauge %s %s %s." format(group, listGauge.getName, listGauge)) + putAndGetGroup(group).putIfAbsent(listGauge.getName, listGauge) + val realListGauge = metrics.get(group).get(listGauge.getName).asInstanceOf[ListGauge[T]] + listeners.foreach(_.onListGauge(group, realListGauge)) + realListGauge + } + private def putAndGetGroup(group: String) = { metrics.putIfAbsent(group, new ConcurrentHashMap[String, Metric]) metrics.get(group) diff --git a/samza-core/src/main/scala/org/apache/samza/metrics/reporter/JmxReporter.scala b/samza-core/src/main/scala/org/apache/samza/metrics/reporter/JmxReporter.scala index 5aa363ad4c..c601b29319 100644 --- a/samza-core/src/main/scala/org/apache/samza/metrics/reporter/JmxReporter.scala +++ b/samza-core/src/main/scala/org/apache/samza/metrics/reporter/JmxReporter.scala @@ -45,13 +45,9 @@ class JmxReporter(server: MBeanServer) extends MetricsReporter with Logging { registry.getGroup(group).asScala.foreach { case (name, metric) => metric.visit(new MetricsVisitor { - def counter(counter: Counter) = registerBean(new JmxCounter(counter, getObjectName(group, name, sources(registry)))) - def gauge[T](gauge: Gauge[T]) = registerBean(new JmxGauge(gauge.asInstanceOf[Gauge[Object]], getObjectName(group, name, sources(registry)))) - def timer(timer: Timer) = registerBean(new JmxTimer(timer, getObjectName(group, name, sources(registry)))) - def listGauge[T](listGauge: ListGauge[T]) = registerBean(new JmxListGauge(listGauge.asInstanceOf[ListGauge[Object]], getObjectName(group, name, sources(registry)))) }) @@ -67,15 +63,12 @@ class JmxReporter(server: MBeanServer) extends MetricsReporter with Logging { def onCounter(group: String, counter: Counter) { registerBean(new JmxCounter(counter, getObjectName(group, counter.getName, source))) } - def onGauge(group: String, gauge: Gauge[_]) { registerBean(new JmxGauge(gauge.asInstanceOf[Gauge[Object]], getObjectName(group, gauge.getName, source))) } - def onTimer(group: String, timer: Timer) { registerBean(new JmxTimer(timer, getObjectName(group, timer.getName, source))) } - def onListGauge(group: String, listGauge: ListGauge[_]) { registerBean(new JmxListGauge(listGauge.asInstanceOf[ListGauge[Object]], getObjectName(group, listGauge.getName, source))) } @@ -113,13 +106,11 @@ trait JmxGaugeMBean extends MetricMBean { class JmxGauge(g: org.apache.samza.metrics.Gauge[Object], on: ObjectName) extends JmxGaugeMBean { def getValue = g.getValue - def objectName = on } class JmxListGauge(g: org.apache.samza.metrics.ListGauge[Object], on: ObjectName) extends JmxGaugeMBean { def getValue = g.getValues - def objectName = on } @@ -129,7 +120,6 @@ trait JmxCounterMBean extends MetricMBean { class JmxCounter(c: org.apache.samza.metrics.Counter, on: ObjectName) extends JmxCounterMBean { def getCount() = c.getCount() - def objectName = on } @@ -139,7 +129,6 @@ trait JmxTimerMBean extends MetricMBean { class JmxTimer(t: org.apache.samza.metrics.Timer, on: ObjectName) extends JmxTimerMBean { def getAverageTime() = t.getSnapshot().getAverage() - def objectName = on } diff --git a/samza-core/src/main/scala/org/apache/samza/metrics/reporter/MetricsSnapshotReporter.scala b/samza-core/src/main/scala/org/apache/samza/metrics/reporter/MetricsSnapshotReporter.scala index 1f22cff7e4..d300e90919 100644 --- a/samza-core/src/main/scala/org/apache/samza/metrics/reporter/MetricsSnapshotReporter.scala +++ b/samza-core/src/main/scala/org/apache/samza/metrics/reporter/MetricsSnapshotReporter.scala @@ -19,8 +19,6 @@ package org.apache.samza.metrics.reporter -import java.util - import com.google.common.util.concurrent.ThreadFactoryBuilder import org.apache.samza.metrics._ import org.apache.samza.serializers.Serializer From 863d85082f296bb71ab3a688229dc728f4a93803 Mon Sep 17 00:00:00 2001 From: Ray Matharu Date: Fri, 8 Jun 2018 17:49:33 -0700 Subject: [PATCH 18/33] Minor changes --- .../src/main/java/org/apache/samza/metrics/ListGauge.java | 2 +- .../apache/samza/system/eventhub/TestMetricsRegistry.java | 5 ++--- .../src/main/java/org/apache/samza/metrics/MetricGroup.java | 2 +- .../org/apache/samza/container/SamzaContainerMetrics.scala | 3 +-- 4 files changed, 5 insertions(+), 7 deletions(-) diff --git a/samza-api/src/main/java/org/apache/samza/metrics/ListGauge.java b/samza-api/src/main/java/org/apache/samza/metrics/ListGauge.java index a09d8c835e..545fd45420 100644 --- a/samza-api/src/main/java/org/apache/samza/metrics/ListGauge.java +++ b/samza-api/src/main/java/org/apache/samza/metrics/ListGauge.java @@ -29,7 +29,7 @@ /** - * A {@link ListGauge} is a {@link org.apache.samza.metrics.Metric} that buffers multiple instances of a type T in a list. + * A {@link ListGauge} is a {@link Metric} that buffers multiple instances of a type T in a list. * {@link ListGauge}s are useful for maintaining, recording, or collecting values over time. * For example, a set of specific logging-events (e.g., errors). * diff --git a/samza-azure/src/test/java/org/apache/samza/system/eventhub/TestMetricsRegistry.java b/samza-azure/src/test/java/org/apache/samza/system/eventhub/TestMetricsRegistry.java index 3d913c32f2..01b69edbc2 100644 --- a/samza-azure/src/test/java/org/apache/samza/system/eventhub/TestMetricsRegistry.java +++ b/samza-azure/src/test/java/org/apache/samza/system/eventhub/TestMetricsRegistry.java @@ -72,9 +72,8 @@ public Counter newCounter(String group, Counter counter) { @Override public ListGauge newListGauge(String group, ListGauge listGauge) { listGauges.putIfAbsent(group, new ArrayList()); - ListGauge value = new ListGauge(group); - listGauges.get(group).add(value); - return value; + listGauges.get(group).add(listGauge); + return listGauge; } @Override diff --git a/samza-core/src/main/java/org/apache/samza/metrics/MetricGroup.java b/samza-core/src/main/java/org/apache/samza/metrics/MetricGroup.java index d614293955..fc5784699f 100644 --- a/samza-core/src/main/java/org/apache/samza/metrics/MetricGroup.java +++ b/samza-core/src/main/java/org/apache/samza/metrics/MetricGroup.java @@ -44,7 +44,7 @@ public Counter newCounter(String name) { return registry.newCounter(groupName, (prefix + name).toLowerCase()); } - public ListGauge newListGauge(String name) { + public ListGauge newListGauge(String name) { return registry.newListGauge(groupName, new ListGauge(name)); } diff --git a/samza-core/src/main/scala/org/apache/samza/container/SamzaContainerMetrics.scala b/samza-core/src/main/scala/org/apache/samza/container/SamzaContainerMetrics.scala index ad7b4d2994..a26e6669bf 100644 --- a/samza-core/src/main/scala/org/apache/samza/container/SamzaContainerMetrics.scala +++ b/samza-core/src/main/scala/org/apache/samza/container/SamzaContainerMetrics.scala @@ -48,8 +48,7 @@ class SamzaContainerMetrics( val taskStoreRestorationMetrics: util.Map[TaskName, Gauge[Long]] = new util.HashMap[TaskName, Gauge[Long]]() - // A string-gauge metric to capture exceptions at this container - val exception = newListGauge[String]("exception") + val exceptions = newListGauge[String]("exceptions") def addStoreRestorationGauge(taskName: TaskName, storeName: String) { taskStoreRestorationMetrics.put(taskName, newGauge("%s-%s-restore-time" format(taskName.toString, storeName), -1L)) From 925473c9b299c8bfe0d30d4a76cba58c0f3d7305 Mon Sep 17 00:00:00 2001 From: Ray Matharu Date: Fri, 8 Jun 2018 18:15:54 -0700 Subject: [PATCH 19/33] Adding throwable cause messages and class names to exception event --- .../container/SamzaContainerMetrics.scala | 2 +- .../diagnostics/DiagnosticsAppender.java | 11 ++++- .../DiagnosticsExceptionEvent.java | 40 +++++++++---------- 3 files changed, 30 insertions(+), 23 deletions(-) diff --git a/samza-core/src/main/scala/org/apache/samza/container/SamzaContainerMetrics.scala b/samza-core/src/main/scala/org/apache/samza/container/SamzaContainerMetrics.scala index 1b2339729e..d5cf6c653d 100644 --- a/samza-core/src/main/scala/org/apache/samza/container/SamzaContainerMetrics.scala +++ b/samza-core/src/main/scala/org/apache/samza/container/SamzaContainerMetrics.scala @@ -49,7 +49,7 @@ class SamzaContainerMetrics( val taskStoreRestorationMetrics: util.Map[TaskName, Gauge[Long]] = new util.HashMap[TaskName, Gauge[Long]]() - val exceptions = newListGauge[String]("exceptions") + val exceptions = newListGauge[DiagnosticsExceptionEvent]("exceptions") def addStoreRestorationGauge(taskName: TaskName, storeName: String) { taskStoreRestorationMetrics.put(taskName, newGauge("%s-%s-restore-time" format(taskName.toString, storeName), -1L)) diff --git a/samza-core/src/main/scala/org/apache/samza/diagnostics/DiagnosticsAppender.java b/samza-core/src/main/scala/org/apache/samza/diagnostics/DiagnosticsAppender.java index 4c2c875bdf..477d319f8c 100644 --- a/samza-core/src/main/scala/org/apache/samza/diagnostics/DiagnosticsAppender.java +++ b/samza-core/src/main/scala/org/apache/samza/diagnostics/DiagnosticsAppender.java @@ -42,7 +42,7 @@ public class DiagnosticsAppender extends AppenderSkeleton { private final ListGauge samzaContainerExceptionMetric; public DiagnosticsAppender(SamzaContainerMetrics samzaContainerMetrics) { - this.samzaContainerExceptionMetric = (ListGauge) samzaContainerMetrics.exception(); + this.samzaContainerExceptionMetric = (ListGauge) samzaContainerMetrics.exceptions(); } @Override @@ -51,9 +51,16 @@ protected void append(LoggingEvent loggingEvent) { // if an event with a non-null throwable is received => exception event if (loggingEvent.getThrowableInformation() != null) { + Throwable throwable = loggingEvent.getThrowableInformation().getThrowable(); + Throwable throwableCause = loggingEvent.getThrowableInformation().getThrowable().getCause(); + + String throwableClassName = (throwable == null) ? "" : throwable.getClass().getName(); + String throwableCauseClassName = (throwableCause == null) ? "" : throwableCause.getClass().getName(); + String throwableCauseMessage = (throwableCause == null) ? "" : throwableCause.getMessage(); + DiagnosticsExceptionEvent diagnosticsExceptionEvent = new DiagnosticsExceptionEvent(loggingEvent.timeStamp, loggingEvent.getMessage().toString(), - loggingEvent.getThreadName(), + throwableClassName, throwableCauseMessage, throwableCauseClassName, loggingEvent.getThreadName(), Arrays.toString(loggingEvent.getThrowableInformation().getThrowableStrRep()), getStackTraceIdentifier(loggingEvent.getThrowableInformation().getThrowable().getStackTrace())); diff --git a/samza-core/src/main/scala/org/apache/samza/diagnostics/DiagnosticsExceptionEvent.java b/samza-core/src/main/scala/org/apache/samza/diagnostics/DiagnosticsExceptionEvent.java index 97353cc584..728b91b7b8 100644 --- a/samza-core/src/main/scala/org/apache/samza/diagnostics/DiagnosticsExceptionEvent.java +++ b/samza-core/src/main/scala/org/apache/samza/diagnostics/DiagnosticsExceptionEvent.java @@ -26,61 +26,61 @@ public class DiagnosticsExceptionEvent { private long timestamp; // the timestamp associated with this exception + + private String className; // the classname of the exception private String message; // the string message associated with this exception private String threadName; // the name of the thread on which this exception occurred + private String causeClassName; // the classname of the causing exception (if any) + private String causeMessageName; // the message of the causing exception (if any) + // a compact string representation of this exception, to avoid serializing the entire stack trace private String compactStackTrace; // a unique identifier computed to identify this stack trace private Object stackTraceIdentifier; - public DiagnosticsExceptionEvent(long timestampMillis, String message, String threadName, String compactStackTrace, - Object stackTraceIdentifier) { + public DiagnosticsExceptionEvent(long timestampMillis, String className, String message, String causeClassName, + String causeMessageName, String threadName, String compactStackTrace, Object stackTraceIdentifier) { this.message = message; this.timestamp = timestampMillis; this.threadName = threadName; this.compactStackTrace = compactStackTrace; this.stackTraceIdentifier = stackTraceIdentifier; + this.className = className; + this.causeClassName = causeClassName; + this.causeMessageName = causeMessageName; } public long getTimestamp() { return timestamp; } - public void setTimestamp(long timestamp) { - this.timestamp = timestamp; - } - public String getMessage() { return message; } - public void setMessage(String message) { - this.message = message; - } - public String getThreadName() { return threadName; } - public void setThreadName(String threadName) { - this.threadName = threadName; - } - public String getCompactStackTrace() { return compactStackTrace; } - public void setCompactStackTrace(String compactStackTrace) { - this.compactStackTrace = compactStackTrace; - } - public Object getStackTraceIdentifier() { return stackTraceIdentifier; } - public void setStackTraceIdentifier(Object stackTraceIdentifier) { - this.stackTraceIdentifier = stackTraceIdentifier; + public String getClassName() { + return className; + } + + public String getCauseClassName() { + return causeClassName; + } + + public String getCauseMessageName() { + return causeMessageName; } } \ No newline at end of file From 2ad02648914bfb59d612754c19c2ea671b505379 Mon Sep 17 00:00:00 2001 From: Ray Matharu Date: Wed, 13 Jun 2018 12:31:50 -0700 Subject: [PATCH 20/33] Fixing minor bug around diagnostic-event-initialization, adding blanket-catch --- .../diagnostics/DiagnosticsAppender.java | 47 +++++++++++-------- 1 file changed, 27 insertions(+), 20 deletions(-) diff --git a/samza-core/src/main/scala/org/apache/samza/diagnostics/DiagnosticsAppender.java b/samza-core/src/main/scala/org/apache/samza/diagnostics/DiagnosticsAppender.java index 477d319f8c..fb64e2274c 100644 --- a/samza-core/src/main/scala/org/apache/samza/diagnostics/DiagnosticsAppender.java +++ b/samza-core/src/main/scala/org/apache/samza/diagnostics/DiagnosticsAppender.java @@ -48,26 +48,33 @@ public DiagnosticsAppender(SamzaContainerMetrics samzaContainerMetrics) { @Override protected void append(LoggingEvent loggingEvent) { - // if an event with a non-null throwable is received => exception event - if (loggingEvent.getThrowableInformation() != null) { - - Throwable throwable = loggingEvent.getThrowableInformation().getThrowable(); - Throwable throwableCause = loggingEvent.getThrowableInformation().getThrowable().getCause(); - - String throwableClassName = (throwable == null) ? "" : throwable.getClass().getName(); - String throwableCauseClassName = (throwableCause == null) ? "" : throwableCause.getClass().getName(); - String throwableCauseMessage = (throwableCause == null) ? "" : throwableCause.getMessage(); - - DiagnosticsExceptionEvent diagnosticsExceptionEvent = - new DiagnosticsExceptionEvent(loggingEvent.timeStamp, loggingEvent.getMessage().toString(), - throwableClassName, throwableCauseMessage, throwableCauseClassName, loggingEvent.getThreadName(), - Arrays.toString(loggingEvent.getThrowableInformation().getThrowableStrRep()), - getStackTraceIdentifier(loggingEvent.getThrowableInformation().getThrowable().getStackTrace())); - - samzaContainerExceptionMetric.add(diagnosticsExceptionEvent); - logger.debug("Received DiagnosticsExceptionEvent " + diagnosticsExceptionEvent); - } else { - logger.debug("Received non-exception event with message " + loggingEvent.getMessage()); + try { + + // if an event with a non-null throwable is received => exception event + if (loggingEvent.getThrowableInformation() != null) { + + Throwable throwable = loggingEvent.getThrowableInformation().getThrowable(); + Throwable throwableCause = loggingEvent.getThrowableInformation().getThrowable().getCause(); + + String throwableClassName = (throwable == null) ? "" : throwable.getClass().getName(); + String throwableCauseClassName = (throwableCause == null) ? "" : throwableCause.getClass().getName(); + String throwableCauseMessage = (throwableCause == null) ? "" : throwableCause.getMessage(); + + DiagnosticsExceptionEvent diagnosticsExceptionEvent = + new DiagnosticsExceptionEvent(loggingEvent.timeStamp, throwableClassName, + loggingEvent.getMessage().toString(), throwableCauseClassName, throwableCauseMessage, + loggingEvent.getThreadName(), + Arrays.toString(loggingEvent.getThrowableInformation().getThrowableStrRep()), + getStackTraceIdentifier(loggingEvent.getThrowableInformation().getThrowable().getStackTrace())); + + samzaContainerExceptionMetric.add(diagnosticsExceptionEvent); + logger.debug("Received DiagnosticsExceptionEvent " + diagnosticsExceptionEvent); + } else { + logger.debug("Received non-exception event with message " + loggingEvent.getMessage()); + } + } catch (Exception e) { + // blanket catch of all exceptions so as to not impact any job + logger.error("Exception in logging event parsing", e); } } From cd58669e1fb9628d69dbc95dee27af8b94cc997f Mon Sep 17 00:00:00 2001 From: Ray Matharu Date: Thu, 21 Jun 2018 15:17:06 -0700 Subject: [PATCH 21/33] Removing thread name --- .../apache/samza/container/SamzaContainerMetrics.scala | 2 +- .../org/apache/samza/diagnostics/DiagnosticsAppender.java | 1 - .../samza/diagnostics/DiagnosticsExceptionEvent.java | 8 +------- 3 files changed, 2 insertions(+), 9 deletions(-) diff --git a/samza-core/src/main/scala/org/apache/samza/container/SamzaContainerMetrics.scala b/samza-core/src/main/scala/org/apache/samza/container/SamzaContainerMetrics.scala index d7324d692f..d5cf6c653d 100644 --- a/samza-core/src/main/scala/org/apache/samza/container/SamzaContainerMetrics.scala +++ b/samza-core/src/main/scala/org/apache/samza/container/SamzaContainerMetrics.scala @@ -48,7 +48,7 @@ class SamzaContainerMetrics( val physicalMemoryMb = newGauge[Double]("physical-memory-mb", 0.0F) val taskStoreRestorationMetrics: util.Map[TaskName, Gauge[Long]] = new util.HashMap[TaskName, Gauge[Long]]() - + val exceptions = newListGauge[DiagnosticsExceptionEvent]("exceptions") def addStoreRestorationGauge(taskName: TaskName, storeName: String) { diff --git a/samza-core/src/main/scala/org/apache/samza/diagnostics/DiagnosticsAppender.java b/samza-core/src/main/scala/org/apache/samza/diagnostics/DiagnosticsAppender.java index fb64e2274c..00d08b868f 100644 --- a/samza-core/src/main/scala/org/apache/samza/diagnostics/DiagnosticsAppender.java +++ b/samza-core/src/main/scala/org/apache/samza/diagnostics/DiagnosticsAppender.java @@ -63,7 +63,6 @@ protected void append(LoggingEvent loggingEvent) { DiagnosticsExceptionEvent diagnosticsExceptionEvent = new DiagnosticsExceptionEvent(loggingEvent.timeStamp, throwableClassName, loggingEvent.getMessage().toString(), throwableCauseClassName, throwableCauseMessage, - loggingEvent.getThreadName(), Arrays.toString(loggingEvent.getThrowableInformation().getThrowableStrRep()), getStackTraceIdentifier(loggingEvent.getThrowableInformation().getThrowable().getStackTrace())); diff --git a/samza-core/src/main/scala/org/apache/samza/diagnostics/DiagnosticsExceptionEvent.java b/samza-core/src/main/scala/org/apache/samza/diagnostics/DiagnosticsExceptionEvent.java index 728b91b7b8..edaf3005d3 100644 --- a/samza-core/src/main/scala/org/apache/samza/diagnostics/DiagnosticsExceptionEvent.java +++ b/samza-core/src/main/scala/org/apache/samza/diagnostics/DiagnosticsExceptionEvent.java @@ -29,7 +29,6 @@ public class DiagnosticsExceptionEvent { private String className; // the classname of the exception private String message; // the string message associated with this exception - private String threadName; // the name of the thread on which this exception occurred private String causeClassName; // the classname of the causing exception (if any) private String causeMessageName; // the message of the causing exception (if any) @@ -41,10 +40,9 @@ public class DiagnosticsExceptionEvent { private Object stackTraceIdentifier; public DiagnosticsExceptionEvent(long timestampMillis, String className, String message, String causeClassName, - String causeMessageName, String threadName, String compactStackTrace, Object stackTraceIdentifier) { + String causeMessageName, String compactStackTrace, Object stackTraceIdentifier) { this.message = message; this.timestamp = timestampMillis; - this.threadName = threadName; this.compactStackTrace = compactStackTrace; this.stackTraceIdentifier = stackTraceIdentifier; this.className = className; @@ -60,10 +58,6 @@ public String getMessage() { return message; } - public String getThreadName() { - return threadName; - } - public String getCompactStackTrace() { return compactStackTrace; } From 8a66d16d50e98d8cefd08ccc83b68eb6fbd4dc1c Mon Sep 17 00:00:00 2001 From: Ray Matharu Date: Thu, 21 Jun 2018 17:08:59 -0700 Subject: [PATCH 22/33] Adding throwableInformation to DiagnosticsExceptionEvent, removing other things --- .../diagnostics/DiagnosticsAppender.java | 22 ++------ .../DiagnosticsExceptionEvent.java | 50 +++---------------- 2 files changed, 11 insertions(+), 61 deletions(-) diff --git a/samza-core/src/main/scala/org/apache/samza/diagnostics/DiagnosticsAppender.java b/samza-core/src/main/scala/org/apache/samza/diagnostics/DiagnosticsAppender.java index 00d08b868f..4a1700ba86 100644 --- a/samza-core/src/main/scala/org/apache/samza/diagnostics/DiagnosticsAppender.java +++ b/samza-core/src/main/scala/org/apache/samza/diagnostics/DiagnosticsAppender.java @@ -19,7 +19,6 @@ package org.apache.samza.diagnostics; -import java.util.Arrays; import org.apache.log4j.AppenderSkeleton; import org.apache.log4j.spi.LoggingEvent; import org.apache.samza.container.SamzaContainerMetrics; @@ -34,7 +33,7 @@ * {@link SamzaContainerMetrics}. * * When used inconjunction with {@link org.apache.samza.metrics.reporter.MetricsSnapshotReporter} provides a - * stream of diagnostics-related events.s + * stream of diagnostics-related events. */ public class DiagnosticsAppender extends AppenderSkeleton { @@ -42,7 +41,7 @@ public class DiagnosticsAppender extends AppenderSkeleton { private final ListGauge samzaContainerExceptionMetric; public DiagnosticsAppender(SamzaContainerMetrics samzaContainerMetrics) { - this.samzaContainerExceptionMetric = (ListGauge) samzaContainerMetrics.exceptions(); + this.samzaContainerExceptionMetric = samzaContainerMetrics.exceptions(); } @Override @@ -52,19 +51,8 @@ protected void append(LoggingEvent loggingEvent) { // if an event with a non-null throwable is received => exception event if (loggingEvent.getThrowableInformation() != null) { - - Throwable throwable = loggingEvent.getThrowableInformation().getThrowable(); - Throwable throwableCause = loggingEvent.getThrowableInformation().getThrowable().getCause(); - - String throwableClassName = (throwable == null) ? "" : throwable.getClass().getName(); - String throwableCauseClassName = (throwableCause == null) ? "" : throwableCause.getClass().getName(); - String throwableCauseMessage = (throwableCause == null) ? "" : throwableCause.getMessage(); - DiagnosticsExceptionEvent diagnosticsExceptionEvent = - new DiagnosticsExceptionEvent(loggingEvent.timeStamp, throwableClassName, - loggingEvent.getMessage().toString(), throwableCauseClassName, throwableCauseMessage, - Arrays.toString(loggingEvent.getThrowableInformation().getThrowableStrRep()), - getStackTraceIdentifier(loggingEvent.getThrowableInformation().getThrowable().getStackTrace())); + new DiagnosticsExceptionEvent(loggingEvent.timeStamp, loggingEvent.getThrowableInformation()); samzaContainerExceptionMetric.add(diagnosticsExceptionEvent); logger.debug("Received DiagnosticsExceptionEvent " + diagnosticsExceptionEvent); @@ -77,10 +65,6 @@ protected void append(LoggingEvent loggingEvent) { } } - private int getStackTraceIdentifier(StackTraceElement[] stackTrace) { - return Arrays.hashCode(stackTrace); - } - @Override public void close() { // Do nothing. diff --git a/samza-core/src/main/scala/org/apache/samza/diagnostics/DiagnosticsExceptionEvent.java b/samza-core/src/main/scala/org/apache/samza/diagnostics/DiagnosticsExceptionEvent.java index edaf3005d3..4534cca01a 100644 --- a/samza-core/src/main/scala/org/apache/samza/diagnostics/DiagnosticsExceptionEvent.java +++ b/samza-core/src/main/scala/org/apache/samza/diagnostics/DiagnosticsExceptionEvent.java @@ -18,6 +18,9 @@ */ package org.apache.samza.diagnostics; +import org.apache.log4j.spi.ThrowableInformation; + + /** * This class encapsulates information related to an exception event that is useful for diagnostics. * It used to define container, task, and other metrics as @@ -26,55 +29,18 @@ public class DiagnosticsExceptionEvent { private long timestamp; // the timestamp associated with this exception + private ThrowableInformation throwableInformation; - private String className; // the classname of the exception - private String message; // the string message associated with this exception - - private String causeClassName; // the classname of the causing exception (if any) - private String causeMessageName; // the message of the causing exception (if any) - - // a compact string representation of this exception, to avoid serializing the entire stack trace - private String compactStackTrace; - - // a unique identifier computed to identify this stack trace - private Object stackTraceIdentifier; - - public DiagnosticsExceptionEvent(long timestampMillis, String className, String message, String causeClassName, - String causeMessageName, String compactStackTrace, Object stackTraceIdentifier) { - this.message = message; + public DiagnosticsExceptionEvent(long timestampMillis, ThrowableInformation throwableInformation) { + this.throwableInformation = throwableInformation; this.timestamp = timestampMillis; - this.compactStackTrace = compactStackTrace; - this.stackTraceIdentifier = stackTraceIdentifier; - this.className = className; - this.causeClassName = causeClassName; - this.causeMessageName = causeMessageName; } public long getTimestamp() { return timestamp; } - public String getMessage() { - return message; - } - - public String getCompactStackTrace() { - return compactStackTrace; - } - - public Object getStackTraceIdentifier() { - return stackTraceIdentifier; - } - - public String getClassName() { - return className; - } - - public String getCauseClassName() { - return causeClassName; - } - - public String getCauseMessageName() { - return causeMessageName; + public ThrowableInformation getThrowableInformation() { + return this.throwableInformation; } } \ No newline at end of file From 48cbf3e0fa923b0021dc9510bba71a7f30bec23a Mon Sep 17 00:00:00 2001 From: Ray Matharu Date: Fri, 29 Jun 2018 16:36:45 -0700 Subject: [PATCH 23/33] Moving diagnostics.enabled to jobconfig, adding appender-add check, making exception event generic. --- .../scala/org/apache/samza/config/JobConfig.scala | 5 +++++ .../org/apache/samza/container/SamzaContainer.scala | 13 ++++++------- .../samza/diagnostics/DiagnosticsAppender.java | 13 +++++++------ .../diagnostics/DiagnosticsExceptionEvent.java | 13 +++++-------- 4 files changed, 23 insertions(+), 21 deletions(-) diff --git a/samza-core/src/main/scala/org/apache/samza/config/JobConfig.scala b/samza-core/src/main/scala/org/apache/samza/config/JobConfig.scala index 75e8005eae..fb6ce0a073 100644 --- a/samza-core/src/main/scala/org/apache/samza/config/JobConfig.scala +++ b/samza-core/src/main/scala/org/apache/samza/config/JobConfig.scala @@ -88,6 +88,9 @@ object JobConfig { // across application restarts val JOB_LOGGED_STORE_BASE_DIR = "job.logged.store.base.dir" + // Enables diagnostic appender for logging exception events + val JOB_DIAGNOSTICS_ENABLED = "job.diagnostics.enabled" + implicit def Config2Job(config: Config) = new JobConfig(config) /** @@ -186,4 +189,6 @@ class JobConfig(config: Config) extends ScalaMapConfig(config) with Logging { def getNonLoggedStorePath = getOption(JobConfig.JOB_NON_LOGGED_STORE_BASE_DIR) def getLoggedStorePath = getOption(JobConfig.JOB_LOGGED_STORE_BASE_DIR) + + def getDiagnosticsEnabled = { getBoolean(JobConfig.JOB_DIAGNOSTICS_ENABLED, false) } } diff --git a/samza-core/src/main/scala/org/apache/samza/container/SamzaContainer.scala b/samza-core/src/main/scala/org/apache/samza/container/SamzaContainer.scala index 623e0d8bb9..d61c4ed3d7 100644 --- a/samza-core/src/main/scala/org/apache/samza/container/SamzaContainer.scala +++ b/samza-core/src/main/scala/org/apache/samza/container/SamzaContainer.scala @@ -57,6 +57,7 @@ import org.apache.samza.util.Util import org.apache.samza.util._ import org.apache.samza.{SamzaContainerStatus, SamzaException} import org.apache.samza.diagnostics.DiagnosticsAppender; +import org.apache.log4j.Logger import scala.collection.JavaConverters._ @@ -907,15 +908,13 @@ class SamzaContainer( } def startDiagnostics { - // TODO: where should this reside, MetricConfig? Log4jSystemConfig? or a new separate config? - val DIAGNOSTICS_APPENDER_ENABLE = "samza.diagnostics.enabled" - if (containerContext.config.getBoolean(DIAGNOSTICS_APPENDER_ENABLE, false)) { - import org.apache.log4j.Logger + if (containerContext.config.getDiagnosticsEnabled) { val rootLogger = Logger.getRootLogger - info("Starting Diagnostics Appender.") - val diagnosticsAppender = new DiagnosticsAppender(this.metrics) - rootLogger.addAppender(diagnosticsAppender) + if (rootLogger.getAppender(classOf[DiagnosticsAppender].getName) == null) { + info("Starting diagnostics appender.") + rootLogger.addAppender(new DiagnosticsAppender(this.metrics)) + } } } diff --git a/samza-core/src/main/scala/org/apache/samza/diagnostics/DiagnosticsAppender.java b/samza-core/src/main/scala/org/apache/samza/diagnostics/DiagnosticsAppender.java index 4a1700ba86..0cb457143c 100644 --- a/samza-core/src/main/scala/org/apache/samza/diagnostics/DiagnosticsAppender.java +++ b/samza-core/src/main/scala/org/apache/samza/diagnostics/DiagnosticsAppender.java @@ -37,11 +37,12 @@ */ public class DiagnosticsAppender extends AppenderSkeleton { - private final Logger logger = LoggerFactory.getLogger(this.getClass().getName()); + private static final Logger LOG = LoggerFactory.getLogger(DiagnosticsAppender.class); private final ListGauge samzaContainerExceptionMetric; public DiagnosticsAppender(SamzaContainerMetrics samzaContainerMetrics) { this.samzaContainerExceptionMetric = samzaContainerMetrics.exceptions(); + this.setName(DiagnosticsAppender.class.getName()); } @Override @@ -51,17 +52,17 @@ protected void append(LoggingEvent loggingEvent) { // if an event with a non-null throwable is received => exception event if (loggingEvent.getThrowableInformation() != null) { - DiagnosticsExceptionEvent diagnosticsExceptionEvent = - new DiagnosticsExceptionEvent(loggingEvent.timeStamp, loggingEvent.getThrowableInformation()); + DiagnosticsExceptionEvent diagnosticsExceptionEvent = new DiagnosticsExceptionEvent(loggingEvent.timeStamp, + loggingEvent.getThrowableInformation().getThrowable()); samzaContainerExceptionMetric.add(diagnosticsExceptionEvent); - logger.debug("Received DiagnosticsExceptionEvent " + diagnosticsExceptionEvent); + LOG.debug("Received DiagnosticsExceptionEvent " + diagnosticsExceptionEvent); } else { - logger.debug("Received non-exception event with message " + loggingEvent.getMessage()); + LOG.debug("Received non-exception event with message " + loggingEvent.getMessage()); } } catch (Exception e) { // blanket catch of all exceptions so as to not impact any job - logger.error("Exception in logging event parsing", e); + LOG.error("Exception in logging event parsing", e); } } diff --git a/samza-core/src/main/scala/org/apache/samza/diagnostics/DiagnosticsExceptionEvent.java b/samza-core/src/main/scala/org/apache/samza/diagnostics/DiagnosticsExceptionEvent.java index 4534cca01a..64fb2c6403 100644 --- a/samza-core/src/main/scala/org/apache/samza/diagnostics/DiagnosticsExceptionEvent.java +++ b/samza-core/src/main/scala/org/apache/samza/diagnostics/DiagnosticsExceptionEvent.java @@ -18,9 +18,6 @@ */ package org.apache.samza.diagnostics; -import org.apache.log4j.spi.ThrowableInformation; - - /** * This class encapsulates information related to an exception event that is useful for diagnostics. * It used to define container, task, and other metrics as @@ -29,10 +26,10 @@ public class DiagnosticsExceptionEvent { private long timestamp; // the timestamp associated with this exception - private ThrowableInformation throwableInformation; + private Throwable throwable; - public DiagnosticsExceptionEvent(long timestampMillis, ThrowableInformation throwableInformation) { - this.throwableInformation = throwableInformation; + public DiagnosticsExceptionEvent(long timestampMillis, Throwable throwable) { + this.throwable = throwable; this.timestamp = timestampMillis; } @@ -40,7 +37,7 @@ public long getTimestamp() { return timestamp; } - public ThrowableInformation getThrowableInformation() { - return this.throwableInformation; + public Throwable getThrowable() { + return this.throwable; } } \ No newline at end of file From 346affdab05be815958e5bfd59e2de0b5a0256e8 Mon Sep 17 00:00:00 2001 From: Ray Matharu Date: Mon, 9 Jul 2018 17:23:04 -0700 Subject: [PATCH 24/33] Adding MDCMap to DiagnosticExceptionEvent for storing context --- .../samza/diagnostics/DiagnosticsAppender.java | 6 +++--- .../samza/diagnostics/DiagnosticsExceptionEvent.java | 12 +++++++++++- 2 files changed, 14 insertions(+), 4 deletions(-) diff --git a/samza-core/src/main/scala/org/apache/samza/diagnostics/DiagnosticsAppender.java b/samza-core/src/main/scala/org/apache/samza/diagnostics/DiagnosticsAppender.java index 0cb457143c..e3f5153fbf 100644 --- a/samza-core/src/main/scala/org/apache/samza/diagnostics/DiagnosticsAppender.java +++ b/samza-core/src/main/scala/org/apache/samza/diagnostics/DiagnosticsAppender.java @@ -49,11 +49,11 @@ public DiagnosticsAppender(SamzaContainerMetrics samzaContainerMetrics) { protected void append(LoggingEvent loggingEvent) { try { - // if an event with a non-null throwable is received => exception event if (loggingEvent.getThrowableInformation() != null) { - DiagnosticsExceptionEvent diagnosticsExceptionEvent = new DiagnosticsExceptionEvent(loggingEvent.timeStamp, - loggingEvent.getThrowableInformation().getThrowable()); + DiagnosticsExceptionEvent diagnosticsExceptionEvent = + new DiagnosticsExceptionEvent(loggingEvent.timeStamp, loggingEvent.getThrowableInformation().getThrowable(), + loggingEvent.getProperties()); samzaContainerExceptionMetric.add(diagnosticsExceptionEvent); LOG.debug("Received DiagnosticsExceptionEvent " + diagnosticsExceptionEvent); diff --git a/samza-core/src/main/scala/org/apache/samza/diagnostics/DiagnosticsExceptionEvent.java b/samza-core/src/main/scala/org/apache/samza/diagnostics/DiagnosticsExceptionEvent.java index 64fb2c6403..8f4686d2f5 100644 --- a/samza-core/src/main/scala/org/apache/samza/diagnostics/DiagnosticsExceptionEvent.java +++ b/samza-core/src/main/scala/org/apache/samza/diagnostics/DiagnosticsExceptionEvent.java @@ -18,6 +18,9 @@ */ package org.apache.samza.diagnostics; +import java.util.Map; + + /** * This class encapsulates information related to an exception event that is useful for diagnostics. * It used to define container, task, and other metrics as @@ -27,10 +30,13 @@ public class DiagnosticsExceptionEvent { private long timestamp; // the timestamp associated with this exception private Throwable throwable; + private Map mdcMap; + // the MDC map associated with this exception, used to store/obtain any context associated with the throwable - public DiagnosticsExceptionEvent(long timestampMillis, Throwable throwable) { + public DiagnosticsExceptionEvent(long timestampMillis, Throwable throwable, Map mdcMap) { this.throwable = throwable; this.timestamp = timestampMillis; + this.mdcMap = mdcMap; } public long getTimestamp() { @@ -40,4 +46,8 @@ public long getTimestamp() { public Throwable getThrowable() { return this.throwable; } + + public Map getMdcMap() { + return mdcMap; + } } \ No newline at end of file From 7f7885b5dd5d6bf5a74b9f2cabe5466435aeb616 Mon Sep 17 00:00:00 2001 From: Ray Matharu Date: Wed, 11 Jul 2018 15:19:17 -0700 Subject: [PATCH 25/33] Adding default constructor to enable serializing and deserializing --- .../DiagnosticsExceptionEvent.java | 25 +++++++++++++++++-- .../samza/metrics/reporter/Metrics.scala | 6 +++++ 2 files changed, 29 insertions(+), 2 deletions(-) diff --git a/samza-core/src/main/scala/org/apache/samza/diagnostics/DiagnosticsExceptionEvent.java b/samza-core/src/main/scala/org/apache/samza/diagnostics/DiagnosticsExceptionEvent.java index 8f4686d2f5..d2e0ef2f07 100644 --- a/samza-core/src/main/scala/org/apache/samza/diagnostics/DiagnosticsExceptionEvent.java +++ b/samza-core/src/main/scala/org/apache/samza/diagnostics/DiagnosticsExceptionEvent.java @@ -18,6 +18,8 @@ */ package org.apache.samza.diagnostics; +import java.util.Arrays; +import java.util.HashMap; import java.util.Map; @@ -31,12 +33,15 @@ public class DiagnosticsExceptionEvent { private long timestamp; // the timestamp associated with this exception private Throwable throwable; private Map mdcMap; - // the MDC map associated with this exception, used to store/obtain any context associated with the throwable + // the MDC map associated with this exception, used to store/obtain any context associated with the throwable + + public DiagnosticsExceptionEvent() { + } public DiagnosticsExceptionEvent(long timestampMillis, Throwable throwable, Map mdcMap) { this.throwable = throwable; this.timestamp = timestampMillis; - this.mdcMap = mdcMap; + this.mdcMap = new HashMap(mdcMap); } public long getTimestamp() { @@ -50,4 +55,20 @@ public Throwable getThrowable() { public Map getMdcMap() { return mdcMap; } + + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + DiagnosticsExceptionEvent that = (DiagnosticsExceptionEvent) o; + + // Throwable provides no equals impl, so we assume, Class, Message & stacktrace equality suffices + return timestamp == that.timestamp && mdcMap.equals(that.mdcMap) && this.throwable.getClass() + .equals(that.throwable.getClass()) && this.throwable.getMessage().equals(that.throwable.getMessage()) + && Arrays.equals(this.throwable.getStackTrace(), that.throwable.getStackTrace()); + } } \ No newline at end of file diff --git a/samza-core/src/main/scala/org/apache/samza/metrics/reporter/Metrics.scala b/samza-core/src/main/scala/org/apache/samza/metrics/reporter/Metrics.scala index 8a58cd2e80..111b46411d 100644 --- a/samza-core/src/main/scala/org/apache/samza/metrics/reporter/Metrics.scala +++ b/samza-core/src/main/scala/org/apache/samza/metrics/reporter/Metrics.scala @@ -19,9 +19,11 @@ package org.apache.samza.metrics.reporter +import java.util import java.util.Collections import java.util.HashMap import java.util.Map + import scala.collection.JavaConverters._ object Metrics { @@ -52,4 +54,8 @@ class Metrics(metrics: Map[String, Map[String, Object]]) { def get(group: String) = immutableMetrics.get(group) def getAsMap(): Map[String, Map[String, Object]] = Collections.unmodifiableMap(immutableMetrics) + + def this() { + this(new util.HashMap[String, Map[String, Object]]()) + } } From 92a29b47907a5c4ae806c1e3a5e0fb73795d7da2 Mon Sep 17 00:00:00 2001 From: Ray Matharu Date: Wed, 11 Jul 2018 15:19:58 -0700 Subject: [PATCH 26/33] Adding SnapshotSerdeV2 to enable serializing of ExceptionEvents for diagnostics, with unittest --- .../serializers/MetricsSnapshotSerdeV2.java | 80 +++++++++++++++++++ .../MetricsSnapshotSerdeV2Factory.java | 31 +++++++ .../TestMetricsSnapshotSerdeV2.java | 48 +++++++++++ 3 files changed, 159 insertions(+) create mode 100644 samza-core/src/main/scala/org/apache/samza/serializers/MetricsSnapshotSerdeV2.java create mode 100644 samza-core/src/main/scala/org/apache/samza/serializers/MetricsSnapshotSerdeV2Factory.java create mode 100644 samza-core/src/test/java/org/apache/samza/serializers/model/serializers/TestMetricsSnapshotSerdeV2.java diff --git a/samza-core/src/main/scala/org/apache/samza/serializers/MetricsSnapshotSerdeV2.java b/samza-core/src/main/scala/org/apache/samza/serializers/MetricsSnapshotSerdeV2.java new file mode 100644 index 0000000000..cbb110727a --- /dev/null +++ b/samza-core/src/main/scala/org/apache/samza/serializers/MetricsSnapshotSerdeV2.java @@ -0,0 +1,80 @@ +/* + * 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.serializers; + +import java.io.IOException; +import java.util.HashMap; +import java.util.Map; +import org.apache.samza.metrics.reporter.MetricsSnapshot; +import org.codehaus.jackson.annotate.JsonAutoDetect; +import org.codehaus.jackson.annotate.JsonMethod; +import org.codehaus.jackson.map.DeserializationConfig; +import org.codehaus.jackson.map.ObjectMapper; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + + +public class MetricsSnapshotSerdeV2 implements Serde { + + private static final Logger LOG = LoggerFactory.getLogger(MetricsSnapshotSerdeV2.class); + private final ObjectMapper jsonMapper; + + public MetricsSnapshotSerdeV2() { + jsonMapper = new ObjectMapper(); + jsonMapper.configure(DeserializationConfig.Feature.FAIL_ON_UNKNOWN_PROPERTIES, false); + jsonMapper.setVisibility(JsonMethod.ALL, JsonAutoDetect.Visibility.ANY); + jsonMapper.enableDefaultTyping(ObjectMapper.DefaultTyping.NON_FINAL); + // required to serialize Throwable with type and other info + } + + @Override + public MetricsSnapshot fromBytes(byte[] bytes) { + try { + return MetricsSnapshot.fromMap( + jsonMapper.readValue(bytes, new HashMap>().getClass())); + } catch (IOException e) { + LOG.info("Exception while deserializing", e); + } + return null; + } + + @Override + public byte[] toBytes(MetricsSnapshot metricsSnapshot) { + try { + return jsonMapper.writeValueAsString(convertMap(metricsSnapshot.getAsMap())).getBytes("UTF-8"); + } catch (IOException e) { + LOG.info("Exception while serializing", e); + } + return null; + } + + /** Unmodifiable maps should not be serialized with type, because UnmodifiableMap cannot be deserialized. + * So we convert to HashMap. This is a Jackson limitation. + */ + private HashMap convertMap(Map map) { + HashMap retVal = new HashMap(map); + for (Map.Entry entry : map.entrySet()) { + if (entry.getValue() instanceof Map) { + retVal.put(entry.getKey(), convertMap((Map) entry.getValue())); + } + } + return retVal; + } +} \ No newline at end of file diff --git a/samza-core/src/main/scala/org/apache/samza/serializers/MetricsSnapshotSerdeV2Factory.java b/samza-core/src/main/scala/org/apache/samza/serializers/MetricsSnapshotSerdeV2Factory.java new file mode 100644 index 0000000000..49e07705e2 --- /dev/null +++ b/samza-core/src/main/scala/org/apache/samza/serializers/MetricsSnapshotSerdeV2Factory.java @@ -0,0 +1,31 @@ +/* + * 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.serializers; + +import org.apache.samza.config.Config; +import org.apache.samza.metrics.reporter.MetricsSnapshot; + + +public class MetricsSnapshotSerdeV2Factory implements SerdeFactory { + @Override + public Serde getSerde(String name, Config config) { + return new MetricsSnapshotSerdeV2(); + } +} diff --git a/samza-core/src/test/java/org/apache/samza/serializers/model/serializers/TestMetricsSnapshotSerdeV2.java b/samza-core/src/test/java/org/apache/samza/serializers/model/serializers/TestMetricsSnapshotSerdeV2.java new file mode 100644 index 0000000000..6c888bff70 --- /dev/null +++ b/samza-core/src/test/java/org/apache/samza/serializers/model/serializers/TestMetricsSnapshotSerdeV2.java @@ -0,0 +1,48 @@ +package org.apache.samza.serializers.model.serializers; + +import java.util.HashMap; +import java.util.Map; +import org.apache.samza.SamzaException; +import org.apache.samza.diagnostics.DiagnosticsExceptionEvent; +import org.apache.samza.metrics.ListGauge; +import org.apache.samza.metrics.reporter.Metrics; +import org.apache.samza.metrics.reporter.MetricsHeader; +import org.apache.samza.metrics.reporter.MetricsSnapshot; +import org.apache.samza.serializers.MetricsSnapshotSerdeV2; +import org.junit.Assert; +import org.junit.Test; + + +public class TestMetricsSnapshotSerdeV2 { + + @Test + public void testSerde() { + MetricsHeader metricsHeader = + new MetricsHeader("jobName", "i001", "container 0", "source", "300.14.25.1", "1", "1", 1, 1); + + ListGauge listGauge = new ListGauge("exceptions"); + DiagnosticsExceptionEvent diagnosticsExceptionEvent = + new DiagnosticsExceptionEvent(1, new SamzaException("this is a samza exception", new RuntimeException("cause")), + new HashMap()); + listGauge.add(diagnosticsExceptionEvent); + + String samzaContainerMetricsGroupName = "org.apache.samza.container.SamzaContainerMetrics"; + Map> metricMessage = new HashMap<>(); + metricMessage.put(samzaContainerMetricsGroupName, new HashMap<>()); + metricMessage.get(samzaContainerMetricsGroupName).put("exceptions", listGauge.getValues()); + metricMessage.get(samzaContainerMetricsGroupName).put("commit-calls", 0); + + MetricsSnapshot metricsSnapshot = new MetricsSnapshot(metricsHeader, new Metrics(metricMessage)); + + MetricsSnapshotSerdeV2 metricsSnapshotSerde = new MetricsSnapshotSerdeV2(); + byte[] serializedBytes = metricsSnapshotSerde.toBytes(metricsSnapshot); + + MetricsSnapshot deserializedMetricsSnapshot = metricsSnapshotSerde.fromBytes(serializedBytes); + + Assert.assertTrue("Headers map should be equal", + metricsSnapshot.getHeader().getAsMap().equals(deserializedMetricsSnapshot.getHeader().getAsMap())); + + Assert.assertTrue("Metrics map should be equal", + metricsSnapshot.getMetrics().getAsMap().equals(deserializedMetricsSnapshot.getMetrics().getAsMap())); + } +} From e9f4004202906f223ccf6a7073e5432031fb00ce Mon Sep 17 00:00:00 2001 From: Ray Matharu Date: Fri, 13 Jul 2018 17:13:34 -0700 Subject: [PATCH 27/33] Serializing exceptions as throwables, storing classtype separately. --- .../DiagnosticsExceptionEvent.java | 14 +++++--- .../serializers/MetricsSnapshotSerdeV2.java | 21 +++++------ .../TestMetricsSnapshotSerdeV2.java | 36 +++++++++++++++++-- 3 files changed, 52 insertions(+), 19 deletions(-) diff --git a/samza-core/src/main/scala/org/apache/samza/diagnostics/DiagnosticsExceptionEvent.java b/samza-core/src/main/scala/org/apache/samza/diagnostics/DiagnosticsExceptionEvent.java index d2e0ef2f07..49db25f7a6 100644 --- a/samza-core/src/main/scala/org/apache/samza/diagnostics/DiagnosticsExceptionEvent.java +++ b/samza-core/src/main/scala/org/apache/samza/diagnostics/DiagnosticsExceptionEvent.java @@ -31,6 +31,7 @@ public class DiagnosticsExceptionEvent { private long timestamp; // the timestamp associated with this exception + private Class exceptionType; // store the exception type separately private Throwable throwable; private Map mdcMap; // the MDC map associated with this exception, used to store/obtain any context associated with the throwable @@ -40,6 +41,7 @@ public DiagnosticsExceptionEvent() { public DiagnosticsExceptionEvent(long timestampMillis, Throwable throwable, Map mdcMap) { this.throwable = throwable; + this.exceptionType = throwable.getClass(); this.timestamp = timestampMillis; this.mdcMap = new HashMap(mdcMap); } @@ -52,6 +54,10 @@ public Throwable getThrowable() { return this.throwable; } + public Class getExceptionType() { + return this.exceptionType; + } + public Map getMdcMap() { return mdcMap; } @@ -66,9 +72,9 @@ public boolean equals(Object o) { } DiagnosticsExceptionEvent that = (DiagnosticsExceptionEvent) o; - // Throwable provides no equals impl, so we assume, Class, Message & stacktrace equality suffices - return timestamp == that.timestamp && mdcMap.equals(that.mdcMap) && this.throwable.getClass() - .equals(that.throwable.getClass()) && this.throwable.getMessage().equals(that.throwable.getMessage()) - && Arrays.equals(this.throwable.getStackTrace(), that.throwable.getStackTrace()); + // Throwable provides no equals impl, so we assume message & stacktrace equality suffices + return timestamp == that.timestamp && this.exceptionType.equals(that.exceptionType) && mdcMap.equals(that.mdcMap) + && this.throwable.getMessage().equals(that.throwable.getMessage()) && Arrays.equals( + this.throwable.getStackTrace(), that.throwable.getStackTrace()); } } \ No newline at end of file diff --git a/samza-core/src/main/scala/org/apache/samza/serializers/MetricsSnapshotSerdeV2.java b/samza-core/src/main/scala/org/apache/samza/serializers/MetricsSnapshotSerdeV2.java index cbb110727a..6ab7ce8f4f 100644 --- a/samza-core/src/main/scala/org/apache/samza/serializers/MetricsSnapshotSerdeV2.java +++ b/samza-core/src/main/scala/org/apache/samza/serializers/MetricsSnapshotSerdeV2.java @@ -23,9 +23,6 @@ import java.util.HashMap; import java.util.Map; import org.apache.samza.metrics.reporter.MetricsSnapshot; -import org.codehaus.jackson.annotate.JsonAutoDetect; -import org.codehaus.jackson.annotate.JsonMethod; -import org.codehaus.jackson.map.DeserializationConfig; import org.codehaus.jackson.map.ObjectMapper; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -34,21 +31,18 @@ public class MetricsSnapshotSerdeV2 implements Serde { private static final Logger LOG = LoggerFactory.getLogger(MetricsSnapshotSerdeV2.class); - private final ObjectMapper jsonMapper; + private final ObjectMapper objectMapper; public MetricsSnapshotSerdeV2() { - jsonMapper = new ObjectMapper(); - jsonMapper.configure(DeserializationConfig.Feature.FAIL_ON_UNKNOWN_PROPERTIES, false); - jsonMapper.setVisibility(JsonMethod.ALL, JsonAutoDetect.Visibility.ANY); - jsonMapper.enableDefaultTyping(ObjectMapper.DefaultTyping.NON_FINAL); - // required to serialize Throwable with type and other info + objectMapper = new ObjectMapper(); + objectMapper.enableDefaultTyping(ObjectMapper.DefaultTyping.OBJECT_AND_NON_CONCRETE); } @Override public MetricsSnapshot fromBytes(byte[] bytes) { try { return MetricsSnapshot.fromMap( - jsonMapper.readValue(bytes, new HashMap>().getClass())); + objectMapper.readValue(bytes, new HashMap>().getClass())); } catch (IOException e) { LOG.info("Exception while deserializing", e); } @@ -58,15 +52,16 @@ public MetricsSnapshot fromBytes(byte[] bytes) { @Override public byte[] toBytes(MetricsSnapshot metricsSnapshot) { try { - return jsonMapper.writeValueAsString(convertMap(metricsSnapshot.getAsMap())).getBytes("UTF-8"); + return objectMapper.writeValueAsString(convertMap(metricsSnapshot.getAsMap())).getBytes("UTF-8"); } catch (IOException e) { LOG.info("Exception while serializing", e); } return null; } - /** Unmodifiable maps should not be serialized with type, because UnmodifiableMap cannot be deserialized. - * So we convert to HashMap. This is a Jackson limitation. + /** Metrics returns an UnmodifiableMap. + * Unmodifiable maps should not be serialized with type, because UnmodifiableMap cannot be deserialized. + * So we convert to HashMap. */ private HashMap convertMap(Map map) { HashMap retVal = new HashMap(map); diff --git a/samza-core/src/test/java/org/apache/samza/serializers/model/serializers/TestMetricsSnapshotSerdeV2.java b/samza-core/src/test/java/org/apache/samza/serializers/model/serializers/TestMetricsSnapshotSerdeV2.java index 6c888bff70..86017837c5 100644 --- a/samza-core/src/test/java/org/apache/samza/serializers/model/serializers/TestMetricsSnapshotSerdeV2.java +++ b/samza-core/src/test/java/org/apache/samza/serializers/model/serializers/TestMetricsSnapshotSerdeV2.java @@ -1,7 +1,29 @@ +/* + * 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.serializers.model.serializers; +import java.util.Collections; import java.util.HashMap; import java.util.Map; +import org.apache.kafka.common.errors.RecordTooLargeException; +import org.apache.kafka.common.errors.TopicAuthorizationException; import org.apache.samza.SamzaException; import org.apache.samza.diagnostics.DiagnosticsExceptionEvent; import org.apache.samza.metrics.ListGauge; @@ -21,10 +43,20 @@ public void testSerde() { new MetricsHeader("jobName", "i001", "container 0", "source", "300.14.25.1", "1", "1", 1, 1); ListGauge listGauge = new ListGauge("exceptions"); - DiagnosticsExceptionEvent diagnosticsExceptionEvent = + DiagnosticsExceptionEvent diagnosticsExceptionEvent1 = new DiagnosticsExceptionEvent(1, new SamzaException("this is a samza exception", new RuntimeException("cause")), new HashMap()); - listGauge.add(diagnosticsExceptionEvent); + + DiagnosticsExceptionEvent diagnosticsExceptionEvent2 = new DiagnosticsExceptionEvent(2, + new SamzaException("this is a samza exception", new TopicAuthorizationException(Collections.singleton("foo"))), + new HashMap()); + + DiagnosticsExceptionEvent diagnosticsExceptionEvent3 = + new DiagnosticsExceptionEvent(2, new RecordTooLargeException("this record is too large"), new HashMap()); + + listGauge.add(diagnosticsExceptionEvent1); + listGauge.add(diagnosticsExceptionEvent2); + listGauge.add(diagnosticsExceptionEvent3); String samzaContainerMetricsGroupName = "org.apache.samza.container.SamzaContainerMetrics"; Map> metricMessage = new HashMap<>(); From 28c4fec44fcfa66fd474d7c8a2e19ed4585ab3af Mon Sep 17 00:00:00 2001 From: Ray Matharu Date: Fri, 13 Jul 2018 18:52:34 -0700 Subject: [PATCH 28/33] Minor changes to ensure build --- .../samza/diagnostics/DiagnosticsExceptionEvent.java | 6 ++++++ .../serializers/TestMetricsSnapshotSerdeV2.java | 12 ------------ 2 files changed, 6 insertions(+), 12 deletions(-) diff --git a/samza-core/src/main/scala/org/apache/samza/diagnostics/DiagnosticsExceptionEvent.java b/samza-core/src/main/scala/org/apache/samza/diagnostics/DiagnosticsExceptionEvent.java index 49db25f7a6..d87249e788 100644 --- a/samza-core/src/main/scala/org/apache/samza/diagnostics/DiagnosticsExceptionEvent.java +++ b/samza-core/src/main/scala/org/apache/samza/diagnostics/DiagnosticsExceptionEvent.java @@ -21,6 +21,7 @@ import java.util.Arrays; import java.util.HashMap; import java.util.Map; +import java.util.Objects; /** @@ -77,4 +78,9 @@ public boolean equals(Object o) { && this.throwable.getMessage().equals(that.throwable.getMessage()) && Arrays.equals( this.throwable.getStackTrace(), that.throwable.getStackTrace()); } + + @Override + public int hashCode() { + return Objects.hash(timestamp, exceptionType, throwable, mdcMap); + } } \ No newline at end of file diff --git a/samza-core/src/test/java/org/apache/samza/serializers/model/serializers/TestMetricsSnapshotSerdeV2.java b/samza-core/src/test/java/org/apache/samza/serializers/model/serializers/TestMetricsSnapshotSerdeV2.java index 86017837c5..e4255a7acf 100644 --- a/samza-core/src/test/java/org/apache/samza/serializers/model/serializers/TestMetricsSnapshotSerdeV2.java +++ b/samza-core/src/test/java/org/apache/samza/serializers/model/serializers/TestMetricsSnapshotSerdeV2.java @@ -19,11 +19,8 @@ package org.apache.samza.serializers.model.serializers; -import java.util.Collections; import java.util.HashMap; import java.util.Map; -import org.apache.kafka.common.errors.RecordTooLargeException; -import org.apache.kafka.common.errors.TopicAuthorizationException; import org.apache.samza.SamzaException; import org.apache.samza.diagnostics.DiagnosticsExceptionEvent; import org.apache.samza.metrics.ListGauge; @@ -47,16 +44,7 @@ public void testSerde() { new DiagnosticsExceptionEvent(1, new SamzaException("this is a samza exception", new RuntimeException("cause")), new HashMap()); - DiagnosticsExceptionEvent diagnosticsExceptionEvent2 = new DiagnosticsExceptionEvent(2, - new SamzaException("this is a samza exception", new TopicAuthorizationException(Collections.singleton("foo"))), - new HashMap()); - - DiagnosticsExceptionEvent diagnosticsExceptionEvent3 = - new DiagnosticsExceptionEvent(2, new RecordTooLargeException("this record is too large"), new HashMap()); - listGauge.add(diagnosticsExceptionEvent1); - listGauge.add(diagnosticsExceptionEvent2); - listGauge.add(diagnosticsExceptionEvent3); String samzaContainerMetricsGroupName = "org.apache.samza.container.SamzaContainerMetrics"; Map> metricMessage = new HashMap<>(); From edbd2006de3ba59d68694549eed55f2d98a2357e Mon Sep 17 00:00:00 2001 From: Ray Matharu Date: Mon, 23 Jul 2018 16:03:25 -0700 Subject: [PATCH 29/33] Fixing dependencies in Metrics.scala --- .../org/apache/samza/metrics/reporter/Metrics.scala | 9 +++------ 1 file changed, 3 insertions(+), 6 deletions(-) diff --git a/samza-core/src/main/scala/org/apache/samza/metrics/reporter/Metrics.scala b/samza-core/src/main/scala/org/apache/samza/metrics/reporter/Metrics.scala index 111b46411d..218157e846 100644 --- a/samza-core/src/main/scala/org/apache/samza/metrics/reporter/Metrics.scala +++ b/samza-core/src/main/scala/org/apache/samza/metrics/reporter/Metrics.scala @@ -19,11 +19,7 @@ package org.apache.samza.metrics.reporter -import java.util -import java.util.Collections -import java.util.HashMap -import java.util.Map - +import java.util.{Collections, HashMap, Map} import scala.collection.JavaConverters._ object Metrics { @@ -55,7 +51,8 @@ class Metrics(metrics: Map[String, Map[String, Object]]) { def getAsMap(): Map[String, Map[String, Object]] = Collections.unmodifiableMap(immutableMetrics) + // default constructor to enable deserialization by MetricsSnapshotSerdeV2 def this() { - this(new util.HashMap[String, Map[String, Object]]()) + this(new HashMap[String, Map[String, Object]]()) } } From f8ace28e7648f6a78fd5aa41dd807d033d15d354 Mon Sep 17 00:00:00 2001 From: Ray Matharu Date: Mon, 23 Jul 2018 23:51:40 -0700 Subject: [PATCH 30/33] Moving addAppender to DiagnosticsAppender, using config and reflection to instantiate --- .../org/apache/samza/config/JobConfig.scala | 6 +++++ .../samza/container/SamzaContainer.scala | 15 +++-------- .../log4j/SimpleDiagnosticsAppender.java | 26 +++++++++++++++---- 3 files changed, 31 insertions(+), 16 deletions(-) rename samza-core/src/main/scala/org/apache/samza/diagnostics/DiagnosticsAppender.java => samza-log4j/src/main/java/org/apache/samza/logging/log4j/SimpleDiagnosticsAppender.java (71%) diff --git a/samza-core/src/main/scala/org/apache/samza/config/JobConfig.scala b/samza-core/src/main/scala/org/apache/samza/config/JobConfig.scala index fb6ce0a073..1f4a47ba52 100644 --- a/samza-core/src/main/scala/org/apache/samza/config/JobConfig.scala +++ b/samza-core/src/main/scala/org/apache/samza/config/JobConfig.scala @@ -91,6 +91,10 @@ object JobConfig { // Enables diagnostic appender for logging exception events val JOB_DIAGNOSTICS_ENABLED = "job.diagnostics.enabled" + // Specify DiagnosticAppender class + val DIAGNOSTICS_APPENDER_CLASS = "job.diagnostics.appender.class" + + implicit def Config2Job(config: Config) = new JobConfig(config) /** @@ -191,4 +195,6 @@ class JobConfig(config: Config) extends ScalaMapConfig(config) with Logging { def getLoggedStorePath = getOption(JobConfig.JOB_LOGGED_STORE_BASE_DIR) def getDiagnosticsEnabled = { getBoolean(JobConfig.JOB_DIAGNOSTICS_ENABLED, false) } + + def getDiagnosticsAppenderClass = { getOption(JobConfig.DIAGNOSTICS_APPENDER_CLASS) } } diff --git a/samza-core/src/main/scala/org/apache/samza/container/SamzaContainer.scala b/samza-core/src/main/scala/org/apache/samza/container/SamzaContainer.scala index d61c4ed3d7..f13d9db5dd 100644 --- a/samza-core/src/main/scala/org/apache/samza/container/SamzaContainer.scala +++ b/samza-core/src/main/scala/org/apache/samza/container/SamzaContainer.scala @@ -30,7 +30,6 @@ import java.util.concurrent.{ExecutorService, Executors, ScheduledExecutorServic import com.google.common.annotations.VisibleForTesting import com.google.common.util.concurrent.ThreadFactoryBuilder -import org.apache.commons.lang3.exception.ExceptionUtils import org.apache.samza.checkpoint.{CheckpointListener, CheckpointManagerFactory, OffsetManager, OffsetManagerMetrics} import org.apache.samza.config.JobConfig.Config2Job import org.apache.samza.config.MetricsConfig.Config2Metrics @@ -53,11 +52,8 @@ import org.apache.samza.system._ import org.apache.samza.system.chooser.{DefaultChooser, MessageChooserFactory, RoundRobinChooserFactory} import org.apache.samza.table.TableManager import org.apache.samza.task._ -import org.apache.samza.util.Util -import org.apache.samza.util._ +import org.apache.samza.util.{Util, _} import org.apache.samza.{SamzaContainerStatus, SamzaException} -import org.apache.samza.diagnostics.DiagnosticsAppender; -import org.apache.log4j.Logger import scala.collection.JavaConverters._ @@ -909,12 +905,9 @@ class SamzaContainer( def startDiagnostics { if (containerContext.config.getDiagnosticsEnabled) { - val rootLogger = Logger.getRootLogger - - if (rootLogger.getAppender(classOf[DiagnosticsAppender].getName) == null) { - info("Starting diagnostics appender.") - rootLogger.addAppender(new DiagnosticsAppender(this.metrics)) - } + info("Starting diagnostics.") + val diagnosticsAppender = Class.forName(containerContext.config.getDiagnosticsAppenderClass.get). + getDeclaredConstructor(classOf[SamzaContainerMetrics]).newInstance(this.metrics); } } diff --git a/samza-core/src/main/scala/org/apache/samza/diagnostics/DiagnosticsAppender.java b/samza-log4j/src/main/java/org/apache/samza/logging/log4j/SimpleDiagnosticsAppender.java similarity index 71% rename from samza-core/src/main/scala/org/apache/samza/diagnostics/DiagnosticsAppender.java rename to samza-log4j/src/main/java/org/apache/samza/logging/log4j/SimpleDiagnosticsAppender.java index e3f5153fbf..b0ca6e10ef 100644 --- a/samza-core/src/main/scala/org/apache/samza/diagnostics/DiagnosticsAppender.java +++ b/samza-log4j/src/main/java/org/apache/samza/logging/log4j/SimpleDiagnosticsAppender.java @@ -17,11 +17,12 @@ * under the License. */ -package org.apache.samza.diagnostics; +package org.apache.samza.logging.log4j; import org.apache.log4j.AppenderSkeleton; import org.apache.log4j.spi.LoggingEvent; import org.apache.samza.container.SamzaContainerMetrics; +import org.apache.samza.diagnostics.DiagnosticsExceptionEvent; import org.apache.samza.metrics.ListGauge; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -35,14 +36,20 @@ * When used inconjunction with {@link org.apache.samza.metrics.reporter.MetricsSnapshotReporter} provides a * stream of diagnostics-related events. */ -public class DiagnosticsAppender extends AppenderSkeleton { +public class SimpleDiagnosticsAppender extends AppenderSkeleton { - private static final Logger LOG = LoggerFactory.getLogger(DiagnosticsAppender.class); + private static final Logger LOG = LoggerFactory.getLogger(SimpleDiagnosticsAppender.class); private final ListGauge samzaContainerExceptionMetric; - public DiagnosticsAppender(SamzaContainerMetrics samzaContainerMetrics) { + public SimpleDiagnosticsAppender(SamzaContainerMetrics samzaContainerMetrics) { this.samzaContainerExceptionMetric = samzaContainerMetrics.exceptions(); - this.setName(DiagnosticsAppender.class.getName()); + this.setName(SimpleDiagnosticsAppender.class.getName()); + + // ensure appender is attached only once per JVM (regardless of #containers) + if (org.apache.log4j.Logger.getRootLogger().getAppender(SimpleDiagnosticsAppender.class.getName()) == null) { + LOG.info("Attaching diagnostics appender to root logger"); + org.apache.log4j.Logger.getRootLogger().addAppender(this); + } } @Override @@ -79,4 +86,13 @@ public void close() { public boolean requiresLayout() { return false; } + + public static void attach(SamzaContainerMetrics metrics) { + org.apache.log4j.Logger rootLogger = org.apache.log4j.Logger.getRootLogger(); + + if (rootLogger.getAppender(SimpleDiagnosticsAppender.class.getName()) == null) { + LOG.info("Starting diagnostics appender."); + rootLogger.addAppender(new SimpleDiagnosticsAppender(metrics)); + } + } } From a4d11b0a1edc2efde7beeef57af18602ffc3cfc1 Mon Sep 17 00:00:00 2001 From: Ray Matharu Date: Tue, 24 Jul 2018 00:05:57 -0700 Subject: [PATCH 31/33] efactoring useless code out --- .../scala/org/apache/samza/config/JobConfig.scala | 3 +-- .../logging/log4j/SimpleDiagnosticsAppender.java | 11 +---------- 2 files changed, 2 insertions(+), 12 deletions(-) diff --git a/samza-core/src/main/scala/org/apache/samza/config/JobConfig.scala b/samza-core/src/main/scala/org/apache/samza/config/JobConfig.scala index 1f4a47ba52..f23737f0c9 100644 --- a/samza-core/src/main/scala/org/apache/samza/config/JobConfig.scala +++ b/samza-core/src/main/scala/org/apache/samza/config/JobConfig.scala @@ -93,8 +93,7 @@ object JobConfig { // Specify DiagnosticAppender class val DIAGNOSTICS_APPENDER_CLASS = "job.diagnostics.appender.class" - - + implicit def Config2Job(config: Config) = new JobConfig(config) /** diff --git a/samza-log4j/src/main/java/org/apache/samza/logging/log4j/SimpleDiagnosticsAppender.java b/samza-log4j/src/main/java/org/apache/samza/logging/log4j/SimpleDiagnosticsAppender.java index b0ca6e10ef..042bb7dd48 100644 --- a/samza-log4j/src/main/java/org/apache/samza/logging/log4j/SimpleDiagnosticsAppender.java +++ b/samza-log4j/src/main/java/org/apache/samza/logging/log4j/SimpleDiagnosticsAppender.java @@ -39,7 +39,7 @@ public class SimpleDiagnosticsAppender extends AppenderSkeleton { private static final Logger LOG = LoggerFactory.getLogger(SimpleDiagnosticsAppender.class); - private final ListGauge samzaContainerExceptionMetric; + protected final ListGauge samzaContainerExceptionMetric; public SimpleDiagnosticsAppender(SamzaContainerMetrics samzaContainerMetrics) { this.samzaContainerExceptionMetric = samzaContainerMetrics.exceptions(); @@ -86,13 +86,4 @@ public void close() { public boolean requiresLayout() { return false; } - - public static void attach(SamzaContainerMetrics metrics) { - org.apache.log4j.Logger rootLogger = org.apache.log4j.Logger.getRootLogger(); - - if (rootLogger.getAppender(SimpleDiagnosticsAppender.class.getName()) == null) { - LOG.info("Starting diagnostics appender."); - rootLogger.addAppender(new SimpleDiagnosticsAppender(metrics)); - } - } } From 6e274e7684705bf362c0315a98ec4dc70ed47595 Mon Sep 17 00:00:00 2001 From: Ray Matharu Date: Wed, 25 Jul 2018 17:56:51 -0700 Subject: [PATCH 32/33] Making appender-attachment thread safe, adding error logging for appender instantiation, adding default value in jobConfig --- .../scala/org/apache/samza/config/JobConfig.scala | 6 ++++-- .../apache/samza/container/SamzaContainer.scala | 14 ++++++++++++-- .../logging/log4j/SimpleDiagnosticsAppender.java | 14 +++++++++++++- 3 files changed, 29 insertions(+), 5 deletions(-) diff --git a/samza-core/src/main/scala/org/apache/samza/config/JobConfig.scala b/samza-core/src/main/scala/org/apache/samza/config/JobConfig.scala index f23737f0c9..5c815e54dc 100644 --- a/samza-core/src/main/scala/org/apache/samza/config/JobConfig.scala +++ b/samza-core/src/main/scala/org/apache/samza/config/JobConfig.scala @@ -93,7 +93,7 @@ object JobConfig { // Specify DiagnosticAppender class val DIAGNOSTICS_APPENDER_CLASS = "job.diagnostics.appender.class" - + implicit def Config2Job(config: Config) = new JobConfig(config) /** @@ -195,5 +195,7 @@ class JobConfig(config: Config) extends ScalaMapConfig(config) with Logging { def getDiagnosticsEnabled = { getBoolean(JobConfig.JOB_DIAGNOSTICS_ENABLED, false) } - def getDiagnosticsAppenderClass = { getOption(JobConfig.DIAGNOSTICS_APPENDER_CLASS) } + def getDiagnosticsAppenderClass = { + getOrDefault(JobConfig.DIAGNOSTICS_APPENDER_CLASS, "org.apache.samza.logging.log4j.SimpleDiagnosticsAppender") + } } diff --git a/samza-core/src/main/scala/org/apache/samza/container/SamzaContainer.scala b/samza-core/src/main/scala/org/apache/samza/container/SamzaContainer.scala index f13d9db5dd..3f96223aad 100644 --- a/samza-core/src/main/scala/org/apache/samza/container/SamzaContainer.scala +++ b/samza-core/src/main/scala/org/apache/samza/container/SamzaContainer.scala @@ -21,6 +21,7 @@ package org.apache.samza.container import java.io.File import java.lang.management.ManagementFactory +import java.lang.reflect.InvocationTargetException import java.net.{URL, UnknownHostException} import java.nio.file.Path import java.time.Duration @@ -906,8 +907,17 @@ class SamzaContainer( def startDiagnostics { if (containerContext.config.getDiagnosticsEnabled) { info("Starting diagnostics.") - val diagnosticsAppender = Class.forName(containerContext.config.getDiagnosticsAppenderClass.get). - getDeclaredConstructor(classOf[SamzaContainerMetrics]).newInstance(this.metrics); + + try { + val diagnosticsAppender = Class.forName(containerContext.config.getDiagnosticsAppenderClass). + getDeclaredConstructor(classOf[SamzaContainerMetrics]).newInstance(this.metrics); + } + catch { + case e@(_: ClassNotFoundException | _: InstantiationException | _: InvocationTargetException) => { + error("Failed to instantiate diagnostic appender", e) + throw new ConfigException("Failed to instantiate diagnostic appender class specified in config", e) + } + } } } diff --git a/samza-log4j/src/main/java/org/apache/samza/logging/log4j/SimpleDiagnosticsAppender.java b/samza-log4j/src/main/java/org/apache/samza/logging/log4j/SimpleDiagnosticsAppender.java index 042bb7dd48..31f0d47d8b 100644 --- a/samza-log4j/src/main/java/org/apache/samza/logging/log4j/SimpleDiagnosticsAppender.java +++ b/samza-log4j/src/main/java/org/apache/samza/logging/log4j/SimpleDiagnosticsAppender.java @@ -37,14 +37,26 @@ * stream of diagnostics-related events. */ public class SimpleDiagnosticsAppender extends AppenderSkeleton { - private static final Logger LOG = LoggerFactory.getLogger(SimpleDiagnosticsAppender.class); + + // simple object to synchronize root logger attachment + private static final Object SYNCHRONIZATION_OBJECT = new Object(); protected final ListGauge samzaContainerExceptionMetric; + /** + * A simple log4j1.2.* appender, which attaches itself to the root logger. + * Attachment to the root logger is thread safe. + */ public SimpleDiagnosticsAppender(SamzaContainerMetrics samzaContainerMetrics) { this.samzaContainerExceptionMetric = samzaContainerMetrics.exceptions(); this.setName(SimpleDiagnosticsAppender.class.getName()); + synchronized (SYNCHRONIZATION_OBJECT) { + this.attachAppenderToRootLogger(); + } + } + + private void attachAppenderToRootLogger() { // ensure appender is attached only once per JVM (regardless of #containers) if (org.apache.log4j.Logger.getRootLogger().getAppender(SimpleDiagnosticsAppender.class.getName()) == null) { LOG.info("Attaching diagnostics appender to root logger"); From 7c727f71d3fedf1d60c45bda1a6b3b850621ff22 Mon Sep 17 00:00:00 2001 From: Ray Matharu Date: Thu, 26 Jul 2018 15:13:21 -0700 Subject: [PATCH 33/33] Adding constant to store class, adding class name to log message --- .../src/main/scala/org/apache/samza/config/JobConfig.scala | 3 ++- .../main/scala/org/apache/samza/container/SamzaContainer.scala | 3 ++- 2 files changed, 4 insertions(+), 2 deletions(-) diff --git a/samza-core/src/main/scala/org/apache/samza/config/JobConfig.scala b/samza-core/src/main/scala/org/apache/samza/config/JobConfig.scala index 5c815e54dc..7cebcc6424 100644 --- a/samza-core/src/main/scala/org/apache/samza/config/JobConfig.scala +++ b/samza-core/src/main/scala/org/apache/samza/config/JobConfig.scala @@ -93,6 +93,7 @@ object JobConfig { // Specify DiagnosticAppender class val DIAGNOSTICS_APPENDER_CLASS = "job.diagnostics.appender.class" + val DEFAULT_DIAGNOSTICS_APPENDER_CLASS = "org.apache.samza.logging.log4j.SimpleDiagnosticsAppender" implicit def Config2Job(config: Config) = new JobConfig(config) @@ -196,6 +197,6 @@ class JobConfig(config: Config) extends ScalaMapConfig(config) with Logging { def getDiagnosticsEnabled = { getBoolean(JobConfig.JOB_DIAGNOSTICS_ENABLED, false) } def getDiagnosticsAppenderClass = { - getOrDefault(JobConfig.DIAGNOSTICS_APPENDER_CLASS, "org.apache.samza.logging.log4j.SimpleDiagnosticsAppender") + getOrDefault(JobConfig.DIAGNOSTICS_APPENDER_CLASS, JobConfig.DEFAULT_DIAGNOSTICS_APPENDER_CLASS) } } diff --git a/samza-core/src/main/scala/org/apache/samza/container/SamzaContainer.scala b/samza-core/src/main/scala/org/apache/samza/container/SamzaContainer.scala index 3f96223aad..ca3377a746 100644 --- a/samza-core/src/main/scala/org/apache/samza/container/SamzaContainer.scala +++ b/samza-core/src/main/scala/org/apache/samza/container/SamzaContainer.scala @@ -915,7 +915,8 @@ class SamzaContainer( catch { case e@(_: ClassNotFoundException | _: InstantiationException | _: InvocationTargetException) => { error("Failed to instantiate diagnostic appender", e) - throw new ConfigException("Failed to instantiate diagnostic appender class specified in config", e) + throw new ConfigException("Failed to instantiate diagnostic appender class " + + containerContext.config.getDiagnosticsAppenderClass, e) } } }