You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@pinot.apache.org by sn...@apache.org on 2022/12/15 07:56:28 UTC

[pinot] branch master updated: allow gauge stored in metric registry to be updated (#9961)

This is an automated email from the ASF dual-hosted git repository.

snlee pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/pinot.git


The following commit(s) were added to refs/heads/master by this push:
     new 8bb551e27b allow gauge stored in metric registry to be updated (#9961)
8bb551e27b is described below

commit 8bb551e27b5295d693d6b80d26c83b7e44f0d126
Author: Haitao Zhang <ha...@startree.ai>
AuthorDate: Wed Dec 14 23:56:22 2022 -0800

    allow gauge stored in metric registry to be updated (#9961)
    
    * allow gauge stored in metric registry to be updated
    
    * fix existing tests and add new tests
    
    * address comments
---
 .../pinot/common/metrics/AbstractMetrics.java      | 25 +++++++--
 .../pinot/common/metrics/AbstractMetricsTest.java  | 65 ++++++++++++++++++++++
 .../plugin/metrics/dropwizard/DropwizardGauge.java | 27 ++++++---
 .../dropwizard/DropwizardMetricsRegistry.java      |  5 +-
 ...zardGauge.java => DropwizardSettableGauge.java} | 38 +++++++------
 .../metrics/dropwizard/DropwizardGaugeTest.java}   | 38 +++++--------
 .../dropwizard/DropwizardMetricsRegistryTest.java} | 43 +++++---------
 .../dropwizard/DropwizardSettableGaugeTest.java    | 45 +++++++++++++++
 .../pinot/plugin/metrics/yammer/YammerGauge.java   | 31 ++++++-----
 .../metrics/yammer/YammerMetricsRegistry.java      |  2 +-
 .../{YammerGauge.java => YammerSettableGauge.java} | 39 +++++++------
 .../plugin/metrics/yammer/YammerGaugeTest.java}    | 43 +++++---------
 .../metrics/yammer/YammerMetricsRegistryTest.java} | 48 +++++-----------
 .../metrics/yammer/YammerSettableGaugeTest.java    | 45 +++++++++++++++
 .../org/apache/pinot/spi/metrics/PinotGauge.java   | 16 +++++-
 .../{PinotGauge.java => SettableValue.java}        | 33 +++++------
 16 files changed, 337 insertions(+), 206 deletions(-)

diff --git a/pinot-common/src/main/java/org/apache/pinot/common/metrics/AbstractMetrics.java b/pinot-common/src/main/java/org/apache/pinot/common/metrics/AbstractMetrics.java
index 740ae7c4b9..cb9759714b 100644
--- a/pinot-common/src/main/java/org/apache/pinot/common/metrics/AbstractMetrics.java
+++ b/pinot-common/src/main/java/org/apache/pinot/common/metrics/AbstractMetrics.java
@@ -28,8 +28,10 @@ import java.util.concurrent.Callable;
 import java.util.concurrent.ConcurrentHashMap;
 import java.util.concurrent.TimeUnit;
 import java.util.concurrent.atomic.AtomicLong;
+import java.util.function.Supplier;
 import java.util.stream.Collectors;
 import org.apache.pinot.common.Utils;
+import org.apache.pinot.spi.metrics.PinotGauge;
 import org.apache.pinot.spi.metrics.PinotMeter;
 import org.apache.pinot.spi.metrics.PinotMetricName;
 import org.apache.pinot.spi.metrics.PinotMetricUtils;
@@ -42,7 +44,8 @@ import org.slf4j.LoggerFactory;
 
 /**
  * Common code for metrics implementations.
- *
+ * TODO: 1. With gauge updatable, we can remove _gaugeValues 2. Remove methods with callback in name since the callback
+ *   function can not be updated.
  */
 public abstract class AbstractMetrics<QP extends AbstractMetrics.QueryPhase, M extends AbstractMetrics.Meter,
     G extends AbstractMetrics.Gauge, T extends AbstractMetrics.Timer> {
@@ -479,6 +482,7 @@ public abstract class AbstractMetrics<QP extends AbstractMetrics.QueryPhase, M e
 
   /**
    * Adds a new gauge whose values are retrieved from a callback function.
+   * Once added, the callback function cannot be updated.
    *
    * @param metricName The name of the metric
    * @param valueCallback The callback function used to retrieve the value of the gauge
@@ -497,6 +501,20 @@ public abstract class AbstractMetrics<QP extends AbstractMetrics.QueryPhase, M e
             }));
   }
 
+  /**
+   * Adds or updates a gauge whose values are retrieved from the given supplier function.
+   * The supplier function can be updated by calling this method again.
+   *
+   * @param metricName The name of the metric
+   * @param valueSupplier The supplier function used to retrieve the value of the gauge
+   */
+  public void addOrUpdateGauge(final String metricName, final Supplier<Long> valueSupplier) {
+    PinotGauge<Long> pinotGauge = PinotMetricUtils.makeGauge(_metricsRegistry,
+        PinotMetricUtils.makePinotMetricName(_clazz, _metricPrefix + metricName),
+        PinotMetricUtils.makePinotGauge(avoid -> valueSupplier.get()));
+    pinotGauge.setValueSupplier(valueSupplier);
+  }
+
   /**
    * Removes a table gauge given the table name and the gauge.
    * The add/remove is expected to work correctly in case of being invoked across multiple threads.
@@ -515,9 +533,8 @@ public abstract class AbstractMetrics<QP extends AbstractMetrics.QueryPhase, M e
    * @param gaugeName gauge name
    */
   public void removeGauge(final String gaugeName) {
-    if (_gaugeValues.remove(gaugeName) != null) {
-      removeCallbackGauge(gaugeName);
-    }
+    _gaugeValues.remove(gaugeName);
+    removeCallbackGauge(gaugeName);
   }
 
   /**
diff --git a/pinot-common/src/test/java/org/apache/pinot/common/metrics/AbstractMetricsTest.java b/pinot-common/src/test/java/org/apache/pinot/common/metrics/AbstractMetricsTest.java
new file mode 100644
index 0000000000..b1027986d1
--- /dev/null
+++ b/pinot-common/src/test/java/org/apache/pinot/common/metrics/AbstractMetricsTest.java
@@ -0,0 +1,65 @@
+/**
+ * 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.pinot.common.metrics;
+
+import com.yammer.metrics.core.MetricName;
+import org.apache.pinot.plugin.metrics.yammer.YammerMetric;
+import org.apache.pinot.plugin.metrics.yammer.YammerMetricName;
+import org.apache.pinot.plugin.metrics.yammer.YammerMetricsRegistry;
+import org.apache.pinot.plugin.metrics.yammer.YammerSettableGauge;
+import org.apache.pinot.spi.env.PinotConfiguration;
+import org.apache.pinot.spi.metrics.PinotMetric;
+import org.apache.pinot.spi.metrics.PinotMetricUtils;
+import org.testng.Assert;
+import org.testng.annotations.Test;
+
+import static org.apache.pinot.spi.utils.CommonConstants.CONFIG_OF_METRICS_FACTORY_CLASS_NAME;
+
+
+public class AbstractMetricsTest {
+  @Test
+  public void testAddOrUpdateGauge() {
+    PinotConfiguration pinotConfiguration = new PinotConfiguration();
+    pinotConfiguration.setProperty(CONFIG_OF_METRICS_FACTORY_CLASS_NAME,
+        "org.apache.pinot.plugin.metrics.yammer.YammerMetricsFactory");
+    PinotMetricUtils.init(pinotConfiguration);
+    ControllerMetrics controllerMetrics = new ControllerMetrics(new YammerMetricsRegistry());
+    String metricName = "test";
+    // add gauge
+    controllerMetrics.addOrUpdateGauge(metricName, () -> 1L);
+    checkGauge(controllerMetrics, metricName, 1);
+
+    // update gauge
+    controllerMetrics.addOrUpdateGauge(metricName, () -> 2L);
+    checkGauge(controllerMetrics, metricName, 2);
+
+    // remove gauge
+    controllerMetrics.removeGauge(metricName);
+    Assert.assertTrue(controllerMetrics.getMetricsRegistry().allMetrics().isEmpty());
+  }
+
+  private void checkGauge(ControllerMetrics controllerMetrics, String metricName, long value) {
+    Assert.assertEquals(controllerMetrics.getMetricsRegistry().allMetrics().size(), 1);
+    PinotMetric pinotMetric = controllerMetrics.getMetricsRegistry().allMetrics()
+        .get(new YammerMetricName(new MetricName(ControllerMetrics.class, "pinot.controller." + metricName)));
+    Assert.assertTrue(pinotMetric instanceof YammerMetric);
+    Assert.assertTrue(pinotMetric.getMetric() instanceof YammerSettableGauge);
+    Assert.assertEquals(((YammerSettableGauge<Long>) pinotMetric.getMetric()).value(), Long.valueOf(value));
+  }
+}
diff --git a/pinot-plugins/pinot-metrics/pinot-dropwizard/src/main/java/org/apache/pinot/plugin/metrics/dropwizard/DropwizardGauge.java b/pinot-plugins/pinot-metrics/pinot-dropwizard/src/main/java/org/apache/pinot/plugin/metrics/dropwizard/DropwizardGauge.java
index 01d0b24dc6..d9c2d8e668 100644
--- a/pinot-plugins/pinot-metrics/pinot-dropwizard/src/main/java/org/apache/pinot/plugin/metrics/dropwizard/DropwizardGauge.java
+++ b/pinot-plugins/pinot-metrics/pinot-dropwizard/src/main/java/org/apache/pinot/plugin/metrics/dropwizard/DropwizardGauge.java
@@ -18,35 +18,44 @@
  */
 package org.apache.pinot.plugin.metrics.dropwizard;
 
-import com.codahale.metrics.Gauge;
 import java.util.function.Function;
+import java.util.function.Supplier;
 import org.apache.pinot.spi.metrics.PinotGauge;
 
-
 public class DropwizardGauge<T> implements PinotGauge<T> {
 
-  private final Gauge<T> _gauge;
+  private final DropwizardSettableGauge<T> _settableGauge;
 
-  public DropwizardGauge(Gauge<T> gauge) {
-    _gauge = gauge;
+  public DropwizardGauge(DropwizardSettableGauge<T> settableGauge) {
+    _settableGauge = settableGauge;
   }
 
   public DropwizardGauge(Function<Void, T> condition) {
-    this(() -> condition.apply(null));
+    this(new DropwizardSettableGauge<>(() -> condition.apply(null)));
   }
 
   @Override
   public Object getGauge() {
-    return _gauge;
+    return _settableGauge;
   }
 
   @Override
   public Object getMetric() {
-    return _gauge;
+    return _settableGauge;
   }
 
   @Override
   public T value() {
-    return _gauge.getValue();
+    return _settableGauge.getValue();
+  }
+
+  @Override
+  public void setValue(T value) {
+    _settableGauge.setValue(value);
+  }
+
+  @Override
+  public void setValueSupplier(Supplier<T> valueSupplier) {
+    _settableGauge.setValueSupplier(valueSupplier);
   }
 }
diff --git a/pinot-plugins/pinot-metrics/pinot-dropwizard/src/main/java/org/apache/pinot/plugin/metrics/dropwizard/DropwizardMetricsRegistry.java b/pinot-plugins/pinot-metrics/pinot-dropwizard/src/main/java/org/apache/pinot/plugin/metrics/dropwizard/DropwizardMetricsRegistry.java
index 712d4918e5..3f8f270e5c 100644
--- a/pinot-plugins/pinot-metrics/pinot-dropwizard/src/main/java/org/apache/pinot/plugin/metrics/dropwizard/DropwizardMetricsRegistry.java
+++ b/pinot-plugins/pinot-metrics/pinot-dropwizard/src/main/java/org/apache/pinot/plugin/metrics/dropwizard/DropwizardMetricsRegistry.java
@@ -18,7 +18,6 @@
  */
 package org.apache.pinot.plugin.metrics.dropwizard;
 
-import com.codahale.metrics.Gauge;
 import com.codahale.metrics.Metric;
 import com.codahale.metrics.MetricRegistry;
 import com.codahale.metrics.MetricRegistryListener;
@@ -51,8 +50,8 @@ public class DropwizardMetricsRegistry implements PinotMetricsRegistry {
   @Override
   public <T> PinotGauge<T> newGauge(PinotMetricName name, final PinotGauge<T> gauge) {
     final String metricName = name.getMetricName().toString();
-    return (gauge == null) ? new DropwizardGauge<T>((Gauge) _metricRegistry.gauge(metricName))
-        : new DropwizardGauge<T>(_metricRegistry.gauge(metricName, () -> (Gauge) gauge.getGauge()));
+    return (gauge == null) ? new DropwizardGauge<T>((DropwizardSettableGauge) _metricRegistry.gauge(metricName))
+        : new DropwizardGauge<T>(_metricRegistry.gauge(metricName, () -> (DropwizardSettableGauge) gauge.getGauge()));
   }
 
   @Override
diff --git a/pinot-plugins/pinot-metrics/pinot-dropwizard/src/main/java/org/apache/pinot/plugin/metrics/dropwizard/DropwizardGauge.java b/pinot-plugins/pinot-metrics/pinot-dropwizard/src/main/java/org/apache/pinot/plugin/metrics/dropwizard/DropwizardSettableGauge.java
similarity index 51%
copy from pinot-plugins/pinot-metrics/pinot-dropwizard/src/main/java/org/apache/pinot/plugin/metrics/dropwizard/DropwizardGauge.java
copy to pinot-plugins/pinot-metrics/pinot-dropwizard/src/main/java/org/apache/pinot/plugin/metrics/dropwizard/DropwizardSettableGauge.java
index 01d0b24dc6..7cc7a3aa58 100644
--- a/pinot-plugins/pinot-metrics/pinot-dropwizard/src/main/java/org/apache/pinot/plugin/metrics/dropwizard/DropwizardGauge.java
+++ b/pinot-plugins/pinot-metrics/pinot-dropwizard/src/main/java/org/apache/pinot/plugin/metrics/dropwizard/DropwizardSettableGauge.java
@@ -18,35 +18,39 @@
  */
 package org.apache.pinot.plugin.metrics.dropwizard;
 
-import com.codahale.metrics.Gauge;
-import java.util.function.Function;
-import org.apache.pinot.spi.metrics.PinotGauge;
+import com.codahale.metrics.SettableGauge;
+import java.util.function.Supplier;
+import org.apache.pinot.spi.metrics.SettableValue;
 
+/**
+ * DropwizardSettableGauge extends {@link SettableGauge} and implements {@link SettableValue}, allowing setting a value
+ * or a value supplier to provide the gauge value
+ *
+ * @param <T> the type of the metric's value
+ */
+public class DropwizardSettableGauge<T> implements SettableGauge<T>, SettableValue<T> {
+  private Supplier<T> _valueSupplier;
 
-public class DropwizardGauge<T> implements PinotGauge<T> {
-
-  private final Gauge<T> _gauge;
-
-  public DropwizardGauge(Gauge<T> gauge) {
-    _gauge = gauge;
+  public DropwizardSettableGauge(Supplier<T> valueSupplier) {
+    setValueSupplier(valueSupplier);
   }
 
-  public DropwizardGauge(Function<Void, T> condition) {
-    this(() -> condition.apply(null));
+  public DropwizardSettableGauge(T value) {
+    setValue(value);
   }
 
   @Override
-  public Object getGauge() {
-    return _gauge;
+  public void setValueSupplier(Supplier<T> valueSupplier) {
+    _valueSupplier = valueSupplier;
   }
 
   @Override
-  public Object getMetric() {
-    return _gauge;
+  public void setValue(T value) {
+    _valueSupplier = () -> value;
   }
 
   @Override
-  public T value() {
-    return _gauge.getValue();
+  public T getValue() {
+    return _valueSupplier.get();
   }
 }
diff --git a/pinot-plugins/pinot-metrics/pinot-dropwizard/src/main/java/org/apache/pinot/plugin/metrics/dropwizard/DropwizardGauge.java b/pinot-plugins/pinot-metrics/pinot-dropwizard/src/test/java/org/apache/pinot/plugin/metrics/dropwizard/DropwizardGaugeTest.java
similarity index 55%
copy from pinot-plugins/pinot-metrics/pinot-dropwizard/src/main/java/org/apache/pinot/plugin/metrics/dropwizard/DropwizardGauge.java
copy to pinot-plugins/pinot-metrics/pinot-dropwizard/src/test/java/org/apache/pinot/plugin/metrics/dropwizard/DropwizardGaugeTest.java
index 01d0b24dc6..6a92f4e1fd 100644
--- a/pinot-plugins/pinot-metrics/pinot-dropwizard/src/main/java/org/apache/pinot/plugin/metrics/dropwizard/DropwizardGauge.java
+++ b/pinot-plugins/pinot-metrics/pinot-dropwizard/src/test/java/org/apache/pinot/plugin/metrics/dropwizard/DropwizardGaugeTest.java
@@ -18,35 +18,23 @@
  */
 package org.apache.pinot.plugin.metrics.dropwizard;
 
-import com.codahale.metrics.Gauge;
-import java.util.function.Function;
-import org.apache.pinot.spi.metrics.PinotGauge;
+import org.testng.Assert;
+import org.testng.annotations.Test;
 
 
-public class DropwizardGauge<T> implements PinotGauge<T> {
+public class DropwizardGaugeTest {
+  @Test
+  public void testUpdateGaugeValue() {
+    DropwizardSettableGauge<Long> dropwizardSettableGauge = new DropwizardSettableGauge<>(1L);
+    DropwizardGauge<Long> dropwizardGauge = new DropwizardGauge<>(dropwizardSettableGauge);
 
-  private final Gauge<T> _gauge;
+    Assert.assertEquals(dropwizardGauge.getGauge(), dropwizardSettableGauge);
+    Assert.assertEquals(dropwizardGauge.value(), Long.valueOf(1L));
 
-  public DropwizardGauge(Gauge<T> gauge) {
-    _gauge = gauge;
-  }
-
-  public DropwizardGauge(Function<Void, T> condition) {
-    this(() -> condition.apply(null));
-  }
-
-  @Override
-  public Object getGauge() {
-    return _gauge;
-  }
-
-  @Override
-  public Object getMetric() {
-    return _gauge;
-  }
+    dropwizardGauge.setValue(2L);
+    Assert.assertEquals(dropwizardGauge.value(), Long.valueOf(2L));
 
-  @Override
-  public T value() {
-    return _gauge.getValue();
+    dropwizardGauge.setValueSupplier(() -> 3L);
+    Assert.assertEquals(dropwizardGauge.value(), Long.valueOf(3L));
   }
 }
diff --git a/pinot-plugins/pinot-metrics/pinot-dropwizard/src/main/java/org/apache/pinot/plugin/metrics/dropwizard/DropwizardGauge.java b/pinot-plugins/pinot-metrics/pinot-dropwizard/src/test/java/org/apache/pinot/plugin/metrics/dropwizard/DropwizardMetricsRegistryTest.java
similarity index 54%
copy from pinot-plugins/pinot-metrics/pinot-dropwizard/src/main/java/org/apache/pinot/plugin/metrics/dropwizard/DropwizardGauge.java
copy to pinot-plugins/pinot-metrics/pinot-dropwizard/src/test/java/org/apache/pinot/plugin/metrics/dropwizard/DropwizardMetricsRegistryTest.java
index 01d0b24dc6..4d4bdde658 100644
--- a/pinot-plugins/pinot-metrics/pinot-dropwizard/src/main/java/org/apache/pinot/plugin/metrics/dropwizard/DropwizardGauge.java
+++ b/pinot-plugins/pinot-metrics/pinot-dropwizard/src/test/java/org/apache/pinot/plugin/metrics/dropwizard/DropwizardMetricsRegistryTest.java
@@ -18,35 +18,20 @@
  */
 package org.apache.pinot.plugin.metrics.dropwizard;
 
-import com.codahale.metrics.Gauge;
-import java.util.function.Function;
 import org.apache.pinot.spi.metrics.PinotGauge;
-
-
-public class DropwizardGauge<T> implements PinotGauge<T> {
-
-  private final Gauge<T> _gauge;
-
-  public DropwizardGauge(Gauge<T> gauge) {
-    _gauge = gauge;
-  }
-
-  public DropwizardGauge(Function<Void, T> condition) {
-    this(() -> condition.apply(null));
-  }
-
-  @Override
-  public Object getGauge() {
-    return _gauge;
-  }
-
-  @Override
-  public Object getMetric() {
-    return _gauge;
-  }
-
-  @Override
-  public T value() {
-    return _gauge.getValue();
+import org.testng.Assert;
+import org.testng.annotations.Test;
+
+
+public class DropwizardMetricsRegistryTest {
+  @Test
+  public void testNewGauge() {
+    DropwizardMetricsRegistry dropwizardMetricsRegistry = new DropwizardMetricsRegistry();
+    DropwizardSettableGauge<Long> dropwizardSettableGauge = new DropwizardSettableGauge<>(1L);
+    DropwizardGauge<Long> dropwizardGauge = new DropwizardGauge<>(dropwizardSettableGauge);
+    PinotGauge<Long> pinotGauge = dropwizardMetricsRegistry.newGauge(new DropwizardMetricName("test"), dropwizardGauge);
+    Assert.assertEquals(pinotGauge.value(), Long.valueOf(1L));
+    pinotGauge = dropwizardMetricsRegistry.newGauge(new DropwizardMetricName("test"), null);
+    Assert.assertEquals(pinotGauge.value(), Long.valueOf(1L));
   }
 }
diff --git a/pinot-plugins/pinot-metrics/pinot-dropwizard/src/test/java/org/apache/pinot/plugin/metrics/dropwizard/DropwizardSettableGaugeTest.java b/pinot-plugins/pinot-metrics/pinot-dropwizard/src/test/java/org/apache/pinot/plugin/metrics/dropwizard/DropwizardSettableGaugeTest.java
new file mode 100644
index 0000000000..7d9530db08
--- /dev/null
+++ b/pinot-plugins/pinot-metrics/pinot-dropwizard/src/test/java/org/apache/pinot/plugin/metrics/dropwizard/DropwizardSettableGaugeTest.java
@@ -0,0 +1,45 @@
+/**
+ * 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.pinot.plugin.metrics.dropwizard;
+
+import org.testng.Assert;
+import org.testng.annotations.Test;
+
+
+public class DropwizardSettableGaugeTest {
+  @Test
+  public void testCreateDropwizardSettableGaugeImplWithValue() {
+    DropwizardSettableGauge<Long> dropwizardSettableGauge = new DropwizardSettableGauge<>(1L);
+    setAngCheck(dropwizardSettableGauge);
+  }
+
+  @Test
+  public void testCreateDropwizardSettableGaugeImplWithValueSupplier() {
+    DropwizardSettableGauge<Long> dropwizardSettableGauge = new DropwizardSettableGauge<>(() -> 1L);
+    setAngCheck(dropwizardSettableGauge);
+  }
+
+  private void setAngCheck(DropwizardSettableGauge<Long> dropwizardSettableGauge) {
+    Assert.assertEquals(dropwizardSettableGauge.getValue(), Long.valueOf(1L));
+    dropwizardSettableGauge.setValue(2L);
+    Assert.assertEquals(dropwizardSettableGauge.getValue(), Long.valueOf(2L));
+    dropwizardSettableGauge.setValueSupplier(() -> 3L);
+    Assert.assertEquals(dropwizardSettableGauge.getValue(), Long.valueOf(3L));
+  }
+}
diff --git a/pinot-plugins/pinot-metrics/pinot-yammer/src/main/java/org/apache/pinot/plugin/metrics/yammer/YammerGauge.java b/pinot-plugins/pinot-metrics/pinot-yammer/src/main/java/org/apache/pinot/plugin/metrics/yammer/YammerGauge.java
index c08452b461..745fb697af 100644
--- a/pinot-plugins/pinot-metrics/pinot-yammer/src/main/java/org/apache/pinot/plugin/metrics/yammer/YammerGauge.java
+++ b/pinot-plugins/pinot-metrics/pinot-yammer/src/main/java/org/apache/pinot/plugin/metrics/yammer/YammerGauge.java
@@ -18,40 +18,45 @@
  */
 package org.apache.pinot.plugin.metrics.yammer;
 
-import com.yammer.metrics.core.Gauge;
 import java.util.function.Function;
+import java.util.function.Supplier;
 import org.apache.pinot.spi.metrics.PinotGauge;
 
 
 public class YammerGauge<T> implements PinotGauge<T> {
 
-  private final Gauge<T> _gauge;
+  private final YammerSettableGauge<T> _settableGauge;
 
-  public YammerGauge(Gauge<T> gauge) {
-    _gauge = gauge;
+  public YammerGauge(YammerSettableGauge<T> settableGauge) {
+    _settableGauge = settableGauge;
   }
 
   public YammerGauge(Function<Void, T> condition) {
-    this(new Gauge<T>() {
-      @Override
-      public T value() {
-        return condition.apply(null);
-      }
-    });
+    this(new YammerSettableGauge<>(() -> condition.apply(null)));
   }
 
   @Override
   public Object getGauge() {
-    return _gauge;
+    return _settableGauge;
   }
 
   @Override
   public Object getMetric() {
-    return _gauge;
+    return _settableGauge;
   }
 
   @Override
   public T value() {
-    return _gauge.value();
+    return _settableGauge.value();
+  }
+
+  @Override
+  public void setValue(T value) {
+    _settableGauge.setValue(value);
+  }
+
+  @Override
+  public void setValueSupplier(Supplier<T> valueSupplier) {
+    _settableGauge.setValueSupplier(valueSupplier);
   }
 }
diff --git a/pinot-plugins/pinot-metrics/pinot-yammer/src/main/java/org/apache/pinot/plugin/metrics/yammer/YammerMetricsRegistry.java b/pinot-plugins/pinot-metrics/pinot-yammer/src/main/java/org/apache/pinot/plugin/metrics/yammer/YammerMetricsRegistry.java
index 28a4ef9d2b..e155ecd05b 100644
--- a/pinot-plugins/pinot-metrics/pinot-yammer/src/main/java/org/apache/pinot/plugin/metrics/yammer/YammerMetricsRegistry.java
+++ b/pinot-plugins/pinot-metrics/pinot-yammer/src/main/java/org/apache/pinot/plugin/metrics/yammer/YammerMetricsRegistry.java
@@ -56,7 +56,7 @@ public class YammerMetricsRegistry implements PinotMetricsRegistry {
 
   @Override
   public <T> PinotGauge<T> newGauge(PinotMetricName name, PinotGauge<T> gauge) {
-    return new YammerGauge<T>(
+    return new YammerGauge<T>((YammerSettableGauge<T>)
         _metricsRegistry.newGauge((MetricName) name.getMetricName(), (Gauge<T>) gauge.getGauge()));
   }
 
diff --git a/pinot-plugins/pinot-metrics/pinot-yammer/src/main/java/org/apache/pinot/plugin/metrics/yammer/YammerGauge.java b/pinot-plugins/pinot-metrics/pinot-yammer/src/main/java/org/apache/pinot/plugin/metrics/yammer/YammerSettableGauge.java
similarity index 55%
copy from pinot-plugins/pinot-metrics/pinot-yammer/src/main/java/org/apache/pinot/plugin/metrics/yammer/YammerGauge.java
copy to pinot-plugins/pinot-metrics/pinot-yammer/src/main/java/org/apache/pinot/plugin/metrics/yammer/YammerSettableGauge.java
index c08452b461..0fa10b39c7 100644
--- a/pinot-plugins/pinot-metrics/pinot-yammer/src/main/java/org/apache/pinot/plugin/metrics/yammer/YammerGauge.java
+++ b/pinot-plugins/pinot-metrics/pinot-yammer/src/main/java/org/apache/pinot/plugin/metrics/yammer/YammerSettableGauge.java
@@ -19,39 +19,38 @@
 package org.apache.pinot.plugin.metrics.yammer;
 
 import com.yammer.metrics.core.Gauge;
-import java.util.function.Function;
-import org.apache.pinot.spi.metrics.PinotGauge;
+import java.util.function.Supplier;
+import org.apache.pinot.spi.metrics.SettableValue;
 
+/**
+ * YammerSettableGauge extends {@link Gauge} and implements {@link SettableValue}, allowing setting a value or a value
+ * supplier to provide the gauge value.
+ *
+ * @param <T> the type of the metric's value
+ */
+public class YammerSettableGauge<T> extends Gauge<T> implements SettableValue<T> {
+  private Supplier<T> _valueSupplier;
 
-public class YammerGauge<T> implements PinotGauge<T> {
-
-  private final Gauge<T> _gauge;
-
-  public YammerGauge(Gauge<T> gauge) {
-    _gauge = gauge;
+  public YammerSettableGauge(Supplier<T> valueSupplier) {
+    _valueSupplier = valueSupplier;
   }
 
-  public YammerGauge(Function<Void, T> condition) {
-    this(new Gauge<T>() {
-      @Override
-      public T value() {
-        return condition.apply(null);
-      }
-    });
+  public YammerSettableGauge(T value) {
+    setValue(value);
   }
 
   @Override
-  public Object getGauge() {
-    return _gauge;
+  public void setValue(T value) {
+    _valueSupplier = () -> value;
   }
 
   @Override
-  public Object getMetric() {
-    return _gauge;
+  public void setValueSupplier(Supplier<T> valueSupplier) {
+    _valueSupplier = valueSupplier;
   }
 
   @Override
   public T value() {
-    return _gauge.value();
+    return _valueSupplier.get();
   }
 }
diff --git a/pinot-plugins/pinot-metrics/pinot-yammer/src/main/java/org/apache/pinot/plugin/metrics/yammer/YammerGauge.java b/pinot-plugins/pinot-metrics/pinot-yammer/src/test/java/org/apache/pinot/plugin/metrics/yammer/YammerGaugeTest.java
similarity index 57%
copy from pinot-plugins/pinot-metrics/pinot-yammer/src/main/java/org/apache/pinot/plugin/metrics/yammer/YammerGauge.java
copy to pinot-plugins/pinot-metrics/pinot-yammer/src/test/java/org/apache/pinot/plugin/metrics/yammer/YammerGaugeTest.java
index c08452b461..253d6f5d95 100644
--- a/pinot-plugins/pinot-metrics/pinot-yammer/src/main/java/org/apache/pinot/plugin/metrics/yammer/YammerGauge.java
+++ b/pinot-plugins/pinot-metrics/pinot-yammer/src/test/java/org/apache/pinot/plugin/metrics/yammer/YammerGaugeTest.java
@@ -18,40 +18,23 @@
  */
 package org.apache.pinot.plugin.metrics.yammer;
 
-import com.yammer.metrics.core.Gauge;
-import java.util.function.Function;
-import org.apache.pinot.spi.metrics.PinotGauge;
+import org.testng.Assert;
+import org.testng.annotations.Test;
 
 
-public class YammerGauge<T> implements PinotGauge<T> {
+public class YammerGaugeTest {
+  @Test
+  public void testUpdateGaugeValue() {
+    YammerSettableGauge<Long> yammerSettableGauge = new YammerSettableGauge<>(1L);
+    YammerGauge<Long> yammerGauge = new YammerGauge<>(yammerSettableGauge);
 
-  private final Gauge<T> _gauge;
+    Assert.assertEquals(yammerGauge.getGauge(), yammerSettableGauge);
+    Assert.assertEquals(yammerGauge.value(), Long.valueOf(1L));
 
-  public YammerGauge(Gauge<T> gauge) {
-    _gauge = gauge;
-  }
-
-  public YammerGauge(Function<Void, T> condition) {
-    this(new Gauge<T>() {
-      @Override
-      public T value() {
-        return condition.apply(null);
-      }
-    });
-  }
-
-  @Override
-  public Object getGauge() {
-    return _gauge;
-  }
-
-  @Override
-  public Object getMetric() {
-    return _gauge;
-  }
+    yammerGauge.setValue(2L);
+    Assert.assertEquals(yammerGauge.value(), Long.valueOf(2L));
 
-  @Override
-  public T value() {
-    return _gauge.value();
+    yammerGauge.setValueSupplier(() -> 3L);
+    Assert.assertEquals(yammerGauge.value(), Long.valueOf(3L));
   }
 }
diff --git a/pinot-plugins/pinot-metrics/pinot-yammer/src/main/java/org/apache/pinot/plugin/metrics/yammer/YammerGauge.java b/pinot-plugins/pinot-metrics/pinot-yammer/src/test/java/org/apache/pinot/plugin/metrics/yammer/YammerMetricsRegistryTest.java
similarity index 57%
copy from pinot-plugins/pinot-metrics/pinot-yammer/src/main/java/org/apache/pinot/plugin/metrics/yammer/YammerGauge.java
copy to pinot-plugins/pinot-metrics/pinot-yammer/src/test/java/org/apache/pinot/plugin/metrics/yammer/YammerMetricsRegistryTest.java
index c08452b461..bc53d8ac18 100644
--- a/pinot-plugins/pinot-metrics/pinot-yammer/src/main/java/org/apache/pinot/plugin/metrics/yammer/YammerGauge.java
+++ b/pinot-plugins/pinot-metrics/pinot-yammer/src/test/java/org/apache/pinot/plugin/metrics/yammer/YammerMetricsRegistryTest.java
@@ -18,40 +18,20 @@
  */
 package org.apache.pinot.plugin.metrics.yammer;
 
-import com.yammer.metrics.core.Gauge;
-import java.util.function.Function;
+import com.yammer.metrics.core.MetricName;
 import org.apache.pinot.spi.metrics.PinotGauge;
-
-
-public class YammerGauge<T> implements PinotGauge<T> {
-
-  private final Gauge<T> _gauge;
-
-  public YammerGauge(Gauge<T> gauge) {
-    _gauge = gauge;
-  }
-
-  public YammerGauge(Function<Void, T> condition) {
-    this(new Gauge<T>() {
-      @Override
-      public T value() {
-        return condition.apply(null);
-      }
-    });
-  }
-
-  @Override
-  public Object getGauge() {
-    return _gauge;
-  }
-
-  @Override
-  public Object getMetric() {
-    return _gauge;
-  }
-
-  @Override
-  public T value() {
-    return _gauge.value();
+import org.testng.Assert;
+import org.testng.annotations.Test;
+
+
+public class YammerMetricsRegistryTest {
+  @Test
+  public void testNewGaugeNoError() {
+    YammerMetricsRegistry yammerMetricsRegistry = new YammerMetricsRegistry();
+    YammerSettableGauge<Long> yammerSettableGauge = new YammerSettableGauge<>(1L);
+    YammerGauge<Long> yammerGauge = new YammerGauge<>(yammerSettableGauge);
+    MetricName metricName = new MetricName(this.getClass(), "test");
+    PinotGauge<Long> pinotGauge = yammerMetricsRegistry.newGauge(new YammerMetricName(metricName), yammerGauge);
+    Assert.assertEquals(pinotGauge.value(), Long.valueOf(1L));
   }
 }
diff --git a/pinot-plugins/pinot-metrics/pinot-yammer/src/test/java/org/apache/pinot/plugin/metrics/yammer/YammerSettableGaugeTest.java b/pinot-plugins/pinot-metrics/pinot-yammer/src/test/java/org/apache/pinot/plugin/metrics/yammer/YammerSettableGaugeTest.java
new file mode 100644
index 0000000000..1549420d6e
--- /dev/null
+++ b/pinot-plugins/pinot-metrics/pinot-yammer/src/test/java/org/apache/pinot/plugin/metrics/yammer/YammerSettableGaugeTest.java
@@ -0,0 +1,45 @@
+/**
+ * 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.pinot.plugin.metrics.yammer;
+
+import org.testng.Assert;
+import org.testng.annotations.Test;
+
+
+public class YammerSettableGaugeTest {
+  @Test
+  public void testCreateDropwizardSettableGaugeImplWithValue() {
+    YammerSettableGauge<Long> yammerSettableGauge = new YammerSettableGauge<>(1L);
+    setAngCheck(yammerSettableGauge);
+  }
+
+  @Test
+  public void testCreateDropwizardSettableGaugeImplWithValueSupplier() {
+    YammerSettableGauge<Long> yammerSettableGauge = new YammerSettableGauge<>(() -> 1L);
+    setAngCheck(yammerSettableGauge);
+  }
+
+  private void setAngCheck(YammerSettableGauge<Long> yammerSettableGauge) {
+    Assert.assertEquals(yammerSettableGauge.value(), Long.valueOf(1L));
+    yammerSettableGauge.setValue(2L);
+    Assert.assertEquals(yammerSettableGauge.value(), Long.valueOf(2L));
+    yammerSettableGauge.setValueSupplier(() -> 3L);
+    Assert.assertEquals(yammerSettableGauge.value(), Long.valueOf(3L));
+  }
+}
diff --git a/pinot-spi/src/main/java/org/apache/pinot/spi/metrics/PinotGauge.java b/pinot-spi/src/main/java/org/apache/pinot/spi/metrics/PinotGauge.java
index 3f0470ec55..8e511c5a9c 100644
--- a/pinot-spi/src/main/java/org/apache/pinot/spi/metrics/PinotGauge.java
+++ b/pinot-spi/src/main/java/org/apache/pinot/spi/metrics/PinotGauge.java
@@ -18,6 +18,8 @@
  */
 package org.apache.pinot.spi.metrics;
 
+import java.util.function.Supplier;
+
 /**
  * A gauge metric is an instantaneous reading of a particular value. To instrument a queue's depth,
  * for example:<br>
@@ -32,14 +34,24 @@ package org.apache.pinot.spi.metrics;
  *
  * @param <T> the type of the metric's value
  */
-public interface PinotGauge<T> extends PinotMetric {
+public interface PinotGauge<T> extends PinotMetric, SettableValue<T> {
 
   Object getGauge();
 
+  @Override
+  default void setValue(T value) {
+    throw new RuntimeException("setValue is not implemented");
+  }
+
+  @Override
+  default void setValueSupplier(Supplier<T> valueSupplier) {
+    throw new RuntimeException("setValueSupplier is not implemented");
+  }
+
   /**
    * Returns the metric's current value.
    *
    * @return the metric's current value
    */
-  public abstract T value();
+  T value();
 }
diff --git a/pinot-spi/src/main/java/org/apache/pinot/spi/metrics/PinotGauge.java b/pinot-spi/src/main/java/org/apache/pinot/spi/metrics/SettableValue.java
similarity index 58%
copy from pinot-spi/src/main/java/org/apache/pinot/spi/metrics/PinotGauge.java
copy to pinot-spi/src/main/java/org/apache/pinot/spi/metrics/SettableValue.java
index 3f0470ec55..05212a6bfa 100644
--- a/pinot-spi/src/main/java/org/apache/pinot/spi/metrics/PinotGauge.java
+++ b/pinot-spi/src/main/java/org/apache/pinot/spi/metrics/SettableValue.java
@@ -18,28 +18,23 @@
  */
 package org.apache.pinot.spi.metrics;
 
+import java.util.function.Supplier;
+
+
 /**
- * A gauge metric is an instantaneous reading of a particular value. To instrument a queue's depth,
- * for example:<br>
- * <pre><code>
- * final Queue&lt;String&gt; queue = new ConcurrentLinkedQueue&lt;String&gt;();
- * final Gauge&lt;Integer&gt; queueDepth = new Gauge&lt;Integer&gt;() {
- *     public Integer value() {
- *         return queue.size();
- *     }
- * };
- * </code></pre>
- *
- * @param <T> the type of the metric's value
+ * SettableValue allows the value to be set to a value or provided by a value supplier.
+ * @param <T> the type of the value to be set.
  */
-public interface PinotGauge<T> extends PinotMetric {
-
-  Object getGauge();
+public interface SettableValue<T> {
+  /**
+   * Sets the value.
+   * @param value the value to set.
+   */
+  void setValue(T value);
 
   /**
-   * Returns the metric's current value.
-   *
-   * @return the metric's current value
+   * Sets the value supplier.
+   * @param valueSupplier the value supplier to set.
    */
-  public abstract T value();
+  void setValueSupplier(Supplier<T> valueSupplier);
 }


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@pinot.apache.org
For additional commands, e-mail: commits-help@pinot.apache.org