Copilot commented on code in PR #17031:
URL: https://github.com/apache/pinot/pull/17031#discussion_r2438453305


##########
pinot-plugins/pinot-input-format/pinot-arrow/src/main/java/org/apache/pinot/plugin/inputformat/arrow/ArrowMessageDecoder.java:
##########
@@ -0,0 +1,122 @@
+/**
+ * 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.inputformat.arrow;
+
+
+import java.io.ByteArrayInputStream;
+import java.nio.channels.Channels;
+import java.nio.channels.ReadableByteChannel;
+import java.util.Arrays;
+import java.util.Map;
+import java.util.Set;
+import javax.annotation.Nullable;
+import org.apache.arrow.memory.RootAllocator;
+import org.apache.arrow.vector.VectorSchemaRoot;
+import org.apache.arrow.vector.ipc.ArrowStreamReader;
+import org.apache.pinot.spi.data.readers.GenericRow;
+import org.apache.pinot.spi.stream.StreamMessageDecoder;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * ArrowMessageDecoder is used to decode Apache Arrow IPC format messages into 
Pinot GenericRow.
+ * This decoder handles Arrow streaming format and converts Arrow data to 
Pinot's columnar format.
+ */
+public class ArrowMessageDecoder implements StreamMessageDecoder<byte[]> {
+  public static final String ARROW_SCHEMA_CONFIG = "arrow.schema.config";

Review Comment:
   The constant `ARROW_SCHEMA_CONFIG` is defined but never used in this class 
or the test files. Consider removing this unused constant or documenting its 
intended future use.
   ```suggestion
   
   ```



##########
pinot-plugins/pinot-input-format/pinot-arrow/src/main/java/org/apache/pinot/plugin/inputformat/arrow/ArrowMessageDecoder.java:
##########
@@ -0,0 +1,122 @@
+/**
+ * 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.inputformat.arrow;
+
+
+import java.io.ByteArrayInputStream;
+import java.nio.channels.Channels;
+import java.nio.channels.ReadableByteChannel;
+import java.util.Arrays;
+import java.util.Map;
+import java.util.Set;
+import javax.annotation.Nullable;
+import org.apache.arrow.memory.RootAllocator;
+import org.apache.arrow.vector.VectorSchemaRoot;
+import org.apache.arrow.vector.ipc.ArrowStreamReader;
+import org.apache.pinot.spi.data.readers.GenericRow;
+import org.apache.pinot.spi.stream.StreamMessageDecoder;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * ArrowMessageDecoder is used to decode Apache Arrow IPC format messages into 
Pinot GenericRow.
+ * This decoder handles Arrow streaming format and converts Arrow data to 
Pinot's columnar format.
+ */
+public class ArrowMessageDecoder implements StreamMessageDecoder<byte[]> {
+  public static final String ARROW_SCHEMA_CONFIG = "arrow.schema.config";
+  public static final String ARROW_ALLOCATOR_LIMIT = "arrow.allocator.limit";
+  public static final String DEFAULT_ALLOCATOR_LIMIT = "268435456"; // 256MB 
default
+
+  private static final Logger logger = 
LoggerFactory.getLogger(ArrowMessageDecoder.class);
+
+  private String _kafkaTopicName;
+  private Set<String> _fieldsToRead;
+  private RootAllocator _allocator;
+  private ArrowToGenericRowConverter _converter;
+
+  @Override
+  public void init(Map<String, String> props, Set<String> fieldsToRead, String 
topicName)
+      throws Exception {
+    _kafkaTopicName = topicName;
+    _fieldsToRead = fieldsToRead;
+
+    // Initialize Arrow allocator with configurable memory limit
+    long allocatorLimit =
+        Long.parseLong(props.getOrDefault(ARROW_ALLOCATOR_LIMIT, 
DEFAULT_ALLOCATOR_LIMIT));
+    _allocator = new RootAllocator(allocatorLimit);
+
+    // Initialize Arrow to GenericRow converter (processes all fields)
+    _converter = new ArrowToGenericRowConverter();
+
+    logger.info(
+        "Initialized ArrowMessageDecoder for topic: {} with allocator limit: 
{} bytes",
+        topicName,
+        allocatorLimit);
+  }
+
+  @Nullable
+  @Override
+  public GenericRow decode(byte[] payload, GenericRow destination) {
+    try (ByteArrayInputStream inputStream = new ByteArrayInputStream(payload);
+        ReadableByteChannel channel = Channels.newChannel(inputStream);
+        ArrowStreamReader reader = new ArrowStreamReader(channel, _allocator)) 
{
+
+      // Read the Arrow schema and data
+      VectorSchemaRoot root = reader.getVectorSchemaRoot();
+      if (!reader.loadNextBatch()) {
+        logger.warn("No data found in Arrow message for topic: {}", 
_kafkaTopicName);
+        return null;
+      }
+
+      // Convert Arrow data to GenericRow using converter
+      GenericRow row = _converter.convert(reader, root, destination);
+
+      return row;
+    } catch (Exception e) {
+      logger.error(
+          "Error decoding Arrow message for kafka topic {} : {}",
+          _kafkaTopicName,
+          Arrays.toString(payload),
+          e);

Review Comment:
   Logging the full payload with `Arrays.toString(payload)` could expose 
sensitive data in logs. Consider logging only payload metadata like length or a 
hash instead of the actual content.



##########
pinot-plugins/pinot-input-format/pinot-arrow/src/main/java/org/apache/pinot/plugin/inputformat/arrow/ArrowMessageDecoder.java:
##########
@@ -0,0 +1,122 @@
+/**
+ * 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.inputformat.arrow;
+
+
+import java.io.ByteArrayInputStream;
+import java.nio.channels.Channels;
+import java.nio.channels.ReadableByteChannel;
+import java.util.Arrays;
+import java.util.Map;
+import java.util.Set;
+import javax.annotation.Nullable;
+import org.apache.arrow.memory.RootAllocator;
+import org.apache.arrow.vector.VectorSchemaRoot;
+import org.apache.arrow.vector.ipc.ArrowStreamReader;
+import org.apache.pinot.spi.data.readers.GenericRow;
+import org.apache.pinot.spi.stream.StreamMessageDecoder;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * ArrowMessageDecoder is used to decode Apache Arrow IPC format messages into 
Pinot GenericRow.
+ * This decoder handles Arrow streaming format and converts Arrow data to 
Pinot's columnar format.
+ */
+public class ArrowMessageDecoder implements StreamMessageDecoder<byte[]> {
+  public static final String ARROW_SCHEMA_CONFIG = "arrow.schema.config";
+  public static final String ARROW_ALLOCATOR_LIMIT = "arrow.allocator.limit";
+  public static final String DEFAULT_ALLOCATOR_LIMIT = "268435456"; // 256MB 
default
+
+  private static final Logger logger = 
LoggerFactory.getLogger(ArrowMessageDecoder.class);
+
+  private String _kafkaTopicName;
+  private Set<String> _fieldsToRead;
+  private RootAllocator _allocator;
+  private ArrowToGenericRowConverter _converter;
+
+  @Override
+  public void init(Map<String, String> props, Set<String> fieldsToRead, String 
topicName)
+      throws Exception {
+    _kafkaTopicName = topicName;
+    _fieldsToRead = fieldsToRead;

Review Comment:
   The field `_fieldsToRead` is assigned in the `init` method but never used 
elsewhere in the class. Either implement field filtering logic based on this 
set or remove it if all fields should always be read.
   ```suggestion
     private RootAllocator _allocator;
     private ArrowToGenericRowConverter _converter;
   
     @Override
     public void init(Map<String, String> props, String topicName)
         throws Exception {
       _kafkaTopicName = topicName;
   ```



##########
pinot-plugins/pinot-input-format/pinot-arrow/src/main/java/org/apache/pinot/plugin/inputformat/arrow/ArrowToGenericRowConverter.java:
##########
@@ -0,0 +1,238 @@
+/**
+ * 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.inputformat.arrow;
+
+import java.sql.Timestamp;
+import java.time.LocalDateTime;
+import java.time.ZoneOffset;
+import java.util.ArrayList;
+import java.util.LinkedHashMap;
+import java.util.List;
+import java.util.Map;
+import javax.annotation.Nullable;
+import org.apache.arrow.vector.FieldVector;
+import org.apache.arrow.vector.ValueVector;
+import org.apache.arrow.vector.VectorSchemaRoot;
+import org.apache.arrow.vector.complex.MapVector;
+import org.apache.arrow.vector.complex.impl.UnionMapReader;
+import org.apache.arrow.vector.dictionary.DictionaryEncoder;
+import org.apache.arrow.vector.ipc.ArrowStreamReader;
+import org.apache.arrow.vector.util.Text;
+import org.apache.pinot.spi.data.readers.GenericRow;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Utility class for converting Apache Arrow VectorSchemaRoot to Pinot {@code 
GenericRow}. Processes
+ * all fields and handles multiple rows from Arrow batch.
+ */
+public class ArrowToGenericRowConverter {
+  private static final Logger logger = 
LoggerFactory.getLogger(ArrowToGenericRowConverter.class);
+
+  /** Default constructor that processes all fields from Arrow batch. */
+  public ArrowToGenericRowConverter() {
+    logger.debug("ArrowToGenericRowConverter created for processing all 
fields");
+  }
+
+  /**
+   * Converts an Arrow VectorSchemaRoot to a Pinot {@code GenericRow}. 
Processes ALL rows from the
+   * Arrow batch and stores them as a list using MULTIPLE_RECORDS_KEY.
+   *
+   * @param reader ArrowStreamReader containing the data
+   * @param root Arrow VectorSchemaRoot containing the data
+   * @param destination Optional destination {@code GenericRow}, will create 
new if null
+   * @return {@code GenericRow} containing {@code List<GenericRow>} with all 
converted rows, or null
+   *     if no data available
+   */
+  @Nullable
+  public GenericRow convert(
+      ArrowStreamReader reader, VectorSchemaRoot root, GenericRow destination) 
{
+    if (root == null) {
+      logger.warn("Cannot convert null VectorSchemaRoot");
+      return null;
+    }
+
+    if (destination == null) {
+      destination = new GenericRow();
+    }
+
+    int rowCount = root.getRowCount();
+    if (rowCount == 0) {
+      logger.warn("No rows found in Arrow data");
+      return destination;
+    }
+
+    List<GenericRow> rows = new ArrayList<>(rowCount);
+
+    // Process all rows from the Arrow batch
+    for (int rowIndex = 0; rowIndex < rowCount; rowIndex++) {
+      GenericRow row = convertSingleRow(reader, root, rowIndex);
+      if (row != null) {
+        rows.add(row);
+      }
+    }
+
+    if (!rows.isEmpty()) {
+      // Use Pinot's MULTIPLE_RECORDS_KEY to store the list of rows
+      destination.putValue(GenericRow.MULTIPLE_RECORDS_KEY, rows);
+      logger.debug("Converted {} rows from Arrow batch", rows.size());
+    }
+
+    return destination;
+  }
+
+  /**
+   * Converts a single row from Arrow VectorSchemaRoot.
+   *
+   * @param reader ArrowStreamReader containing the data
+   * @param root Arrow VectorSchemaRoot containing the data
+   * @param rowIndex Index of the row to convert (0-based)
+   * @return {@code GenericRow} with converted data, or null if row index is 
invalid
+   */
+  @Nullable
+  private GenericRow convertSingleRow(
+      ArrowStreamReader reader, VectorSchemaRoot root, int rowIndex) {
+    GenericRow row = new GenericRow();
+    int convertedFields = 0;
+
+    // Process all fields in the Arrow schema
+    for (int i = 0; i < root.getFieldVectors().size(); i++) {
+      Object value;
+
+      FieldVector fieldVector = root.getFieldVectors().get(i);
+      String fieldName = fieldVector.getField().getName();
+      try {
+        if (fieldVector.getField().getDictionary() != null) {
+          long dictionaryId = fieldVector.getField().getDictionary().getId();
+          try (ValueVector realFieldVector =
+              DictionaryEncoder.decode(
+                  fieldVector, 
reader.getDictionaryVectors().get(dictionaryId))) {
+            value = realFieldVector.getObject(rowIndex);
+          }
+        } else {
+          value = fieldVector.getObject(rowIndex);
+        }
+        if (value != null) {
+          // Convert Arrow-specific types to Pinot-compatible types
+          Object pinotCompatibleValue = 
convertArrowTypeToPinotCompatible(value);
+          row.putValue(fieldName, pinotCompatibleValue);
+          convertedFields++;
+        }
+      } catch (Exception e) {
+        logger.error("Error extracting value for field: {} at row {}", 
fieldName, rowIndex, e);
+      }
+    }
+
+    logger.debug("Converted {} fields from Arrow row {} to GenericRow", 
convertedFields, rowIndex);
+    return row;
+  }
+
+  /**
+   * Converts Arrow-specific data types to Pinot-compatible types. This method 
handles the
+   * incompatibility issues between Arrow's native data types and what Pinot 
expects.
+   *
+   * @param value The raw value from Arrow fieldVector.getObject()
+   * @return A Pinot-compatible version of the value
+   */
+  @Nullable
+  private Object convertArrowTypeToPinotCompatible(@Nullable Object value) {
+    if (value == null) {
+      return null;
+    }
+
+    // Handle nested List and Map values, including Arrow MapVector's 
representation
+    if (value instanceof List) {
+      List<?> originalList = (List<?>) value;
+      if (!originalList.isEmpty()) {
+        boolean looksLikeMapEntries = true;
+        boolean sawNonNull = false;
+        for (Object entryObj : originalList) {
+          if (entryObj == null) {
+            continue;
+          }
+          sawNonNull = true;
+          if (!(entryObj instanceof Map)) {
+            looksLikeMapEntries = false;
+            break;
+          }
+          @SuppressWarnings("unchecked")
+          Map<Object, Object> entryMap = (Map<Object, Object>) entryObj;
+          if (!entryMap.containsKey(MapVector.KEY_NAME)) {
+            looksLikeMapEntries = false;
+            break;
+          }
+        }
+        if (looksLikeMapEntries && sawNonNull) {
+          Map<String, Object> flattened = new 
LinkedHashMap<>(originalList.size());
+          for (Object entryObj : originalList) {
+            if (entryObj == null) {
+              continue;
+            }
+            @SuppressWarnings("unchecked")
+            Map<Object, Object> entryMap = (Map<Object, Object>) entryObj;
+            Object rawKey = entryMap.get(MapVector.KEY_NAME);
+            Object rawVal = entryMap.get(MapVector.VALUE_NAME);
+            Object convertedKey = convertArrowTypeToPinotCompatible(rawKey);
+            Object convertedVal = convertArrowTypeToPinotCompatible(rawVal);
+            flattened.put(String.valueOf(convertedKey), convertedVal);
+          }
+          return flattened;
+        }
+      }
+
+      List<Object> convertedList = new ArrayList<>(originalList.size());
+      for (Object element : originalList) {
+        convertedList.add(convertArrowTypeToPinotCompatible(element));
+      }
+      return convertedList;
+    }
+
+    // Handle Arrow Text type -> String conversion
+    if (value instanceof Text) {
+      // Arrow VarCharVector.getObject() returns Text objects, but Pinot 
expects String
+      return value.toString();
+    }
+
+    // Handle Arrow LocalDateTime -> java.sql.Timestamp conversion
+    if (value instanceof LocalDateTime) {
+      // Arrow TimeStampMilliVector.getObject() returns LocalDateTime, but 
Pinot expects
+      // java.sql.Timestamp objects for proper timestamp handling and native 
support
+      LocalDateTime dateTime = (LocalDateTime) value;
+      return Timestamp.from(dateTime.toInstant(ZoneOffset.UTC));
+    }
+
+    // Handle other potential Arrow-specific types that might cause issues
+
+    // For primitive types (Integer, Double, Boolean) and other Java standard 
types,
+    // Arrow returns standard Java objects that are already Pinot-compatible
+    return value;
+  }
+
+  private Object flattenArrowMap(MapVector fieldVector, int rowIndex) {
+    Map<String, Object> flattened = new LinkedHashMap<>();
+    UnionMapReader reader = fieldVector.getReader();
+    reader.setPosition(rowIndex);
+    while (reader.next()) {
+      flattened.put(
+          reader.key().readObject().toString(),
+          convertArrowTypeToPinotCompatible(reader.value().readObject()));
+    }
+    return flattened;
+  }

Review Comment:
   The method `flattenArrowMap` is defined but never called within this class. 
This is dead code that should either be removed or integrated into the 
conversion logic if it was intended to handle MapVector types.
   ```suggestion
   
   ```



##########
pinot-plugins/pinot-input-format/pinot-arrow/src/main/java/org/apache/pinot/plugin/inputformat/arrow/ArrowMessageDecoder.java:
##########
@@ -0,0 +1,122 @@
+/**
+ * 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.inputformat.arrow;
+
+
+import java.io.ByteArrayInputStream;
+import java.nio.channels.Channels;
+import java.nio.channels.ReadableByteChannel;
+import java.util.Arrays;
+import java.util.Map;
+import java.util.Set;
+import javax.annotation.Nullable;
+import org.apache.arrow.memory.RootAllocator;
+import org.apache.arrow.vector.VectorSchemaRoot;
+import org.apache.arrow.vector.ipc.ArrowStreamReader;
+import org.apache.pinot.spi.data.readers.GenericRow;
+import org.apache.pinot.spi.stream.StreamMessageDecoder;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * ArrowMessageDecoder is used to decode Apache Arrow IPC format messages into 
Pinot GenericRow.
+ * This decoder handles Arrow streaming format and converts Arrow data to 
Pinot's columnar format.
+ */
+public class ArrowMessageDecoder implements StreamMessageDecoder<byte[]> {
+  public static final String ARROW_SCHEMA_CONFIG = "arrow.schema.config";
+  public static final String ARROW_ALLOCATOR_LIMIT = "arrow.allocator.limit";
+  public static final String DEFAULT_ALLOCATOR_LIMIT = "268435456"; // 256MB 
default
+
+  private static final Logger logger = 
LoggerFactory.getLogger(ArrowMessageDecoder.class);
+
+  private String _kafkaTopicName;
+  private Set<String> _fieldsToRead;
+  private RootAllocator _allocator;
+  private ArrowToGenericRowConverter _converter;
+
+  @Override
+  public void init(Map<String, String> props, Set<String> fieldsToRead, String 
topicName)
+      throws Exception {
+    _kafkaTopicName = topicName;
+    _fieldsToRead = fieldsToRead;
+
+    // Initialize Arrow allocator with configurable memory limit
+    long allocatorLimit =
+        Long.parseLong(props.getOrDefault(ARROW_ALLOCATOR_LIMIT, 
DEFAULT_ALLOCATOR_LIMIT));
+    _allocator = new RootAllocator(allocatorLimit);
+
+    // Initialize Arrow to GenericRow converter (processes all fields)
+    _converter = new ArrowToGenericRowConverter();
+
+    logger.info(
+        "Initialized ArrowMessageDecoder for topic: {} with allocator limit: 
{} bytes",
+        topicName,
+        allocatorLimit);
+  }
+
+  @Nullable
+  @Override
+  public GenericRow decode(byte[] payload, GenericRow destination) {
+    try (ByteArrayInputStream inputStream = new ByteArrayInputStream(payload);
+        ReadableByteChannel channel = Channels.newChannel(inputStream);
+        ArrowStreamReader reader = new ArrowStreamReader(channel, _allocator)) 
{
+
+      // Read the Arrow schema and data
+      VectorSchemaRoot root = reader.getVectorSchemaRoot();
+      if (!reader.loadNextBatch()) {
+        logger.warn("No data found in Arrow message for topic: {}", 
_kafkaTopicName);
+        return null;
+      }
+
+      // Convert Arrow data to GenericRow using converter
+      GenericRow row = _converter.convert(reader, root, destination);
+
+      return row;
+    } catch (Exception e) {
+      logger.error(
+          "Error decoding Arrow message for kafka topic {} : {}",
+          _kafkaTopicName,
+          Arrays.toString(payload),
+          e);
+      return null;
+    }
+  }
+
+  @Nullable
+  @Override
+  public GenericRow decode(byte[] payload, int offset, int length, GenericRow 
destination) {
+    return decode(Arrays.copyOfRange(payload, offset, offset + length), 
destination);
+  }
+
+  private String buildEventLatencyMetricName(String kafkaTopicName) {
+    return "ServerMetrics.realtime.arrow.consumer.delay."
+        + kafkaTopicName.replace(".", "_");
+  }
+

Review Comment:
   The method `buildEventLatencyMetricName` is defined but never called. This 
appears to be dead code that should be removed unless metrics collection is 
planned for future implementation.
   ```suggestion
   
   ```



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

To unsubscribe, e-mail: [email protected]

For queries about this service, please contact Infrastructure at:
[email protected]


---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]

Reply via email to