You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@pinot.apache.org by ja...@apache.org on 2023/02/23 19:34:45 UTC

[pinot] branch master updated: Add Theta Sketch Aggregation for StarTree Indexes (#10288)

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

jackie 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 9b49bca554 Add Theta Sketch Aggregation for StarTree Indexes (#10288)
9b49bca554 is described below

commit 9b49bca55476b431171f4face5f28e6aeef9fe7f
Author: Andi Miller <an...@andimiller.net>
AuthorDate: Thu Feb 23 19:34:38 2023 +0000

    Add Theta Sketch Aggregation for StarTree Indexes (#10288)
---
 .../v2/DistinctCountThetaSketchStarTreeV2Test.java |  51 ++++++
 .../DistinctCountThetaSketchValueAggregator.java   | 178 +++++++++++++++++++++
 .../local/aggregator/ValueAggregatorFactory.java   |   6 +
 ...istinctCountThetaSketchValueAggregatorTest.java | 165 +++++++++++++++++++
 .../apache/pinot/spi/utils/CommonConstants.java    |  20 ++-
 5 files changed, 412 insertions(+), 8 deletions(-)

diff --git a/pinot-core/src/test/java/org/apache/pinot/core/startree/v2/DistinctCountThetaSketchStarTreeV2Test.java b/pinot-core/src/test/java/org/apache/pinot/core/startree/v2/DistinctCountThetaSketchStarTreeV2Test.java
new file mode 100644
index 0000000000..4e924c9d0c
--- /dev/null
+++ b/pinot-core/src/test/java/org/apache/pinot/core/startree/v2/DistinctCountThetaSketchStarTreeV2Test.java
@@ -0,0 +1,51 @@
+/**
+ * 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.core.startree.v2;
+
+import java.util.Random;
+import org.apache.datasketches.theta.Sketch;
+import org.apache.pinot.segment.local.aggregator.DistinctCountThetaSketchValueAggregator;
+import org.apache.pinot.segment.local.aggregator.ValueAggregator;
+import org.apache.pinot.spi.data.FieldSpec.DataType;
+
+import static org.testng.Assert.assertEquals;
+
+
+public class DistinctCountThetaSketchStarTreeV2Test extends BaseStarTreeV2Test<Object, Sketch> {
+
+  @Override
+  ValueAggregator<Object, Sketch> getValueAggregator() {
+    return new DistinctCountThetaSketchValueAggregator();
+  }
+
+  @Override
+  DataType getRawValueType() {
+    return DataType.INT;
+  }
+
+  @Override
+  Object getRandomRawValue(Random random) {
+    return random.nextInt(100);
+  }
+
+  @Override
+  void assertAggregatedValue(Sketch starTreeResult, Sketch nonStarTreeResult) {
+    assertEquals(starTreeResult.getEstimate(), nonStarTreeResult.getEstimate());
+  }
+}
diff --git a/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/aggregator/DistinctCountThetaSketchValueAggregator.java b/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/aggregator/DistinctCountThetaSketchValueAggregator.java
new file mode 100644
index 0000000000..6acba3ed05
--- /dev/null
+++ b/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/aggregator/DistinctCountThetaSketchValueAggregator.java
@@ -0,0 +1,178 @@
+/**
+ * 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.segment.local.aggregator;
+
+import java.util.Arrays;
+import java.util.stream.StreamSupport;
+import org.apache.datasketches.theta.Sketch;
+import org.apache.datasketches.theta.Sketches;
+import org.apache.datasketches.theta.Union;
+import org.apache.datasketches.theta.UpdateSketch;
+import org.apache.pinot.segment.local.utils.CustomSerDeUtils;
+import org.apache.pinot.segment.spi.AggregationFunctionType;
+import org.apache.pinot.spi.data.FieldSpec.DataType;
+import org.apache.pinot.spi.utils.CommonConstants;
+
+
+public class DistinctCountThetaSketchValueAggregator implements ValueAggregator<Object, Sketch> {
+  public static final DataType AGGREGATED_VALUE_TYPE = DataType.BYTES;
+
+  private final Union _union;
+
+  // This changes a lot similar to the Bitmap aggregator
+  private int _maxByteSize;
+
+  public DistinctCountThetaSketchValueAggregator() {
+    // TODO: Handle configurable nominal entries for StarTreeBuilder
+    _union = Union.builder().setNominalEntries(CommonConstants.Helix.DEFAULT_THETA_SKETCH_NOMINAL_ENTRIES).buildUnion();
+  }
+
+  @Override
+  public AggregationFunctionType getAggregationType() {
+    return AggregationFunctionType.DISTINCTCOUNTTHETASKETCH;
+  }
+
+  @Override
+  public DataType getAggregatedValueType() {
+    return AGGREGATED_VALUE_TYPE;
+  }
+
+  // Utility method to create a theta sketch with one item in it
+  private Sketch singleItemSketch(Object rawValue) {
+    // TODO: Handle configurable nominal entries for StarTreeBuilder
+    UpdateSketch sketch =
+        Sketches.updateSketchBuilder().setNominalEntries(CommonConstants.Helix.DEFAULT_THETA_SKETCH_NOMINAL_ENTRIES)
+            .build();
+    if (rawValue instanceof String) {
+      sketch.update((String) rawValue);
+    } else if (rawValue instanceof Integer) {
+      sketch.update((Integer) rawValue);
+    } else if (rawValue instanceof Long) {
+      sketch.update((Long) rawValue);
+    } else if (rawValue instanceof Double) {
+      sketch.update((Double) rawValue);
+    } else if (rawValue instanceof Float) {
+      sketch.update((Float) rawValue);
+    } else if (rawValue instanceof Object[]) {
+      addObjectsToSketch((Object[]) rawValue, sketch);
+    } else {
+      throw new IllegalStateException(
+          "Unsupported data type for Theta Sketch aggregation: " + rawValue.getClass().getName());
+    }
+    return sketch.compact();
+  }
+
+  private void addObjectsToSketch(Object[] rawValues, UpdateSketch updateSketch) {
+    if (rawValues instanceof String[]) {
+      for (String s : (String[]) rawValues) {
+        updateSketch.update(s);
+      }
+    } else if (rawValues instanceof Integer[]) {
+      for (Integer i : (Integer[]) rawValues) {
+        updateSketch.update(i);
+      }
+    } else if (rawValues instanceof Long[]) {
+      for (Long l : (Long[]) rawValues) {
+        updateSketch.update(l);
+      }
+    } else if (rawValues instanceof Double[]) {
+      for (Double d : (Double[]) rawValues) {
+        updateSketch.update(d);
+      }
+    } else if (rawValues instanceof Float[]) {
+      for (Float f : (Float[]) rawValues) {
+        updateSketch.update(f);
+      }
+    } else {
+      throw new IllegalStateException(
+          "Unsupported data type for Theta Sketch aggregation: " + rawValues.getClass().getName());
+    }
+  }
+
+  // Utility method to merge two sketches
+  private Sketch union(Sketch left, Sketch right) {
+    return _union.union(left, right);
+  }
+
+  // Utility method to make an empty sketch
+  private Sketch empty() {
+    // TODO: Handle configurable nominal entries for StarTreeBuilder
+    return Sketches.updateSketchBuilder().setNominalEntries(CommonConstants.Helix.DEFAULT_THETA_SKETCH_NOMINAL_ENTRIES)
+        .build().compact();
+  }
+
+  @Override
+  public Sketch getInitialAggregatedValue(Object rawValue) {
+    Sketch initialValue;
+    if (rawValue instanceof byte[]) { // Serialized Sketch
+      byte[] bytes = (byte[]) rawValue;
+      initialValue = deserializeAggregatedValue(bytes);
+      _maxByteSize = Math.max(_maxByteSize, bytes.length);
+    } else if (rawValue instanceof byte[][]) { // Multiple Serialized Sketches
+      byte[][] serializedSketches = (byte[][]) rawValue;
+      initialValue = StreamSupport.stream(Arrays.stream(serializedSketches).spliterator(), false)
+          .map(this::deserializeAggregatedValue).reduce(this::union).orElseGet(this::empty);
+      _maxByteSize = Math.max(_maxByteSize, initialValue.getCurrentBytes());
+    } else {
+      initialValue = singleItemSketch(rawValue);
+      _maxByteSize = Math.max(_maxByteSize, initialValue.getCurrentBytes());
+    }
+    return initialValue;
+  }
+
+  @Override
+  public Sketch applyRawValue(Sketch value, Object rawValue) {
+    Sketch right;
+    if (rawValue instanceof byte[]) {
+      right = deserializeAggregatedValue((byte[]) rawValue);
+    } else {
+      right = singleItemSketch(rawValue);
+    }
+    Sketch result = union(value, right).compact();
+    _maxByteSize = Math.max(_maxByteSize, result.getCurrentBytes());
+    return result;
+  }
+
+  @Override
+  public Sketch applyAggregatedValue(Sketch value, Sketch aggregatedValue) {
+    Sketch result = union(value, aggregatedValue);
+    _maxByteSize = Math.max(_maxByteSize, result.getCurrentBytes());
+    return result;
+  }
+
+  @Override
+  public Sketch cloneAggregatedValue(Sketch value) {
+    return deserializeAggregatedValue(serializeAggregatedValue(value));
+  }
+
+  @Override
+  public int getMaxAggregatedValueByteSize() {
+    return _maxByteSize;
+  }
+
+  @Override
+  public byte[] serializeAggregatedValue(Sketch value) {
+    return CustomSerDeUtils.DATA_SKETCH_SER_DE.serialize(value);
+  }
+
+  @Override
+  public Sketch deserializeAggregatedValue(byte[] bytes) {
+    return CustomSerDeUtils.DATA_SKETCH_SER_DE.deserialize(bytes);
+  }
+}
diff --git a/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/aggregator/ValueAggregatorFactory.java b/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/aggregator/ValueAggregatorFactory.java
index 092a285804..aa4bdb410b 100644
--- a/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/aggregator/ValueAggregatorFactory.java
+++ b/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/aggregator/ValueAggregatorFactory.java
@@ -63,6 +63,9 @@ public class ValueAggregatorFactory {
       case PERCENTILETDIGEST:
       case PERCENTILERAWTDIGEST:
         return new PercentileTDigestValueAggregator();
+      case DISTINCTCOUNTTHETASKETCH:
+      case DISTINCTCOUNTRAWTHETASKETCH:
+        return new DistinctCountThetaSketchValueAggregator();
       default:
         throw new IllegalStateException("Unsupported aggregation type: " + aggregationType);
     }
@@ -101,6 +104,9 @@ public class ValueAggregatorFactory {
       case PERCENTILETDIGEST:
       case PERCENTILERAWTDIGEST:
         return PercentileTDigestValueAggregator.AGGREGATED_VALUE_TYPE;
+      case DISTINCTCOUNTTHETASKETCH:
+      case DISTINCTCOUNTRAWTHETASKETCH:
+        return DistinctCountThetaSketchValueAggregator.AGGREGATED_VALUE_TYPE;
       default:
         throw new IllegalStateException("Unsupported aggregation type: " + aggregationType);
     }
diff --git a/pinot-segment-local/src/test/java/org/apache/pinot/segment/local/aggregator/DistinctCountThetaSketchValueAggregatorTest.java b/pinot-segment-local/src/test/java/org/apache/pinot/segment/local/aggregator/DistinctCountThetaSketchValueAggregatorTest.java
new file mode 100644
index 0000000000..822335cfb0
--- /dev/null
+++ b/pinot-segment-local/src/test/java/org/apache/pinot/segment/local/aggregator/DistinctCountThetaSketchValueAggregatorTest.java
@@ -0,0 +1,165 @@
+/**
+ * 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.segment.local.aggregator;
+
+import java.util.stream.IntStream;
+import org.apache.datasketches.theta.Sketch;
+import org.apache.datasketches.theta.Sketches;
+import org.apache.datasketches.theta.Union;
+import org.apache.datasketches.theta.UpdateSketch;
+import org.apache.pinot.spi.utils.CommonConstants;
+import org.testng.annotations.Test;
+
+import static org.testng.Assert.assertEquals;
+import static org.testng.Assert.assertThrows;
+
+
+public class DistinctCountThetaSketchValueAggregatorTest {
+
+  @Test
+  public void initialShouldCreateSingleItemSketch() {
+    DistinctCountThetaSketchValueAggregator agg = new DistinctCountThetaSketchValueAggregator();
+    assertEquals(agg.getInitialAggregatedValue("hello world").getEstimate(), 1.0);
+  }
+
+  @Test
+  public void initialShouldParseASketch() {
+    UpdateSketch input = Sketches.updateSketchBuilder().build();
+    IntStream.range(0, 1000).forEach(input::update);
+    Sketch result = input.compact();
+    DistinctCountThetaSketchValueAggregator agg = new DistinctCountThetaSketchValueAggregator();
+    byte[] bytes = agg.serializeAggregatedValue(result);
+    assertEquals(agg.getInitialAggregatedValue(bytes).getEstimate(), result.getEstimate());
+
+    // and should update the max size
+    assertEquals(agg.getMaxAggregatedValueByteSize(), result.getCurrentBytes());
+  }
+
+  @Test
+  public void initialShouldParseMultiValueSketches() {
+    UpdateSketch input1 = Sketches.updateSketchBuilder().build();
+    input1.update("hello");
+    UpdateSketch input2 = Sketches.updateSketchBuilder().build();
+    input2.update("world");
+    DistinctCountThetaSketchValueAggregator agg = new DistinctCountThetaSketchValueAggregator();
+    byte[][] bytes = {agg.serializeAggregatedValue(input1), agg.serializeAggregatedValue(input2)};
+    assertEquals(agg.getInitialAggregatedValue(bytes).getEstimate(), 2.0);
+  }
+
+  @Test
+  public void applyAggregatedValueShouldUnion() {
+    UpdateSketch input1 = Sketches.updateSketchBuilder().build();
+    IntStream.range(0, 1000).forEach(input1::update);
+    Sketch result1 = input1.compact();
+    UpdateSketch input2 = Sketches.updateSketchBuilder().build();
+    IntStream.range(0, 1000).forEach(input2::update);
+    Sketch result2 = input2.compact();
+    DistinctCountThetaSketchValueAggregator agg = new DistinctCountThetaSketchValueAggregator();
+    Sketch result = agg.applyAggregatedValue(result1, result2);
+    Union union =
+        Union.builder().setNominalEntries(CommonConstants.Helix.DEFAULT_THETA_SKETCH_NOMINAL_ENTRIES).buildUnion();
+
+    Sketch merged = union.union(result1, result2);
+
+    assertEquals(result.getEstimate(), merged.getEstimate());
+
+    // and should update the max size
+    assertEquals(agg.getMaxAggregatedValueByteSize(), merged.getCurrentBytes());
+  }
+
+  @Test
+  public void applyRawValueShouldUnion() {
+    UpdateSketch input1 = Sketches.updateSketchBuilder().build();
+    IntStream.range(0, 1000).forEach(input1::update);
+    Sketch result1 = input1.compact();
+    UpdateSketch input2 = Sketches.updateSketchBuilder().build();
+    IntStream.range(0, 1000).forEach(input2::update);
+    Sketch result2 = input2.compact();
+    DistinctCountThetaSketchValueAggregator agg = new DistinctCountThetaSketchValueAggregator();
+    byte[] result2bytes = agg.serializeAggregatedValue(result2);
+    Sketch result = agg.applyRawValue(result1, result2bytes);
+    Union union =
+        Union.builder().setNominalEntries(CommonConstants.Helix.DEFAULT_THETA_SKETCH_NOMINAL_ENTRIES).buildUnion();
+
+    Sketch merged = union.union(result1, result2);
+
+    assertEquals(result.getEstimate(), merged.getEstimate());
+
+    // and should update the max size
+    assertEquals(agg.getMaxAggregatedValueByteSize(), merged.getCurrentBytes());
+  }
+
+  @Test
+  public void applyRawValueShouldAdd() {
+    UpdateSketch input1 = Sketches.updateSketchBuilder().build();
+    input1.update("hello".hashCode());
+    Sketch result1 = input1.compact();
+    DistinctCountThetaSketchValueAggregator agg = new DistinctCountThetaSketchValueAggregator();
+    Sketch result = agg.applyRawValue(result1, "world");
+
+    assertEquals(result.getEstimate(), 2.0);
+
+    // and should update the max size
+    assertEquals(agg.getMaxAggregatedValueByteSize(), 32 // may change in future versions of datasketches
+    );
+  }
+
+  @Test
+  public void applyRawValueShouldSupportMultiValue() {
+    UpdateSketch input1 = Sketches.updateSketchBuilder().build();
+    input1.update("hello");
+    Sketch result1 = input1.compact();
+    DistinctCountThetaSketchValueAggregator agg = new DistinctCountThetaSketchValueAggregator();
+    String[] strings = {"hello", "world", "this", "is", "some", "strings"};
+    Sketch result = agg.applyRawValue(result1, (Object) strings);
+
+    assertEquals(result.getEstimate(), 6.0);
+
+    // and should update the max size
+    assertEquals(agg.getMaxAggregatedValueByteSize(), 64 // may change in future versions of datasketches
+    );
+  }
+
+  @Test
+  public void getInitialValueShouldSupportDifferentTypes() {
+    DistinctCountThetaSketchValueAggregator agg = new DistinctCountThetaSketchValueAggregator();
+    assertEquals(agg.getInitialAggregatedValue(12345).getEstimate(), 1.0);
+    assertEquals(agg.getInitialAggregatedValue(12345L).getEstimate(), 1.0);
+    assertEquals(agg.getInitialAggregatedValue(12.345f).getEstimate(), 1.0);
+    assertEquals(agg.getInitialAggregatedValue(12.345d).getEstimate(), 1.0);
+    assertThrows(() -> agg.getInitialAggregatedValue(new Object()));
+  }
+
+  @Test
+  public void getInitialValueShouldSupportMultiValueTypes() {
+    DistinctCountThetaSketchValueAggregator agg = new DistinctCountThetaSketchValueAggregator();
+    Integer[] ints = {12345};
+    assertEquals(agg.getInitialAggregatedValue(ints).getEstimate(), 1.0);
+    Long[] longs = {12345L};
+    assertEquals(agg.getInitialAggregatedValue(longs).getEstimate(), 1.0);
+    Float[] floats = {12.345f};
+    assertEquals(agg.getInitialAggregatedValue(floats).getEstimate(), 1.0);
+    Double[] doubles = {12.345d};
+    assertEquals(agg.getInitialAggregatedValue(doubles).getEstimate(), 1.0);
+    Object[] objects = {new Object()};
+    assertThrows(() -> agg.getInitialAggregatedValue(objects));
+    byte[][] zeroSketches = {};
+    assertEquals(agg.getInitialAggregatedValue(zeroSketches).getEstimate(), 0.0);
+  }
+}
diff --git a/pinot-spi/src/main/java/org/apache/pinot/spi/utils/CommonConstants.java b/pinot-spi/src/main/java/org/apache/pinot/spi/utils/CommonConstants.java
index 3a821c3b2b..2f0dedcb5b 100644
--- a/pinot-spi/src/main/java/org/apache/pinot/spi/utils/CommonConstants.java
+++ b/pinot-spi/src/main/java/org/apache/pinot/spi/utils/CommonConstants.java
@@ -46,6 +46,7 @@ public class CommonConstants {
       "org.apache.pinot.plugin.metrics.yammer.YammerMetricsFactory";
 
   public static final String SWAGGER_AUTHORIZATION_KEY = "oauth";
+
   /**
    * The state of the consumer for a given segment
    */
@@ -91,6 +92,10 @@ public class CommonConstants {
     public static final String DEFAULT_HYPERLOGLOG_LOG2M_KEY = "default.hyperloglog.log2m";
     public static final int DEFAULT_HYPERLOGLOG_LOG2M = 8;
 
+    // 2 to the power of 16, for tradeoffs see datasketches library documentation:
+    // https://datasketches.apache.org/docs/Theta/ThetaErrorTable.html
+    public static final int DEFAULT_THETA_SKETCH_NOMINAL_ENTRIES = 65536;
+
     // Whether to rewrite DistinctCount to DistinctCountBitmap
     public static final String ENABLE_DISTINCT_COUNT_BITMAP_OVERRIDE_KEY = "enable.distinct.count.bitmap.override";
 
@@ -262,8 +267,7 @@ public class CommonConstants {
 
     public static final String CONTROLLER_URL = "pinot.broker.controller.url";
 
-    public static final String CONFIG_OF_BROKER_REQUEST_CLIENT_IP_LOGGING =
-        "pinot.broker.request.client.ip.logging";
+    public static final String CONFIG_OF_BROKER_REQUEST_CLIENT_IP_LOGGING = "pinot.broker.request.client.ip.logging";
 
     // TODO: Support populating clientIp for GrpcRequestIdentity.
     public static final boolean DEFAULT_BROKER_REQUEST_CLIENT_IP_LOGGING = false;
@@ -271,10 +275,10 @@ public class CommonConstants {
     public static final String CONFIG_OF_LOGGER_ROOT_DIR = "pinot.broker.logger.root.dir";
     public static final String CONFIG_OF_SWAGGER_BROKER_ENABLED = "pinot.broker.swagger.enabled";
     public static final boolean DEFAULT_SWAGGER_BROKER_ENABLED = true;
-    public static final String CONFIG_OF_ENABLE_THREAD_CPU_TIME_MEASUREMENT
-        = "pinot.broker.instance.enableThreadCpuTimeMeasurement";
-    public static final String CONFIG_OF_ENABLE_THREAD_ALLOCATED_BYTES_MEASUREMENT
-        = "pinot.broker.instance.enableThreadAllocatedBytesMeasurement";
+    public static final String CONFIG_OF_ENABLE_THREAD_CPU_TIME_MEASUREMENT =
+        "pinot.broker.instance.enableThreadCpuTimeMeasurement";
+    public static final String CONFIG_OF_ENABLE_THREAD_ALLOCATED_BYTES_MEASUREMENT =
+        "pinot.broker.instance.enableThreadAllocatedBytesMeasurement";
     public static final boolean DEFAULT_ENABLE_THREAD_CPU_TIME_MEASUREMENT = false;
     public static final boolean DEFAULT_THREAD_ALLOCATED_BYTES_MEASUREMENT = false;
 
@@ -772,8 +776,8 @@ public class CommonConstants {
     public static final String CONFIG_OF_SLEEP_TIME_DENOMINATOR = "accounting.sleep.time.denominator";
     public static final int DEFAULT_SLEEP_TIME_DENOMINATOR = 3;
 
-    public static final String CONFIG_OF_MIN_MEMORY_FOOTPRINT_TO_KILL_RATIO
-        = "accounting.min.memory.footprint.to.kill.ratio";
+    public static final String CONFIG_OF_MIN_MEMORY_FOOTPRINT_TO_KILL_RATIO =
+        "accounting.min.memory.footprint.to.kill.ratio";
     public static final double DEFAULT_MEMORY_FOOTPRINT_TO_KILL_RATIO = 0.025;
 
     public static final String CONFIG_OF_GC_BACKOFF_COUNT = "accounting.gc.backoff.count";


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