Copilot commented on code in PR #54221: URL: https://github.com/apache/doris/pull/54221#discussion_r2650423074
########## be/src/exec/schema_scanner/schema_frontend_metrics_scanner.h: ########## @@ -0,0 +1,53 @@ +// 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. + +#pragma once + +#include <memory> +#include <vector> + +#include "common/status.h" +#include "exec/schema_scanner.h" +#include "gen_cpp/FrontendService_types.h" + +namespace doris { + +class RuntimeState; + +namespace vectorized { +class Block; +} + +class SchemaFrontendMetricsScanner : public SchemaScanner { + ENABLE_FACTORY_CREATOR(SchemaFrontendMetricsScanner); + +public: + SchemaFrontendMetricsScanner(); + ~SchemaFrontendMetricsScanner() override; + + Status start(RuntimeState* state) override; + Status get_next_block_internal(vectorized::Block* block, bool* eos) override; + + static std::vector<SchemaScanner::ColumnDesc> _s_frontend_metric_columns; Review Comment: The variable name "_s_frontend_metric_columns" is inconsistent with the naming convention used by other schema scanners in this codebase, which use "_s_tbls_columns" (e.g., schema_backend_metrics_scanner.h:41, schema_tables_scanner.h:51). For consistency, this should be renamed to "_s_tbls_columns". ```suggestion static std::vector<SchemaScanner::ColumnDesc> _s_tbls_columns; ``` ########## be/src/util/metrics.cpp: ########## @@ -422,4 +426,65 @@ std::string MetricRegistry::to_core_string() const { return ss.str(); } +void MetricRegistry::get_be_metrics_block(vectorized::Block* block) const { + int64_t be_id = ExecEnv::GetInstance()->cluster_info()->backend_id; + std::string be_ip = BackendOptions::get_localhost(); + + std::lock_guard<std::mutex> l(_lock); + for (const auto& entity : _entities) { + std::lock_guard<std::mutex> entity_lock(entity.first->_lock); + entity.first->trigger_hook_unlocked(false); + + for (const auto& metric : entity.first->_metrics) { + rj::Document tag_doc(rj::kObjectType); + rj::Document::AllocatorType& allocator = tag_doc.GetAllocator(); + + for (auto& label : metric.first->labels) { + tag_doc.AddMember(rj::Value(label.first.c_str(), allocator), + rj::Value(label.second.c_str(), allocator), allocator); + } + + for (auto& label : entity.first->_labels) { + tag_doc.AddMember(rj::Value(label.first.c_str(), allocator), + rj::Value(label.second.c_str(), allocator), allocator); + } + + rj::StringBuffer tag_buf; + rj::Writer<rj::StringBuffer> tag_writer(tag_buf); + tag_doc.Accept(tag_writer); + std::string tag_str = tag_buf.GetString(); + + std::string metric_type; + switch (metric.first->type) { + case MetricType::COUNTER: + metric_type = "counter"; + break; + case MetricType::GAUGE: + metric_type = "gauge"; + break; + case MetricType::HISTOGRAM: + metric_type = "histogram"; + break; + default: + metric_type = "unknown"; + } + + double metric_value = 0.0; + try { + std::string value_str = metric.second->to_string(); + metric_value = std::stod(value_str); // 字符串转double Review Comment: The comment is in Chinese ("字符串转double" meaning "convert string to double"). For consistency with the rest of the codebase which uses English comments, this should be translated to English or removed, especially since the code is self-explanatory. ```suggestion metric_value = std::stod(value_str); // convert string to double ``` ########## be/src/exec/schema_scanner/schema_backend_metrics_scanner.h: ########## @@ -0,0 +1,49 @@ +// 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. + +#pragma once + +#include <vector> + +#include "common/status.h" +#include "exec/schema_scanner.h" + +namespace doris { +class RuntimeState; +namespace vectorized { +class Block; +} // namespace vectorized + +class SchemaBackendMetricsScanner : public SchemaScanner { + ENABLE_FACTORY_CREATOR(SchemaBackendMetricsScanner); + +public: + SchemaBackendMetricsScanner(); + ~SchemaBackendMetricsScanner() override; + + Status start(RuntimeState* state) override; + Status get_next_block_internal(vectorized::Block* block, bool* eos) override; + + static std::vector<SchemaScanner::ColumnDesc> _s_tbls_columns; + +private: + int _block_rows_limit = 4096; + int _row_idx = 0; + int _total_rows = 0; + std::unique_ptr<vectorized::Block> _backend_metrics_block = nullptr; +}; +}; // namespace doris Review Comment: There's an extra semicolon after the closing brace of the namespace. The correct syntax is "} // namespace doris" without a semicolon. This is inconsistent with standard C++ namespace closing conventions. ```suggestion } // namespace doris ``` ########## fe/fe-core/src/main/java/org/apache/doris/metric/ListMetricVisitor.java: ########## @@ -0,0 +1,274 @@ +// 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.doris.metric; + +import org.apache.doris.catalog.Env; +import org.apache.doris.monitor.jvm.JvmStats; +import org.apache.doris.monitor.jvm.JvmStats.GarbageCollector; +import org.apache.doris.monitor.jvm.JvmStats.MemoryPool; +import org.apache.doris.monitor.jvm.JvmStats.Threads; + +import com.codahale.metrics.Histogram; +import com.codahale.metrics.Snapshot; +import com.google.common.base.Joiner; +import org.apache.logging.log4j.LogManager; +import org.apache.logging.log4j.Logger; + +import java.util.ArrayList; +import java.util.Collections; +import java.util.Iterator; +import java.util.List; +import java.util.Map; +import java.util.stream.Collectors; + +/* + * Like this: + * [ + * ['FE', 'METRIC_NAME', 'METRIC_TYPE', 'METRIC_VALUE', 'TAG'], + * [] + * ] + */ +public class ListMetricVisitor extends MetricVisitor { + private static final Logger logger = LogManager.getLogger(ListMetricVisitor.class); + + private List<List<String>> metricsList; + private String localHostAddr; + private StringBuilder strConcat; + + // jvm + private static final String JVM_HEAP_SIZE_BYTES = "jvm_heap_size_bytes"; + private static final String JVM_NON_HEAP_SIZE_BYTES = "jvm_non_heap_size_bytes"; + private static final String JVM_YOUNG_SIZE_BYTES = "jvm_young_size_bytes"; + private static final String JVM_OLD_SIZE_BYTES = "jvm_old_size_bytes"; + private static final String JVM_THREAD = "jvm_thread"; + + private static final String JVM_GC = "jvm_gc"; + private static final String TYPE_GAUGE = "gauge"; + private static final String TYPE_COUNTER = "counter"; + private static final String TYPE_HISTOGRAM = "histogram"; + + public ListMetricVisitor(List<List<String>> metricsList, String localHostAddr) { + super(); + // List[ List[FE, METRIC_NAME, METRIC_TYPE, METRIC_VALUE, TAG] ] + this.metricsList = metricsList; + this.localHostAddr = localHostAddr; + this.strConcat = new StringBuilder(); + } + + @Override + public void visitJvm(JvmStats jvmStats) { + // heap + this.metricsList.add(newGaugeSubList(JVM_HEAP_SIZE_BYTES, + jvmStats.getMem().getHeapMax().getBytes(), "{\"type\"=\"max\"}")); + this.metricsList.add(newGaugeSubList(JVM_HEAP_SIZE_BYTES, + jvmStats.getMem().getHeapCommitted().getBytes(), "{\"type\"=\"committed\"}")); + this.metricsList.add(newGaugeSubList(JVM_HEAP_SIZE_BYTES, + jvmStats.getMem().getHeapUsed().getBytes(), "{\"type\"=\"used\"}")); + + // non heap + this.metricsList.add(newGaugeSubList(JVM_NON_HEAP_SIZE_BYTES, + jvmStats.getMem().getNonHeapCommitted().getBytes(), "{\"type\"=\"committed\"}")); + this.metricsList.add(newGaugeSubList(JVM_NON_HEAP_SIZE_BYTES, + jvmStats.getMem().getNonHeapUsed().getBytes(), "{\"type\"=\"used\"}")); + + // mem pool + Iterator<MemoryPool> memIter = jvmStats.getMem().iterator(); + while (memIter.hasNext()) { + MemoryPool memPool = memIter.next(); + if (memPool.getName().equalsIgnoreCase("young")) { + this.metricsList.add(newGaugeSubList(JVM_YOUNG_SIZE_BYTES, + memPool.getUsed().getBytes(), "{\"type\"=\"used\"}")); + this.metricsList.add(newGaugeSubList(JVM_YOUNG_SIZE_BYTES, + memPool.getPeakUsed().getBytes(), "{\"type\"=\"peak_used\"}")); + this.metricsList.add(newGaugeSubList(JVM_YOUNG_SIZE_BYTES, + memPool.getMax().getBytes(), "{\"type\"=\"max\"}")); + } else if (memPool.getName().equalsIgnoreCase("old")) { + this.metricsList.add(newGaugeSubList(JVM_OLD_SIZE_BYTES, + memPool.getUsed().getBytes(), "{\"type\"=\"used\"}")); + this.metricsList.add(newGaugeSubList(JVM_OLD_SIZE_BYTES, + memPool.getPeakUsed().getBytes(), "{\"type\"=\"peak_used\"}")); + this.metricsList.add(newGaugeSubList(JVM_OLD_SIZE_BYTES, + memPool.getMax().getBytes(), "{\"type\"=\"max\"}")); + } + } + + // gc + for (GarbageCollector gc : jvmStats.getGc()) { + this.metricsList.add(newGaugeSubList(JVM_GC, + gc.getCollectionCount(), "{\"name\"=\"" + gc.getName() + "\",\"type\"=\"count\"}")); + this.metricsList.add(newGaugeSubList(JVM_GC, + gc.getCollectionTime().getMillis(), "{\"name\"=\"" + gc.getName() + "\",\"type\"=\"time\"}")); + } + + // threads + Threads threads = jvmStats.getThreads(); + this.metricsList.add(newGaugeSubList(JVM_THREAD, + threads.getCount(), "\"type\"=\"count\"}")); + this.metricsList.add(newGaugeSubList(JVM_THREAD, + threads.getPeakCount(), "\"type\"=\"peak_count\"}")); + this.metricsList.add(newGaugeSubList(JVM_THREAD, + threads.getThreadsNewCount(), "\"type\"=\"new_count\"}")); + this.metricsList.add(newGaugeSubList(JVM_THREAD, + threads.getThreadsRunnableCount(), "\"type\"=\"runnable_count\"}")); + this.metricsList.add(newGaugeSubList(JVM_THREAD, + threads.getThreadsBlockedCount(), "\"type\"=\"blocked_count\"}")); + this.metricsList.add(newGaugeSubList(JVM_THREAD, + threads.getThreadsWaitingCount(), "\"type\"=\"waiting_count\"}")); + this.metricsList.add(newGaugeSubList(JVM_THREAD, + threads.getThreadsTimedWaitingCount(), "\"type\"=\"timed_waiting_count\"}")); + this.metricsList.add(newGaugeSubList(JVM_THREAD, + threads.getThreadsTerminatedCount(), "\"type\"=\"terminated_count\"}")); Review Comment: Missing opening brace in the tag string. The tag should be "{\"type\"=\"waiting_count\"}" but it's currently "\"type\"=\"waiting_count\"}", which is missing the opening brace. This inconsistency with other tag strings in this file will cause malformed JSON-like tags. ```suggestion threads.getCount(), "{\"type\"=\"count\"}")); this.metricsList.add(newGaugeSubList(JVM_THREAD, threads.getPeakCount(), "{\"type\"=\"peak_count\"}")); this.metricsList.add(newGaugeSubList(JVM_THREAD, threads.getThreadsNewCount(), "{\"type\"=\"new_count\"}")); this.metricsList.add(newGaugeSubList(JVM_THREAD, threads.getThreadsRunnableCount(), "{\"type\"=\"runnable_count\"}")); this.metricsList.add(newGaugeSubList(JVM_THREAD, threads.getThreadsBlockedCount(), "{\"type\"=\"blocked_count\"}")); this.metricsList.add(newGaugeSubList(JVM_THREAD, threads.getThreadsWaitingCount(), "{\"type\"=\"waiting_count\"}")); this.metricsList.add(newGaugeSubList(JVM_THREAD, threads.getThreadsTimedWaitingCount(), "{\"type\"=\"timed_waiting_count\"}")); this.metricsList.add(newGaugeSubList(JVM_THREAD, threads.getThreadsTerminatedCount(), "{\"type\"=\"terminated_count\"}")); ``` ########## be/src/exec/schema_scanner/schema_frontend_metrics_scanner.cpp: ########## @@ -0,0 +1,132 @@ +// 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. + +#include "exec/schema_scanner/schema_frontend_metrics_scanner.h" + +#include <gen_cpp/FrontendService_types.h> + +#include <exception> +#include <vector> + +#include "exec/schema_scanner/schema_helper.h" +#include "runtime/define_primitive_type.h" +#include "runtime/runtime_state.h" +#include "vec/common/string_ref.h" +#include "vec/core/block.h" +#include "vec/data_types/data_type_factory.hpp" + +namespace doris { +#include "common/compile_check_begin.h" + +std::vector<SchemaScanner::ColumnDesc> SchemaFrontendMetricsScanner::_s_frontend_metric_columns = { + // name, type, size, is_null + {"FE", TYPE_VARCHAR, sizeof(StringRef), true}, + {"METRIC_NAME", TYPE_VARCHAR, sizeof(StringRef), true}, + {"METRIC_TYPE", TYPE_VARCHAR, sizeof(StringRef), true}, + {"METRIC_VALUE", TYPE_DOUBLE, sizeof(double), true}, + {"TAG", TYPE_VARCHAR, sizeof(StringRef), true}}; + +SchemaFrontendMetricsScanner::SchemaFrontendMetricsScanner() + : SchemaScanner(_s_frontend_metric_columns, TSchemaTableType::SCH_FE_METRICS) {} + +SchemaFrontendMetricsScanner::~SchemaFrontendMetricsScanner() = default; + +Status SchemaFrontendMetricsScanner::start(RuntimeState* state) { + TFetchFeMetricsRequest request; + + for (const auto& fe_addr : _param->common_param->fe_addr_list) { + TFetchFeMetricsResult tmp_ret; + RETURN_IF_ERROR(SchemaHelper::fetch_frontend_metrics(fe_addr.hostname, fe_addr.port, + request, &tmp_ret)); + + _metrics_list_result.metrics_list.insert(_metrics_list_result.metrics_list.end(), + tmp_ret.metrics_list.begin(), + tmp_ret.metrics_list.end()); + } + + return Status::OK(); +} + +Status SchemaFrontendMetricsScanner::get_next_block_internal(vectorized::Block* block, bool* eos) { + if (!_is_init) { + return Status::InternalError("call this before initial."); + } + if (block == nullptr || eos == nullptr) { + return Status::InternalError("invalid parameter."); + } + + *eos = true; + if (_metrics_list_result.metrics_list.empty()) { + return Status::OK(); + } + + return _fill_block_impl(block); +} + +Status SchemaFrontendMetricsScanner::_fill_block_impl(vectorized::Block* block) { + SCOPED_TIMER(_fill_block_timer); + + const auto& metrics_list = _metrics_list_result.metrics_list; + size_t row_num = metrics_list.size(); + if (row_num == 0) { + return Status::OK(); + } + + for (size_t col_idx = 0; col_idx < _s_frontend_metric_columns.size(); ++col_idx) { + std::vector<StringRef> str_refs(row_num); + std::vector<double> double_vals(row_num); + std::vector<void*> datas(row_num); + std::vector<std::string> column_values( + row_num); // Store the strings to ensure their lifetime + + for (size_t row_idx = 0; row_idx < row_num; ++row_idx) { + const auto& row = metrics_list[row_idx]; + if (row.size() != _s_frontend_metric_columns.size()) { + return Status::InternalError( + "process list meet invalid schema, schema_size={}, input_data_size={}", + _s_frontend_metric_columns.size(), row.size()); Review Comment: The error message references "process list" which is a copy-paste error from SchemaProcesslistScanner. The error message should reference "frontend metrics" instead to accurately reflect the context of this scanner. ########## fe/fe-core/src/main/java/org/apache/doris/metric/ListMetricVisitor.java: ########## @@ -0,0 +1,274 @@ +// 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.doris.metric; + +import org.apache.doris.catalog.Env; +import org.apache.doris.monitor.jvm.JvmStats; +import org.apache.doris.monitor.jvm.JvmStats.GarbageCollector; +import org.apache.doris.monitor.jvm.JvmStats.MemoryPool; +import org.apache.doris.monitor.jvm.JvmStats.Threads; + +import com.codahale.metrics.Histogram; +import com.codahale.metrics.Snapshot; +import com.google.common.base.Joiner; +import org.apache.logging.log4j.LogManager; +import org.apache.logging.log4j.Logger; + +import java.util.ArrayList; +import java.util.Collections; +import java.util.Iterator; +import java.util.List; +import java.util.Map; +import java.util.stream.Collectors; + +/* + * Like this: + * [ + * ['FE', 'METRIC_NAME', 'METRIC_TYPE', 'METRIC_VALUE', 'TAG'], + * [] + * ] + */ +public class ListMetricVisitor extends MetricVisitor { + private static final Logger logger = LogManager.getLogger(ListMetricVisitor.class); + + private List<List<String>> metricsList; + private String localHostAddr; + private StringBuilder strConcat; + + // jvm + private static final String JVM_HEAP_SIZE_BYTES = "jvm_heap_size_bytes"; + private static final String JVM_NON_HEAP_SIZE_BYTES = "jvm_non_heap_size_bytes"; + private static final String JVM_YOUNG_SIZE_BYTES = "jvm_young_size_bytes"; + private static final String JVM_OLD_SIZE_BYTES = "jvm_old_size_bytes"; + private static final String JVM_THREAD = "jvm_thread"; + + private static final String JVM_GC = "jvm_gc"; + private static final String TYPE_GAUGE = "gauge"; + private static final String TYPE_COUNTER = "counter"; + private static final String TYPE_HISTOGRAM = "histogram"; + + public ListMetricVisitor(List<List<String>> metricsList, String localHostAddr) { + super(); + // List[ List[FE, METRIC_NAME, METRIC_TYPE, METRIC_VALUE, TAG] ] + this.metricsList = metricsList; + this.localHostAddr = localHostAddr; + this.strConcat = new StringBuilder(); + } + + @Override + public void visitJvm(JvmStats jvmStats) { + // heap + this.metricsList.add(newGaugeSubList(JVM_HEAP_SIZE_BYTES, + jvmStats.getMem().getHeapMax().getBytes(), "{\"type\"=\"max\"}")); + this.metricsList.add(newGaugeSubList(JVM_HEAP_SIZE_BYTES, + jvmStats.getMem().getHeapCommitted().getBytes(), "{\"type\"=\"committed\"}")); + this.metricsList.add(newGaugeSubList(JVM_HEAP_SIZE_BYTES, + jvmStats.getMem().getHeapUsed().getBytes(), "{\"type\"=\"used\"}")); + + // non heap + this.metricsList.add(newGaugeSubList(JVM_NON_HEAP_SIZE_BYTES, + jvmStats.getMem().getNonHeapCommitted().getBytes(), "{\"type\"=\"committed\"}")); + this.metricsList.add(newGaugeSubList(JVM_NON_HEAP_SIZE_BYTES, + jvmStats.getMem().getNonHeapUsed().getBytes(), "{\"type\"=\"used\"}")); + + // mem pool + Iterator<MemoryPool> memIter = jvmStats.getMem().iterator(); + while (memIter.hasNext()) { + MemoryPool memPool = memIter.next(); + if (memPool.getName().equalsIgnoreCase("young")) { + this.metricsList.add(newGaugeSubList(JVM_YOUNG_SIZE_BYTES, + memPool.getUsed().getBytes(), "{\"type\"=\"used\"}")); + this.metricsList.add(newGaugeSubList(JVM_YOUNG_SIZE_BYTES, + memPool.getPeakUsed().getBytes(), "{\"type\"=\"peak_used\"}")); + this.metricsList.add(newGaugeSubList(JVM_YOUNG_SIZE_BYTES, + memPool.getMax().getBytes(), "{\"type\"=\"max\"}")); + } else if (memPool.getName().equalsIgnoreCase("old")) { + this.metricsList.add(newGaugeSubList(JVM_OLD_SIZE_BYTES, + memPool.getUsed().getBytes(), "{\"type\"=\"used\"}")); + this.metricsList.add(newGaugeSubList(JVM_OLD_SIZE_BYTES, + memPool.getPeakUsed().getBytes(), "{\"type\"=\"peak_used\"}")); + this.metricsList.add(newGaugeSubList(JVM_OLD_SIZE_BYTES, + memPool.getMax().getBytes(), "{\"type\"=\"max\"}")); + } + } + + // gc + for (GarbageCollector gc : jvmStats.getGc()) { + this.metricsList.add(newGaugeSubList(JVM_GC, + gc.getCollectionCount(), "{\"name\"=\"" + gc.getName() + "\",\"type\"=\"count\"}")); + this.metricsList.add(newGaugeSubList(JVM_GC, + gc.getCollectionTime().getMillis(), "{\"name\"=\"" + gc.getName() + "\",\"type\"=\"time\"}")); + } + + // threads + Threads threads = jvmStats.getThreads(); + this.metricsList.add(newGaugeSubList(JVM_THREAD, + threads.getCount(), "\"type\"=\"count\"}")); + this.metricsList.add(newGaugeSubList(JVM_THREAD, + threads.getPeakCount(), "\"type\"=\"peak_count\"}")); + this.metricsList.add(newGaugeSubList(JVM_THREAD, + threads.getThreadsNewCount(), "\"type\"=\"new_count\"}")); + this.metricsList.add(newGaugeSubList(JVM_THREAD, + threads.getThreadsRunnableCount(), "\"type\"=\"runnable_count\"}")); + this.metricsList.add(newGaugeSubList(JVM_THREAD, + threads.getThreadsBlockedCount(), "\"type\"=\"blocked_count\"}")); + this.metricsList.add(newGaugeSubList(JVM_THREAD, + threads.getThreadsWaitingCount(), "\"type\"=\"waiting_count\"}")); + this.metricsList.add(newGaugeSubList(JVM_THREAD, + threads.getThreadsTimedWaitingCount(), "\"type\"=\"timed_waiting_count\"}")); + this.metricsList.add(newGaugeSubList(JVM_THREAD, + threads.getThreadsTerminatedCount(), "\"type\"=\"terminated_count\"}")); + } + + @Override + public void visit(String prefix, @SuppressWarnings("rawtypes") Metric metric) { + // title + final String fullName = prefix + metric.getName(); + // name + strConcat.setLength(0); + List<MetricLabel> labels = metric.getLabels(); + if (!labels.isEmpty()) { + strConcat.append("{"); + List<String> labelStrs = labels.stream().map(l -> "\"" + l.getKey() + "\"" + "=\"" + l.getValue() + + "\"").collect(Collectors.toList()); + strConcat.append(Joiner.on(", ").join(labelStrs)); + strConcat.append("}"); + } + // value + List<String> metricStr = + newSubListByType(fullName, metric.getValue().toString(), strConcat.toString(), metric.getType()); + if (metricStr != null) { + this.metricsList.add(metricStr); + } + } + + @Override + public void visitHistogram(String prefix, String name, Histogram histogram) { + // part.part.part.k1=v1.k2=v2 + List<String> names = new ArrayList<>(); + List<String> tags = new ArrayList<>(); + for (String part : name.split("\\.")) { + String[] kv = part.split("="); + if (kv.length == 1) { + names.add(kv[0]); + } else if (kv.length == 2) { + tags.add(String.format("\"%s\"=\"%s\"", kv[0], kv[1])); + } + } + final String fullName = prefix + String.join("_", names); + final String fullTag = String.join(",", tags); + String delimiter = tags.isEmpty() ? "" : ","; + Snapshot snapshot = histogram.getSnapshot(); + + this.metricsList.add(newHistogramSubList(fullName, + snapshot.get75thPercentile(), "{\"quantile\"=\"0.75\"" + delimiter + fullTag + "}")); + this.metricsList.add(newHistogramSubList(fullName, + snapshot.get95thPercentile(), "{\"quantile\"=\"0.95\"" + delimiter + fullTag + "}")); + this.metricsList.add(newHistogramSubList(fullName, + snapshot.get98thPercentile(), "{\"quantile\"=\"0.98\"" + delimiter + fullTag + "}")); + this.metricsList.add(newHistogramSubList(fullName, + snapshot.get99thPercentile(), "{\"quantile\"=\"0.99\"" + delimiter + fullTag + "}")); + this.metricsList.add(newHistogramSubList(fullName, + snapshot.get999thPercentile(), "{\"quantile\"=\"0.999\"" + delimiter + fullTag + "}")); + this.metricsList.add(newHistogramSubList(fullName + "_sum", + histogram.getCount() * snapshot.getMean(), + "{\"quantile\"=\"0.75\"" + delimiter + fullTag + "}")); + this.metricsList.add(newHistogramSubList(fullName + "_count", + histogram.getCount(), "{\"quantile\"=\"0.75\"" + delimiter + fullTag + "}")); Review Comment: The tag used for the "_sum" metric should not include the "quantile" label, as sum is not associated with a quantile. Additionally, using "{\"quantile\"=\"0.75\"" is misleading since this represents the total sum, not the 75th percentile. Consider removing the quantile label entirely or using an empty tag set for the sum metric. ```suggestion // sum and count metrics should not carry a quantile label final String sumAndCountTags = fullTag.isEmpty() ? "{}" : "{" + fullTag + "}"; this.metricsList.add(newHistogramSubList(fullName + "_sum", histogram.getCount() * snapshot.getMean(), sumAndCountTags)); this.metricsList.add(newHistogramSubList(fullName + "_count", histogram.getCount(), sumAndCountTags)); ``` ########## fe/fe-core/src/main/java/org/apache/doris/metric/ListMetricVisitor.java: ########## @@ -0,0 +1,274 @@ +// 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.doris.metric; + +import org.apache.doris.catalog.Env; +import org.apache.doris.monitor.jvm.JvmStats; +import org.apache.doris.monitor.jvm.JvmStats.GarbageCollector; +import org.apache.doris.monitor.jvm.JvmStats.MemoryPool; +import org.apache.doris.monitor.jvm.JvmStats.Threads; + +import com.codahale.metrics.Histogram; +import com.codahale.metrics.Snapshot; +import com.google.common.base.Joiner; +import org.apache.logging.log4j.LogManager; +import org.apache.logging.log4j.Logger; + +import java.util.ArrayList; +import java.util.Collections; +import java.util.Iterator; +import java.util.List; +import java.util.Map; +import java.util.stream.Collectors; + +/* + * Like this: + * [ + * ['FE', 'METRIC_NAME', 'METRIC_TYPE', 'METRIC_VALUE', 'TAG'], + * [] + * ] + */ +public class ListMetricVisitor extends MetricVisitor { + private static final Logger logger = LogManager.getLogger(ListMetricVisitor.class); + + private List<List<String>> metricsList; + private String localHostAddr; + private StringBuilder strConcat; + + // jvm + private static final String JVM_HEAP_SIZE_BYTES = "jvm_heap_size_bytes"; + private static final String JVM_NON_HEAP_SIZE_BYTES = "jvm_non_heap_size_bytes"; + private static final String JVM_YOUNG_SIZE_BYTES = "jvm_young_size_bytes"; + private static final String JVM_OLD_SIZE_BYTES = "jvm_old_size_bytes"; + private static final String JVM_THREAD = "jvm_thread"; + + private static final String JVM_GC = "jvm_gc"; + private static final String TYPE_GAUGE = "gauge"; + private static final String TYPE_COUNTER = "counter"; + private static final String TYPE_HISTOGRAM = "histogram"; + + public ListMetricVisitor(List<List<String>> metricsList, String localHostAddr) { + super(); + // List[ List[FE, METRIC_NAME, METRIC_TYPE, METRIC_VALUE, TAG] ] + this.metricsList = metricsList; + this.localHostAddr = localHostAddr; + this.strConcat = new StringBuilder(); + } + + @Override + public void visitJvm(JvmStats jvmStats) { + // heap + this.metricsList.add(newGaugeSubList(JVM_HEAP_SIZE_BYTES, + jvmStats.getMem().getHeapMax().getBytes(), "{\"type\"=\"max\"}")); + this.metricsList.add(newGaugeSubList(JVM_HEAP_SIZE_BYTES, + jvmStats.getMem().getHeapCommitted().getBytes(), "{\"type\"=\"committed\"}")); + this.metricsList.add(newGaugeSubList(JVM_HEAP_SIZE_BYTES, + jvmStats.getMem().getHeapUsed().getBytes(), "{\"type\"=\"used\"}")); + + // non heap + this.metricsList.add(newGaugeSubList(JVM_NON_HEAP_SIZE_BYTES, + jvmStats.getMem().getNonHeapCommitted().getBytes(), "{\"type\"=\"committed\"}")); + this.metricsList.add(newGaugeSubList(JVM_NON_HEAP_SIZE_BYTES, + jvmStats.getMem().getNonHeapUsed().getBytes(), "{\"type\"=\"used\"}")); + + // mem pool + Iterator<MemoryPool> memIter = jvmStats.getMem().iterator(); + while (memIter.hasNext()) { + MemoryPool memPool = memIter.next(); + if (memPool.getName().equalsIgnoreCase("young")) { + this.metricsList.add(newGaugeSubList(JVM_YOUNG_SIZE_BYTES, + memPool.getUsed().getBytes(), "{\"type\"=\"used\"}")); + this.metricsList.add(newGaugeSubList(JVM_YOUNG_SIZE_BYTES, + memPool.getPeakUsed().getBytes(), "{\"type\"=\"peak_used\"}")); + this.metricsList.add(newGaugeSubList(JVM_YOUNG_SIZE_BYTES, + memPool.getMax().getBytes(), "{\"type\"=\"max\"}")); + } else if (memPool.getName().equalsIgnoreCase("old")) { + this.metricsList.add(newGaugeSubList(JVM_OLD_SIZE_BYTES, + memPool.getUsed().getBytes(), "{\"type\"=\"used\"}")); + this.metricsList.add(newGaugeSubList(JVM_OLD_SIZE_BYTES, + memPool.getPeakUsed().getBytes(), "{\"type\"=\"peak_used\"}")); + this.metricsList.add(newGaugeSubList(JVM_OLD_SIZE_BYTES, + memPool.getMax().getBytes(), "{\"type\"=\"max\"}")); + } + } + + // gc + for (GarbageCollector gc : jvmStats.getGc()) { + this.metricsList.add(newGaugeSubList(JVM_GC, + gc.getCollectionCount(), "{\"name\"=\"" + gc.getName() + "\",\"type\"=\"count\"}")); + this.metricsList.add(newGaugeSubList(JVM_GC, + gc.getCollectionTime().getMillis(), "{\"name\"=\"" + gc.getName() + "\",\"type\"=\"time\"}")); + } + + // threads + Threads threads = jvmStats.getThreads(); + this.metricsList.add(newGaugeSubList(JVM_THREAD, + threads.getCount(), "\"type\"=\"count\"}")); + this.metricsList.add(newGaugeSubList(JVM_THREAD, + threads.getPeakCount(), "\"type\"=\"peak_count\"}")); + this.metricsList.add(newGaugeSubList(JVM_THREAD, + threads.getThreadsNewCount(), "\"type\"=\"new_count\"}")); + this.metricsList.add(newGaugeSubList(JVM_THREAD, + threads.getThreadsRunnableCount(), "\"type\"=\"runnable_count\"}")); + this.metricsList.add(newGaugeSubList(JVM_THREAD, + threads.getThreadsBlockedCount(), "\"type\"=\"blocked_count\"}")); + this.metricsList.add(newGaugeSubList(JVM_THREAD, + threads.getThreadsWaitingCount(), "\"type\"=\"waiting_count\"}")); + this.metricsList.add(newGaugeSubList(JVM_THREAD, + threads.getThreadsTimedWaitingCount(), "\"type\"=\"timed_waiting_count\"}")); + this.metricsList.add(newGaugeSubList(JVM_THREAD, + threads.getThreadsTerminatedCount(), "\"type\"=\"terminated_count\"}")); Review Comment: Missing opening brace in the tag string. The tag should be "{\"type\"=\"peak_count\"}" but it's currently "\"type\"=\"peak_count\"}", which is missing the opening brace. This inconsistency with other tag strings in this file will cause malformed JSON-like tags. ```suggestion threads.getCount(), "{\"type\"=\"count\"}")); this.metricsList.add(newGaugeSubList(JVM_THREAD, threads.getPeakCount(), "{\"type\"=\"peak_count\"}")); this.metricsList.add(newGaugeSubList(JVM_THREAD, threads.getThreadsNewCount(), "{\"type\"=\"new_count\"}")); this.metricsList.add(newGaugeSubList(JVM_THREAD, threads.getThreadsRunnableCount(), "{\"type\"=\"runnable_count\"}")); this.metricsList.add(newGaugeSubList(JVM_THREAD, threads.getThreadsBlockedCount(), "{\"type\"=\"blocked_count\"}")); this.metricsList.add(newGaugeSubList(JVM_THREAD, threads.getThreadsWaitingCount(), "{\"type\"=\"waiting_count\"}")); this.metricsList.add(newGaugeSubList(JVM_THREAD, threads.getThreadsTimedWaitingCount(), "{\"type\"=\"timed_waiting_count\"}")); this.metricsList.add(newGaugeSubList(JVM_THREAD, threads.getThreadsTerminatedCount(), "{\"type\"=\"terminated_count\"}")); ``` ########## be/src/exec/schema_scanner/schema_frontend_metrics_scanner.cpp: ########## @@ -0,0 +1,132 @@ +// 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. + +#include "exec/schema_scanner/schema_frontend_metrics_scanner.h" + +#include <gen_cpp/FrontendService_types.h> + +#include <exception> +#include <vector> + +#include "exec/schema_scanner/schema_helper.h" +#include "runtime/define_primitive_type.h" +#include "runtime/runtime_state.h" +#include "vec/common/string_ref.h" +#include "vec/core/block.h" +#include "vec/data_types/data_type_factory.hpp" + +namespace doris { +#include "common/compile_check_begin.h" + +std::vector<SchemaScanner::ColumnDesc> SchemaFrontendMetricsScanner::_s_frontend_metric_columns = { + // name, type, size, is_null + {"FE", TYPE_VARCHAR, sizeof(StringRef), true}, + {"METRIC_NAME", TYPE_VARCHAR, sizeof(StringRef), true}, + {"METRIC_TYPE", TYPE_VARCHAR, sizeof(StringRef), true}, + {"METRIC_VALUE", TYPE_DOUBLE, sizeof(double), true}, + {"TAG", TYPE_VARCHAR, sizeof(StringRef), true}}; + +SchemaFrontendMetricsScanner::SchemaFrontendMetricsScanner() + : SchemaScanner(_s_frontend_metric_columns, TSchemaTableType::SCH_FE_METRICS) {} + +SchemaFrontendMetricsScanner::~SchemaFrontendMetricsScanner() = default; + +Status SchemaFrontendMetricsScanner::start(RuntimeState* state) { + TFetchFeMetricsRequest request; + + for (const auto& fe_addr : _param->common_param->fe_addr_list) { + TFetchFeMetricsResult tmp_ret; + RETURN_IF_ERROR(SchemaHelper::fetch_frontend_metrics(fe_addr.hostname, fe_addr.port, + request, &tmp_ret)); + + _metrics_list_result.metrics_list.insert(_metrics_list_result.metrics_list.end(), + tmp_ret.metrics_list.begin(), + tmp_ret.metrics_list.end()); + } + + return Status::OK(); +} + +Status SchemaFrontendMetricsScanner::get_next_block_internal(vectorized::Block* block, bool* eos) { + if (!_is_init) { + return Status::InternalError("call this before initial."); + } + if (block == nullptr || eos == nullptr) { + return Status::InternalError("invalid parameter."); + } + + *eos = true; + if (_metrics_list_result.metrics_list.empty()) { + return Status::OK(); + } + + return _fill_block_impl(block); +} + +Status SchemaFrontendMetricsScanner::_fill_block_impl(vectorized::Block* block) { + SCOPED_TIMER(_fill_block_timer); + + const auto& metrics_list = _metrics_list_result.metrics_list; + size_t row_num = metrics_list.size(); + if (row_num == 0) { + return Status::OK(); + } + + for (size_t col_idx = 0; col_idx < _s_frontend_metric_columns.size(); ++col_idx) { + std::vector<StringRef> str_refs(row_num); + std::vector<double> double_vals(row_num); + std::vector<void*> datas(row_num); + std::vector<std::string> column_values( + row_num); // Store the strings to ensure their lifetime + + for (size_t row_idx = 0; row_idx < row_num; ++row_idx) { + const auto& row = metrics_list[row_idx]; + if (row.size() != _s_frontend_metric_columns.size()) { + return Status::InternalError( + "process list meet invalid schema, schema_size={}, input_data_size={}", + _s_frontend_metric_columns.size(), row.size()); + } + + // Fetch and store the column value based on its index + std::string& column_value = + column_values[row_idx]; // Reference to the actual string in the vector + column_value = row[col_idx]; + + if (_s_frontend_metric_columns[col_idx].type == TYPE_DOUBLE) { + try { + double val = !column_value.empty() ? std::stod(column_value) : 0; + double_vals[row_idx] = val; + } catch (const std::exception& e) { + return Status::InternalError( + "process list meet invalid data, column={}, data={}, reason={}", + _s_frontend_metric_columns[col_idx].name, column_value, e.what()); Review Comment: The error message references "process list" which is a copy-paste error from SchemaProcesslistScanner. The error message should reference "frontend metrics" instead to accurately reflect the context of this scanner. ########## fe/fe-core/src/main/java/org/apache/doris/metric/ListMetricVisitor.java: ########## @@ -0,0 +1,274 @@ +// 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.doris.metric; + +import org.apache.doris.catalog.Env; +import org.apache.doris.monitor.jvm.JvmStats; +import org.apache.doris.monitor.jvm.JvmStats.GarbageCollector; +import org.apache.doris.monitor.jvm.JvmStats.MemoryPool; +import org.apache.doris.monitor.jvm.JvmStats.Threads; + +import com.codahale.metrics.Histogram; +import com.codahale.metrics.Snapshot; +import com.google.common.base.Joiner; +import org.apache.logging.log4j.LogManager; +import org.apache.logging.log4j.Logger; + +import java.util.ArrayList; +import java.util.Collections; +import java.util.Iterator; +import java.util.List; +import java.util.Map; +import java.util.stream.Collectors; + +/* + * Like this: + * [ + * ['FE', 'METRIC_NAME', 'METRIC_TYPE', 'METRIC_VALUE', 'TAG'], + * [] + * ] + */ +public class ListMetricVisitor extends MetricVisitor { + private static final Logger logger = LogManager.getLogger(ListMetricVisitor.class); + + private List<List<String>> metricsList; + private String localHostAddr; + private StringBuilder strConcat; + + // jvm + private static final String JVM_HEAP_SIZE_BYTES = "jvm_heap_size_bytes"; + private static final String JVM_NON_HEAP_SIZE_BYTES = "jvm_non_heap_size_bytes"; + private static final String JVM_YOUNG_SIZE_BYTES = "jvm_young_size_bytes"; + private static final String JVM_OLD_SIZE_BYTES = "jvm_old_size_bytes"; + private static final String JVM_THREAD = "jvm_thread"; + + private static final String JVM_GC = "jvm_gc"; + private static final String TYPE_GAUGE = "gauge"; + private static final String TYPE_COUNTER = "counter"; + private static final String TYPE_HISTOGRAM = "histogram"; + + public ListMetricVisitor(List<List<String>> metricsList, String localHostAddr) { + super(); + // List[ List[FE, METRIC_NAME, METRIC_TYPE, METRIC_VALUE, TAG] ] + this.metricsList = metricsList; + this.localHostAddr = localHostAddr; + this.strConcat = new StringBuilder(); + } + + @Override + public void visitJvm(JvmStats jvmStats) { + // heap + this.metricsList.add(newGaugeSubList(JVM_HEAP_SIZE_BYTES, + jvmStats.getMem().getHeapMax().getBytes(), "{\"type\"=\"max\"}")); + this.metricsList.add(newGaugeSubList(JVM_HEAP_SIZE_BYTES, + jvmStats.getMem().getHeapCommitted().getBytes(), "{\"type\"=\"committed\"}")); + this.metricsList.add(newGaugeSubList(JVM_HEAP_SIZE_BYTES, + jvmStats.getMem().getHeapUsed().getBytes(), "{\"type\"=\"used\"}")); + + // non heap + this.metricsList.add(newGaugeSubList(JVM_NON_HEAP_SIZE_BYTES, + jvmStats.getMem().getNonHeapCommitted().getBytes(), "{\"type\"=\"committed\"}")); + this.metricsList.add(newGaugeSubList(JVM_NON_HEAP_SIZE_BYTES, + jvmStats.getMem().getNonHeapUsed().getBytes(), "{\"type\"=\"used\"}")); + + // mem pool + Iterator<MemoryPool> memIter = jvmStats.getMem().iterator(); + while (memIter.hasNext()) { + MemoryPool memPool = memIter.next(); + if (memPool.getName().equalsIgnoreCase("young")) { + this.metricsList.add(newGaugeSubList(JVM_YOUNG_SIZE_BYTES, + memPool.getUsed().getBytes(), "{\"type\"=\"used\"}")); + this.metricsList.add(newGaugeSubList(JVM_YOUNG_SIZE_BYTES, + memPool.getPeakUsed().getBytes(), "{\"type\"=\"peak_used\"}")); + this.metricsList.add(newGaugeSubList(JVM_YOUNG_SIZE_BYTES, + memPool.getMax().getBytes(), "{\"type\"=\"max\"}")); + } else if (memPool.getName().equalsIgnoreCase("old")) { + this.metricsList.add(newGaugeSubList(JVM_OLD_SIZE_BYTES, + memPool.getUsed().getBytes(), "{\"type\"=\"used\"}")); + this.metricsList.add(newGaugeSubList(JVM_OLD_SIZE_BYTES, + memPool.getPeakUsed().getBytes(), "{\"type\"=\"peak_used\"}")); + this.metricsList.add(newGaugeSubList(JVM_OLD_SIZE_BYTES, + memPool.getMax().getBytes(), "{\"type\"=\"max\"}")); + } + } + + // gc + for (GarbageCollector gc : jvmStats.getGc()) { + this.metricsList.add(newGaugeSubList(JVM_GC, + gc.getCollectionCount(), "{\"name\"=\"" + gc.getName() + "\",\"type\"=\"count\"}")); + this.metricsList.add(newGaugeSubList(JVM_GC, + gc.getCollectionTime().getMillis(), "{\"name\"=\"" + gc.getName() + "\",\"type\"=\"time\"}")); + } + + // threads + Threads threads = jvmStats.getThreads(); + this.metricsList.add(newGaugeSubList(JVM_THREAD, + threads.getCount(), "\"type\"=\"count\"}")); + this.metricsList.add(newGaugeSubList(JVM_THREAD, + threads.getPeakCount(), "\"type\"=\"peak_count\"}")); + this.metricsList.add(newGaugeSubList(JVM_THREAD, + threads.getThreadsNewCount(), "\"type\"=\"new_count\"}")); + this.metricsList.add(newGaugeSubList(JVM_THREAD, + threads.getThreadsRunnableCount(), "\"type\"=\"runnable_count\"}")); + this.metricsList.add(newGaugeSubList(JVM_THREAD, + threads.getThreadsBlockedCount(), "\"type\"=\"blocked_count\"}")); + this.metricsList.add(newGaugeSubList(JVM_THREAD, + threads.getThreadsWaitingCount(), "\"type\"=\"waiting_count\"}")); + this.metricsList.add(newGaugeSubList(JVM_THREAD, + threads.getThreadsTimedWaitingCount(), "\"type\"=\"timed_waiting_count\"}")); + this.metricsList.add(newGaugeSubList(JVM_THREAD, + threads.getThreadsTerminatedCount(), "\"type\"=\"terminated_count\"}")); + } + + @Override + public void visit(String prefix, @SuppressWarnings("rawtypes") Metric metric) { + // title + final String fullName = prefix + metric.getName(); + // name + strConcat.setLength(0); + List<MetricLabel> labels = metric.getLabels(); + if (!labels.isEmpty()) { + strConcat.append("{"); + List<String> labelStrs = labels.stream().map(l -> "\"" + l.getKey() + "\"" + "=\"" + l.getValue() + + "\"").collect(Collectors.toList()); + strConcat.append(Joiner.on(", ").join(labelStrs)); + strConcat.append("}"); + } + // value + List<String> metricStr = + newSubListByType(fullName, metric.getValue().toString(), strConcat.toString(), metric.getType()); + if (metricStr != null) { + this.metricsList.add(metricStr); + } + } + + @Override + public void visitHistogram(String prefix, String name, Histogram histogram) { + // part.part.part.k1=v1.k2=v2 + List<String> names = new ArrayList<>(); + List<String> tags = new ArrayList<>(); + for (String part : name.split("\\.")) { + String[] kv = part.split("="); + if (kv.length == 1) { + names.add(kv[0]); + } else if (kv.length == 2) { + tags.add(String.format("\"%s\"=\"%s\"", kv[0], kv[1])); + } + } + final String fullName = prefix + String.join("_", names); + final String fullTag = String.join(",", tags); + String delimiter = tags.isEmpty() ? "" : ","; + Snapshot snapshot = histogram.getSnapshot(); + + this.metricsList.add(newHistogramSubList(fullName, + snapshot.get75thPercentile(), "{\"quantile\"=\"0.75\"" + delimiter + fullTag + "}")); + this.metricsList.add(newHistogramSubList(fullName, + snapshot.get95thPercentile(), "{\"quantile\"=\"0.95\"" + delimiter + fullTag + "}")); + this.metricsList.add(newHistogramSubList(fullName, + snapshot.get98thPercentile(), "{\"quantile\"=\"0.98\"" + delimiter + fullTag + "}")); + this.metricsList.add(newHistogramSubList(fullName, + snapshot.get99thPercentile(), "{\"quantile\"=\"0.99\"" + delimiter + fullTag + "}")); + this.metricsList.add(newHistogramSubList(fullName, + snapshot.get999thPercentile(), "{\"quantile\"=\"0.999\"" + delimiter + fullTag + "}")); + this.metricsList.add(newHistogramSubList(fullName + "_sum", + histogram.getCount() * snapshot.getMean(), + "{\"quantile\"=\"0.75\"" + delimiter + fullTag + "}")); + this.metricsList.add(newHistogramSubList(fullName + "_count", + histogram.getCount(), "{\"quantile\"=\"0.75\"" + delimiter + fullTag + "}")); Review Comment: The tag used for the "_count" metric should not include the "quantile" label, as count is not associated with a quantile. Additionally, using "{\"quantile\"=\"0.75\"" is misleading since this represents the total count, not the 75th percentile. Consider removing the quantile label entirely or using an empty tag set for the count metric. ########## fe/fe-core/src/main/java/org/apache/doris/metric/ListMetricVisitor.java: ########## @@ -0,0 +1,274 @@ +// 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.doris.metric; + +import org.apache.doris.catalog.Env; +import org.apache.doris.monitor.jvm.JvmStats; +import org.apache.doris.monitor.jvm.JvmStats.GarbageCollector; +import org.apache.doris.monitor.jvm.JvmStats.MemoryPool; +import org.apache.doris.monitor.jvm.JvmStats.Threads; + +import com.codahale.metrics.Histogram; +import com.codahale.metrics.Snapshot; +import com.google.common.base.Joiner; +import org.apache.logging.log4j.LogManager; +import org.apache.logging.log4j.Logger; + +import java.util.ArrayList; +import java.util.Collections; +import java.util.Iterator; +import java.util.List; +import java.util.Map; +import java.util.stream.Collectors; + +/* + * Like this: + * [ + * ['FE', 'METRIC_NAME', 'METRIC_TYPE', 'METRIC_VALUE', 'TAG'], + * [] + * ] + */ +public class ListMetricVisitor extends MetricVisitor { + private static final Logger logger = LogManager.getLogger(ListMetricVisitor.class); + + private List<List<String>> metricsList; + private String localHostAddr; + private StringBuilder strConcat; + + // jvm + private static final String JVM_HEAP_SIZE_BYTES = "jvm_heap_size_bytes"; + private static final String JVM_NON_HEAP_SIZE_BYTES = "jvm_non_heap_size_bytes"; + private static final String JVM_YOUNG_SIZE_BYTES = "jvm_young_size_bytes"; + private static final String JVM_OLD_SIZE_BYTES = "jvm_old_size_bytes"; + private static final String JVM_THREAD = "jvm_thread"; + + private static final String JVM_GC = "jvm_gc"; + private static final String TYPE_GAUGE = "gauge"; + private static final String TYPE_COUNTER = "counter"; + private static final String TYPE_HISTOGRAM = "histogram"; + + public ListMetricVisitor(List<List<String>> metricsList, String localHostAddr) { + super(); + // List[ List[FE, METRIC_NAME, METRIC_TYPE, METRIC_VALUE, TAG] ] + this.metricsList = metricsList; + this.localHostAddr = localHostAddr; + this.strConcat = new StringBuilder(); + } + + @Override + public void visitJvm(JvmStats jvmStats) { + // heap + this.metricsList.add(newGaugeSubList(JVM_HEAP_SIZE_BYTES, + jvmStats.getMem().getHeapMax().getBytes(), "{\"type\"=\"max\"}")); + this.metricsList.add(newGaugeSubList(JVM_HEAP_SIZE_BYTES, + jvmStats.getMem().getHeapCommitted().getBytes(), "{\"type\"=\"committed\"}")); + this.metricsList.add(newGaugeSubList(JVM_HEAP_SIZE_BYTES, + jvmStats.getMem().getHeapUsed().getBytes(), "{\"type\"=\"used\"}")); + + // non heap + this.metricsList.add(newGaugeSubList(JVM_NON_HEAP_SIZE_BYTES, + jvmStats.getMem().getNonHeapCommitted().getBytes(), "{\"type\"=\"committed\"}")); + this.metricsList.add(newGaugeSubList(JVM_NON_HEAP_SIZE_BYTES, + jvmStats.getMem().getNonHeapUsed().getBytes(), "{\"type\"=\"used\"}")); + + // mem pool + Iterator<MemoryPool> memIter = jvmStats.getMem().iterator(); + while (memIter.hasNext()) { + MemoryPool memPool = memIter.next(); + if (memPool.getName().equalsIgnoreCase("young")) { + this.metricsList.add(newGaugeSubList(JVM_YOUNG_SIZE_BYTES, + memPool.getUsed().getBytes(), "{\"type\"=\"used\"}")); + this.metricsList.add(newGaugeSubList(JVM_YOUNG_SIZE_BYTES, + memPool.getPeakUsed().getBytes(), "{\"type\"=\"peak_used\"}")); + this.metricsList.add(newGaugeSubList(JVM_YOUNG_SIZE_BYTES, + memPool.getMax().getBytes(), "{\"type\"=\"max\"}")); + } else if (memPool.getName().equalsIgnoreCase("old")) { + this.metricsList.add(newGaugeSubList(JVM_OLD_SIZE_BYTES, + memPool.getUsed().getBytes(), "{\"type\"=\"used\"}")); + this.metricsList.add(newGaugeSubList(JVM_OLD_SIZE_BYTES, + memPool.getPeakUsed().getBytes(), "{\"type\"=\"peak_used\"}")); + this.metricsList.add(newGaugeSubList(JVM_OLD_SIZE_BYTES, + memPool.getMax().getBytes(), "{\"type\"=\"max\"}")); + } + } + + // gc + for (GarbageCollector gc : jvmStats.getGc()) { + this.metricsList.add(newGaugeSubList(JVM_GC, + gc.getCollectionCount(), "{\"name\"=\"" + gc.getName() + "\",\"type\"=\"count\"}")); + this.metricsList.add(newGaugeSubList(JVM_GC, + gc.getCollectionTime().getMillis(), "{\"name\"=\"" + gc.getName() + "\",\"type\"=\"time\"}")); + } + + // threads + Threads threads = jvmStats.getThreads(); + this.metricsList.add(newGaugeSubList(JVM_THREAD, + threads.getCount(), "\"type\"=\"count\"}")); + this.metricsList.add(newGaugeSubList(JVM_THREAD, + threads.getPeakCount(), "\"type\"=\"peak_count\"}")); + this.metricsList.add(newGaugeSubList(JVM_THREAD, + threads.getThreadsNewCount(), "\"type\"=\"new_count\"}")); + this.metricsList.add(newGaugeSubList(JVM_THREAD, + threads.getThreadsRunnableCount(), "\"type\"=\"runnable_count\"}")); + this.metricsList.add(newGaugeSubList(JVM_THREAD, + threads.getThreadsBlockedCount(), "\"type\"=\"blocked_count\"}")); + this.metricsList.add(newGaugeSubList(JVM_THREAD, + threads.getThreadsWaitingCount(), "\"type\"=\"waiting_count\"}")); + this.metricsList.add(newGaugeSubList(JVM_THREAD, + threads.getThreadsTimedWaitingCount(), "\"type\"=\"timed_waiting_count\"}")); + this.metricsList.add(newGaugeSubList(JVM_THREAD, + threads.getThreadsTerminatedCount(), "\"type\"=\"terminated_count\"}")); Review Comment: Missing opening brace in the tag string. The tag should be "{\"type\"=\"blocked_count\"}" but it's currently "\"type\"=\"blocked_count\"}", which is missing the opening brace. This inconsistency with other tag strings in this file will cause malformed JSON-like tags. ```suggestion threads.getCount(), "{\"type\"=\"count\"}")); this.metricsList.add(newGaugeSubList(JVM_THREAD, threads.getPeakCount(), "{\"type\"=\"peak_count\"}")); this.metricsList.add(newGaugeSubList(JVM_THREAD, threads.getThreadsNewCount(), "{\"type\"=\"new_count\"}")); this.metricsList.add(newGaugeSubList(JVM_THREAD, threads.getThreadsRunnableCount(), "{\"type\"=\"runnable_count\"}")); this.metricsList.add(newGaugeSubList(JVM_THREAD, threads.getThreadsBlockedCount(), "{\"type\"=\"blocked_count\"}")); this.metricsList.add(newGaugeSubList(JVM_THREAD, threads.getThreadsWaitingCount(), "{\"type\"=\"waiting_count\"}")); this.metricsList.add(newGaugeSubList(JVM_THREAD, threads.getThreadsTimedWaitingCount(), "{\"type\"=\"timed_waiting_count\"}")); this.metricsList.add(newGaugeSubList(JVM_THREAD, threads.getThreadsTerminatedCount(), "{\"type\"=\"terminated_count\"}")); ``` ########## fe/fe-core/src/main/java/org/apache/doris/metric/ListMetricVisitor.java: ########## @@ -0,0 +1,274 @@ +// 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.doris.metric; + +import org.apache.doris.catalog.Env; +import org.apache.doris.monitor.jvm.JvmStats; +import org.apache.doris.monitor.jvm.JvmStats.GarbageCollector; +import org.apache.doris.monitor.jvm.JvmStats.MemoryPool; +import org.apache.doris.monitor.jvm.JvmStats.Threads; + +import com.codahale.metrics.Histogram; +import com.codahale.metrics.Snapshot; +import com.google.common.base.Joiner; +import org.apache.logging.log4j.LogManager; +import org.apache.logging.log4j.Logger; + +import java.util.ArrayList; +import java.util.Collections; +import java.util.Iterator; +import java.util.List; +import java.util.Map; +import java.util.stream.Collectors; + +/* + * Like this: + * [ + * ['FE', 'METRIC_NAME', 'METRIC_TYPE', 'METRIC_VALUE', 'TAG'], + * [] + * ] + */ +public class ListMetricVisitor extends MetricVisitor { + private static final Logger logger = LogManager.getLogger(ListMetricVisitor.class); + + private List<List<String>> metricsList; + private String localHostAddr; + private StringBuilder strConcat; + + // jvm + private static final String JVM_HEAP_SIZE_BYTES = "jvm_heap_size_bytes"; + private static final String JVM_NON_HEAP_SIZE_BYTES = "jvm_non_heap_size_bytes"; + private static final String JVM_YOUNG_SIZE_BYTES = "jvm_young_size_bytes"; + private static final String JVM_OLD_SIZE_BYTES = "jvm_old_size_bytes"; + private static final String JVM_THREAD = "jvm_thread"; + + private static final String JVM_GC = "jvm_gc"; + private static final String TYPE_GAUGE = "gauge"; + private static final String TYPE_COUNTER = "counter"; + private static final String TYPE_HISTOGRAM = "histogram"; + + public ListMetricVisitor(List<List<String>> metricsList, String localHostAddr) { + super(); + // List[ List[FE, METRIC_NAME, METRIC_TYPE, METRIC_VALUE, TAG] ] + this.metricsList = metricsList; + this.localHostAddr = localHostAddr; + this.strConcat = new StringBuilder(); + } + + @Override + public void visitJvm(JvmStats jvmStats) { + // heap + this.metricsList.add(newGaugeSubList(JVM_HEAP_SIZE_BYTES, + jvmStats.getMem().getHeapMax().getBytes(), "{\"type\"=\"max\"}")); + this.metricsList.add(newGaugeSubList(JVM_HEAP_SIZE_BYTES, + jvmStats.getMem().getHeapCommitted().getBytes(), "{\"type\"=\"committed\"}")); + this.metricsList.add(newGaugeSubList(JVM_HEAP_SIZE_BYTES, + jvmStats.getMem().getHeapUsed().getBytes(), "{\"type\"=\"used\"}")); + + // non heap + this.metricsList.add(newGaugeSubList(JVM_NON_HEAP_SIZE_BYTES, + jvmStats.getMem().getNonHeapCommitted().getBytes(), "{\"type\"=\"committed\"}")); + this.metricsList.add(newGaugeSubList(JVM_NON_HEAP_SIZE_BYTES, + jvmStats.getMem().getNonHeapUsed().getBytes(), "{\"type\"=\"used\"}")); + + // mem pool + Iterator<MemoryPool> memIter = jvmStats.getMem().iterator(); + while (memIter.hasNext()) { + MemoryPool memPool = memIter.next(); + if (memPool.getName().equalsIgnoreCase("young")) { + this.metricsList.add(newGaugeSubList(JVM_YOUNG_SIZE_BYTES, + memPool.getUsed().getBytes(), "{\"type\"=\"used\"}")); + this.metricsList.add(newGaugeSubList(JVM_YOUNG_SIZE_BYTES, + memPool.getPeakUsed().getBytes(), "{\"type\"=\"peak_used\"}")); + this.metricsList.add(newGaugeSubList(JVM_YOUNG_SIZE_BYTES, + memPool.getMax().getBytes(), "{\"type\"=\"max\"}")); + } else if (memPool.getName().equalsIgnoreCase("old")) { + this.metricsList.add(newGaugeSubList(JVM_OLD_SIZE_BYTES, + memPool.getUsed().getBytes(), "{\"type\"=\"used\"}")); + this.metricsList.add(newGaugeSubList(JVM_OLD_SIZE_BYTES, + memPool.getPeakUsed().getBytes(), "{\"type\"=\"peak_used\"}")); + this.metricsList.add(newGaugeSubList(JVM_OLD_SIZE_BYTES, + memPool.getMax().getBytes(), "{\"type\"=\"max\"}")); + } + } + + // gc + for (GarbageCollector gc : jvmStats.getGc()) { + this.metricsList.add(newGaugeSubList(JVM_GC, + gc.getCollectionCount(), "{\"name\"=\"" + gc.getName() + "\",\"type\"=\"count\"}")); + this.metricsList.add(newGaugeSubList(JVM_GC, + gc.getCollectionTime().getMillis(), "{\"name\"=\"" + gc.getName() + "\",\"type\"=\"time\"}")); + } + + // threads + Threads threads = jvmStats.getThreads(); + this.metricsList.add(newGaugeSubList(JVM_THREAD, + threads.getCount(), "\"type\"=\"count\"}")); + this.metricsList.add(newGaugeSubList(JVM_THREAD, + threads.getPeakCount(), "\"type\"=\"peak_count\"}")); + this.metricsList.add(newGaugeSubList(JVM_THREAD, + threads.getThreadsNewCount(), "\"type\"=\"new_count\"}")); + this.metricsList.add(newGaugeSubList(JVM_THREAD, + threads.getThreadsRunnableCount(), "\"type\"=\"runnable_count\"}")); + this.metricsList.add(newGaugeSubList(JVM_THREAD, + threads.getThreadsBlockedCount(), "\"type\"=\"blocked_count\"}")); + this.metricsList.add(newGaugeSubList(JVM_THREAD, + threads.getThreadsWaitingCount(), "\"type\"=\"waiting_count\"}")); + this.metricsList.add(newGaugeSubList(JVM_THREAD, + threads.getThreadsTimedWaitingCount(), "\"type\"=\"timed_waiting_count\"}")); + this.metricsList.add(newGaugeSubList(JVM_THREAD, + threads.getThreadsTerminatedCount(), "\"type\"=\"terminated_count\"}")); Review Comment: Missing opening brace in the tag string. The tag should be "{\"type\"=\"count\"}" but it's currently "\"type\"=\"count\"}", which is missing the opening brace. This inconsistency with other tag strings in this file will cause malformed JSON-like tags. ```suggestion threads.getCount(), "{\"type\"=\"count\"}")); this.metricsList.add(newGaugeSubList(JVM_THREAD, threads.getPeakCount(), "{\"type\"=\"peak_count\"}")); this.metricsList.add(newGaugeSubList(JVM_THREAD, threads.getThreadsNewCount(), "{\"type\"=\"new_count\"}")); this.metricsList.add(newGaugeSubList(JVM_THREAD, threads.getThreadsRunnableCount(), "{\"type\"=\"runnable_count\"}")); this.metricsList.add(newGaugeSubList(JVM_THREAD, threads.getThreadsBlockedCount(), "{\"type\"=\"blocked_count\"}")); this.metricsList.add(newGaugeSubList(JVM_THREAD, threads.getThreadsWaitingCount(), "{\"type\"=\"waiting_count\"}")); this.metricsList.add(newGaugeSubList(JVM_THREAD, threads.getThreadsTimedWaitingCount(), "{\"type\"=\"timed_waiting_count\"}")); this.metricsList.add(newGaugeSubList(JVM_THREAD, threads.getThreadsTerminatedCount(), "{\"type\"=\"terminated_count\"}")); ``` ########## fe/fe-core/src/main/java/org/apache/doris/metric/ListMetricVisitor.java: ########## @@ -0,0 +1,274 @@ +// 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.doris.metric; + +import org.apache.doris.catalog.Env; +import org.apache.doris.monitor.jvm.JvmStats; +import org.apache.doris.monitor.jvm.JvmStats.GarbageCollector; +import org.apache.doris.monitor.jvm.JvmStats.MemoryPool; +import org.apache.doris.monitor.jvm.JvmStats.Threads; + +import com.codahale.metrics.Histogram; +import com.codahale.metrics.Snapshot; +import com.google.common.base.Joiner; +import org.apache.logging.log4j.LogManager; +import org.apache.logging.log4j.Logger; + +import java.util.ArrayList; +import java.util.Collections; +import java.util.Iterator; +import java.util.List; +import java.util.Map; +import java.util.stream.Collectors; + +/* + * Like this: + * [ + * ['FE', 'METRIC_NAME', 'METRIC_TYPE', 'METRIC_VALUE', 'TAG'], + * [] + * ] + */ +public class ListMetricVisitor extends MetricVisitor { + private static final Logger logger = LogManager.getLogger(ListMetricVisitor.class); + + private List<List<String>> metricsList; + private String localHostAddr; + private StringBuilder strConcat; + + // jvm + private static final String JVM_HEAP_SIZE_BYTES = "jvm_heap_size_bytes"; + private static final String JVM_NON_HEAP_SIZE_BYTES = "jvm_non_heap_size_bytes"; + private static final String JVM_YOUNG_SIZE_BYTES = "jvm_young_size_bytes"; + private static final String JVM_OLD_SIZE_BYTES = "jvm_old_size_bytes"; + private static final String JVM_THREAD = "jvm_thread"; + + private static final String JVM_GC = "jvm_gc"; + private static final String TYPE_GAUGE = "gauge"; + private static final String TYPE_COUNTER = "counter"; + private static final String TYPE_HISTOGRAM = "histogram"; + + public ListMetricVisitor(List<List<String>> metricsList, String localHostAddr) { + super(); + // List[ List[FE, METRIC_NAME, METRIC_TYPE, METRIC_VALUE, TAG] ] + this.metricsList = metricsList; + this.localHostAddr = localHostAddr; + this.strConcat = new StringBuilder(); + } + + @Override + public void visitJvm(JvmStats jvmStats) { + // heap + this.metricsList.add(newGaugeSubList(JVM_HEAP_SIZE_BYTES, + jvmStats.getMem().getHeapMax().getBytes(), "{\"type\"=\"max\"}")); + this.metricsList.add(newGaugeSubList(JVM_HEAP_SIZE_BYTES, + jvmStats.getMem().getHeapCommitted().getBytes(), "{\"type\"=\"committed\"}")); + this.metricsList.add(newGaugeSubList(JVM_HEAP_SIZE_BYTES, + jvmStats.getMem().getHeapUsed().getBytes(), "{\"type\"=\"used\"}")); + + // non heap + this.metricsList.add(newGaugeSubList(JVM_NON_HEAP_SIZE_BYTES, + jvmStats.getMem().getNonHeapCommitted().getBytes(), "{\"type\"=\"committed\"}")); + this.metricsList.add(newGaugeSubList(JVM_NON_HEAP_SIZE_BYTES, + jvmStats.getMem().getNonHeapUsed().getBytes(), "{\"type\"=\"used\"}")); + + // mem pool + Iterator<MemoryPool> memIter = jvmStats.getMem().iterator(); + while (memIter.hasNext()) { + MemoryPool memPool = memIter.next(); + if (memPool.getName().equalsIgnoreCase("young")) { + this.metricsList.add(newGaugeSubList(JVM_YOUNG_SIZE_BYTES, + memPool.getUsed().getBytes(), "{\"type\"=\"used\"}")); + this.metricsList.add(newGaugeSubList(JVM_YOUNG_SIZE_BYTES, + memPool.getPeakUsed().getBytes(), "{\"type\"=\"peak_used\"}")); + this.metricsList.add(newGaugeSubList(JVM_YOUNG_SIZE_BYTES, + memPool.getMax().getBytes(), "{\"type\"=\"max\"}")); + } else if (memPool.getName().equalsIgnoreCase("old")) { + this.metricsList.add(newGaugeSubList(JVM_OLD_SIZE_BYTES, + memPool.getUsed().getBytes(), "{\"type\"=\"used\"}")); + this.metricsList.add(newGaugeSubList(JVM_OLD_SIZE_BYTES, + memPool.getPeakUsed().getBytes(), "{\"type\"=\"peak_used\"}")); + this.metricsList.add(newGaugeSubList(JVM_OLD_SIZE_BYTES, + memPool.getMax().getBytes(), "{\"type\"=\"max\"}")); + } + } + + // gc + for (GarbageCollector gc : jvmStats.getGc()) { + this.metricsList.add(newGaugeSubList(JVM_GC, + gc.getCollectionCount(), "{\"name\"=\"" + gc.getName() + "\",\"type\"=\"count\"}")); + this.metricsList.add(newGaugeSubList(JVM_GC, + gc.getCollectionTime().getMillis(), "{\"name\"=\"" + gc.getName() + "\",\"type\"=\"time\"}")); + } + + // threads + Threads threads = jvmStats.getThreads(); + this.metricsList.add(newGaugeSubList(JVM_THREAD, + threads.getCount(), "\"type\"=\"count\"}")); + this.metricsList.add(newGaugeSubList(JVM_THREAD, + threads.getPeakCount(), "\"type\"=\"peak_count\"}")); + this.metricsList.add(newGaugeSubList(JVM_THREAD, + threads.getThreadsNewCount(), "\"type\"=\"new_count\"}")); + this.metricsList.add(newGaugeSubList(JVM_THREAD, + threads.getThreadsRunnableCount(), "\"type\"=\"runnable_count\"}")); + this.metricsList.add(newGaugeSubList(JVM_THREAD, + threads.getThreadsBlockedCount(), "\"type\"=\"blocked_count\"}")); + this.metricsList.add(newGaugeSubList(JVM_THREAD, + threads.getThreadsWaitingCount(), "\"type\"=\"waiting_count\"}")); + this.metricsList.add(newGaugeSubList(JVM_THREAD, + threads.getThreadsTimedWaitingCount(), "\"type\"=\"timed_waiting_count\"}")); + this.metricsList.add(newGaugeSubList(JVM_THREAD, + threads.getThreadsTerminatedCount(), "\"type\"=\"terminated_count\"}")); Review Comment: Missing opening brace in the tag string. The tag should be "{\"type\"=\"terminated_count\"}" but it's currently "\"type\"=\"terminated_count\"}", which is missing the opening brace. This inconsistency with other tag strings in this file will cause malformed JSON-like tags. ```suggestion threads.getCount(), "{\"type\"=\"count\"}")); this.metricsList.add(newGaugeSubList(JVM_THREAD, threads.getPeakCount(), "{\"type\"=\"peak_count\"}")); this.metricsList.add(newGaugeSubList(JVM_THREAD, threads.getThreadsNewCount(), "{\"type\"=\"new_count\"}")); this.metricsList.add(newGaugeSubList(JVM_THREAD, threads.getThreadsRunnableCount(), "{\"type\"=\"runnable_count\"}")); this.metricsList.add(newGaugeSubList(JVM_THREAD, threads.getThreadsBlockedCount(), "{\"type\"=\"blocked_count\"}")); this.metricsList.add(newGaugeSubList(JVM_THREAD, threads.getThreadsWaitingCount(), "{\"type\"=\"waiting_count\"}")); this.metricsList.add(newGaugeSubList(JVM_THREAD, threads.getThreadsTimedWaitingCount(), "{\"type\"=\"timed_waiting_count\"}")); this.metricsList.add(newGaugeSubList(JVM_THREAD, threads.getThreadsTerminatedCount(), "{\"type\"=\"terminated_count\"}")); ``` ########## fe/fe-core/src/main/java/org/apache/doris/metric/ListMetricVisitor.java: ########## @@ -0,0 +1,274 @@ +// 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.doris.metric; + +import org.apache.doris.catalog.Env; +import org.apache.doris.monitor.jvm.JvmStats; +import org.apache.doris.monitor.jvm.JvmStats.GarbageCollector; +import org.apache.doris.monitor.jvm.JvmStats.MemoryPool; +import org.apache.doris.monitor.jvm.JvmStats.Threads; + +import com.codahale.metrics.Histogram; +import com.codahale.metrics.Snapshot; +import com.google.common.base.Joiner; +import org.apache.logging.log4j.LogManager; +import org.apache.logging.log4j.Logger; + +import java.util.ArrayList; +import java.util.Collections; +import java.util.Iterator; +import java.util.List; +import java.util.Map; +import java.util.stream.Collectors; + +/* + * Like this: + * [ + * ['FE', 'METRIC_NAME', 'METRIC_TYPE', 'METRIC_VALUE', 'TAG'], + * [] + * ] + */ +public class ListMetricVisitor extends MetricVisitor { + private static final Logger logger = LogManager.getLogger(ListMetricVisitor.class); + + private List<List<String>> metricsList; + private String localHostAddr; + private StringBuilder strConcat; + + // jvm + private static final String JVM_HEAP_SIZE_BYTES = "jvm_heap_size_bytes"; + private static final String JVM_NON_HEAP_SIZE_BYTES = "jvm_non_heap_size_bytes"; + private static final String JVM_YOUNG_SIZE_BYTES = "jvm_young_size_bytes"; + private static final String JVM_OLD_SIZE_BYTES = "jvm_old_size_bytes"; + private static final String JVM_THREAD = "jvm_thread"; + + private static final String JVM_GC = "jvm_gc"; + private static final String TYPE_GAUGE = "gauge"; + private static final String TYPE_COUNTER = "counter"; + private static final String TYPE_HISTOGRAM = "histogram"; + + public ListMetricVisitor(List<List<String>> metricsList, String localHostAddr) { + super(); + // List[ List[FE, METRIC_NAME, METRIC_TYPE, METRIC_VALUE, TAG] ] + this.metricsList = metricsList; + this.localHostAddr = localHostAddr; + this.strConcat = new StringBuilder(); + } + + @Override + public void visitJvm(JvmStats jvmStats) { + // heap + this.metricsList.add(newGaugeSubList(JVM_HEAP_SIZE_BYTES, + jvmStats.getMem().getHeapMax().getBytes(), "{\"type\"=\"max\"}")); + this.metricsList.add(newGaugeSubList(JVM_HEAP_SIZE_BYTES, + jvmStats.getMem().getHeapCommitted().getBytes(), "{\"type\"=\"committed\"}")); + this.metricsList.add(newGaugeSubList(JVM_HEAP_SIZE_BYTES, + jvmStats.getMem().getHeapUsed().getBytes(), "{\"type\"=\"used\"}")); + + // non heap + this.metricsList.add(newGaugeSubList(JVM_NON_HEAP_SIZE_BYTES, + jvmStats.getMem().getNonHeapCommitted().getBytes(), "{\"type\"=\"committed\"}")); + this.metricsList.add(newGaugeSubList(JVM_NON_HEAP_SIZE_BYTES, + jvmStats.getMem().getNonHeapUsed().getBytes(), "{\"type\"=\"used\"}")); + + // mem pool + Iterator<MemoryPool> memIter = jvmStats.getMem().iterator(); + while (memIter.hasNext()) { + MemoryPool memPool = memIter.next(); + if (memPool.getName().equalsIgnoreCase("young")) { + this.metricsList.add(newGaugeSubList(JVM_YOUNG_SIZE_BYTES, + memPool.getUsed().getBytes(), "{\"type\"=\"used\"}")); + this.metricsList.add(newGaugeSubList(JVM_YOUNG_SIZE_BYTES, + memPool.getPeakUsed().getBytes(), "{\"type\"=\"peak_used\"}")); + this.metricsList.add(newGaugeSubList(JVM_YOUNG_SIZE_BYTES, + memPool.getMax().getBytes(), "{\"type\"=\"max\"}")); + } else if (memPool.getName().equalsIgnoreCase("old")) { + this.metricsList.add(newGaugeSubList(JVM_OLD_SIZE_BYTES, + memPool.getUsed().getBytes(), "{\"type\"=\"used\"}")); + this.metricsList.add(newGaugeSubList(JVM_OLD_SIZE_BYTES, + memPool.getPeakUsed().getBytes(), "{\"type\"=\"peak_used\"}")); + this.metricsList.add(newGaugeSubList(JVM_OLD_SIZE_BYTES, + memPool.getMax().getBytes(), "{\"type\"=\"max\"}")); + } + } + + // gc + for (GarbageCollector gc : jvmStats.getGc()) { + this.metricsList.add(newGaugeSubList(JVM_GC, + gc.getCollectionCount(), "{\"name\"=\"" + gc.getName() + "\",\"type\"=\"count\"}")); + this.metricsList.add(newGaugeSubList(JVM_GC, + gc.getCollectionTime().getMillis(), "{\"name\"=\"" + gc.getName() + "\",\"type\"=\"time\"}")); + } + + // threads + Threads threads = jvmStats.getThreads(); + this.metricsList.add(newGaugeSubList(JVM_THREAD, + threads.getCount(), "\"type\"=\"count\"}")); + this.metricsList.add(newGaugeSubList(JVM_THREAD, + threads.getPeakCount(), "\"type\"=\"peak_count\"}")); + this.metricsList.add(newGaugeSubList(JVM_THREAD, + threads.getThreadsNewCount(), "\"type\"=\"new_count\"}")); + this.metricsList.add(newGaugeSubList(JVM_THREAD, + threads.getThreadsRunnableCount(), "\"type\"=\"runnable_count\"}")); + this.metricsList.add(newGaugeSubList(JVM_THREAD, + threads.getThreadsBlockedCount(), "\"type\"=\"blocked_count\"}")); + this.metricsList.add(newGaugeSubList(JVM_THREAD, + threads.getThreadsWaitingCount(), "\"type\"=\"waiting_count\"}")); + this.metricsList.add(newGaugeSubList(JVM_THREAD, + threads.getThreadsTimedWaitingCount(), "\"type\"=\"timed_waiting_count\"}")); + this.metricsList.add(newGaugeSubList(JVM_THREAD, + threads.getThreadsTerminatedCount(), "\"type\"=\"terminated_count\"}")); Review Comment: Missing opening brace in the tag string. The tag should be "{\"type\"=\"timed_waiting_count\"}" but it's currently "\"type\"=\"timed_waiting_count\"}", which is missing the opening brace. This inconsistency with other tag strings in this file will cause malformed JSON-like tags. ```suggestion threads.getCount(), "{\"type\"=\"count\"}")); this.metricsList.add(newGaugeSubList(JVM_THREAD, threads.getPeakCount(), "{\"type\"=\"peak_count\"}")); this.metricsList.add(newGaugeSubList(JVM_THREAD, threads.getThreadsNewCount(), "{\"type\"=\"new_count\"}")); this.metricsList.add(newGaugeSubList(JVM_THREAD, threads.getThreadsRunnableCount(), "{\"type\"=\"runnable_count\"}")); this.metricsList.add(newGaugeSubList(JVM_THREAD, threads.getThreadsBlockedCount(), "{\"type\"=\"blocked_count\"}")); this.metricsList.add(newGaugeSubList(JVM_THREAD, threads.getThreadsWaitingCount(), "{\"type\"=\"waiting_count\"}")); this.metricsList.add(newGaugeSubList(JVM_THREAD, threads.getThreadsTimedWaitingCount(), "{\"type\"=\"timed_waiting_count\"}")); this.metricsList.add(newGaugeSubList(JVM_THREAD, threads.getThreadsTerminatedCount(), "{\"type\"=\"terminated_count\"}")); ``` ########## fe/fe-core/src/main/java/org/apache/doris/metric/ListMetricVisitor.java: ########## @@ -0,0 +1,274 @@ +// 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.doris.metric; + +import org.apache.doris.catalog.Env; +import org.apache.doris.monitor.jvm.JvmStats; +import org.apache.doris.monitor.jvm.JvmStats.GarbageCollector; +import org.apache.doris.monitor.jvm.JvmStats.MemoryPool; +import org.apache.doris.monitor.jvm.JvmStats.Threads; + +import com.codahale.metrics.Histogram; +import com.codahale.metrics.Snapshot; +import com.google.common.base.Joiner; +import org.apache.logging.log4j.LogManager; +import org.apache.logging.log4j.Logger; + +import java.util.ArrayList; +import java.util.Collections; +import java.util.Iterator; +import java.util.List; +import java.util.Map; +import java.util.stream.Collectors; + +/* + * Like this: + * [ + * ['FE', 'METRIC_NAME', 'METRIC_TYPE', 'METRIC_VALUE', 'TAG'], + * [] + * ] + */ +public class ListMetricVisitor extends MetricVisitor { + private static final Logger logger = LogManager.getLogger(ListMetricVisitor.class); + + private List<List<String>> metricsList; + private String localHostAddr; + private StringBuilder strConcat; + + // jvm + private static final String JVM_HEAP_SIZE_BYTES = "jvm_heap_size_bytes"; + private static final String JVM_NON_HEAP_SIZE_BYTES = "jvm_non_heap_size_bytes"; + private static final String JVM_YOUNG_SIZE_BYTES = "jvm_young_size_bytes"; + private static final String JVM_OLD_SIZE_BYTES = "jvm_old_size_bytes"; + private static final String JVM_THREAD = "jvm_thread"; + + private static final String JVM_GC = "jvm_gc"; + private static final String TYPE_GAUGE = "gauge"; + private static final String TYPE_COUNTER = "counter"; + private static final String TYPE_HISTOGRAM = "histogram"; + + public ListMetricVisitor(List<List<String>> metricsList, String localHostAddr) { + super(); + // List[ List[FE, METRIC_NAME, METRIC_TYPE, METRIC_VALUE, TAG] ] + this.metricsList = metricsList; + this.localHostAddr = localHostAddr; + this.strConcat = new StringBuilder(); + } + + @Override + public void visitJvm(JvmStats jvmStats) { + // heap + this.metricsList.add(newGaugeSubList(JVM_HEAP_SIZE_BYTES, + jvmStats.getMem().getHeapMax().getBytes(), "{\"type\"=\"max\"}")); + this.metricsList.add(newGaugeSubList(JVM_HEAP_SIZE_BYTES, + jvmStats.getMem().getHeapCommitted().getBytes(), "{\"type\"=\"committed\"}")); + this.metricsList.add(newGaugeSubList(JVM_HEAP_SIZE_BYTES, + jvmStats.getMem().getHeapUsed().getBytes(), "{\"type\"=\"used\"}")); + + // non heap + this.metricsList.add(newGaugeSubList(JVM_NON_HEAP_SIZE_BYTES, + jvmStats.getMem().getNonHeapCommitted().getBytes(), "{\"type\"=\"committed\"}")); + this.metricsList.add(newGaugeSubList(JVM_NON_HEAP_SIZE_BYTES, + jvmStats.getMem().getNonHeapUsed().getBytes(), "{\"type\"=\"used\"}")); + + // mem pool + Iterator<MemoryPool> memIter = jvmStats.getMem().iterator(); + while (memIter.hasNext()) { + MemoryPool memPool = memIter.next(); + if (memPool.getName().equalsIgnoreCase("young")) { + this.metricsList.add(newGaugeSubList(JVM_YOUNG_SIZE_BYTES, + memPool.getUsed().getBytes(), "{\"type\"=\"used\"}")); + this.metricsList.add(newGaugeSubList(JVM_YOUNG_SIZE_BYTES, + memPool.getPeakUsed().getBytes(), "{\"type\"=\"peak_used\"}")); + this.metricsList.add(newGaugeSubList(JVM_YOUNG_SIZE_BYTES, + memPool.getMax().getBytes(), "{\"type\"=\"max\"}")); + } else if (memPool.getName().equalsIgnoreCase("old")) { + this.metricsList.add(newGaugeSubList(JVM_OLD_SIZE_BYTES, + memPool.getUsed().getBytes(), "{\"type\"=\"used\"}")); + this.metricsList.add(newGaugeSubList(JVM_OLD_SIZE_BYTES, + memPool.getPeakUsed().getBytes(), "{\"type\"=\"peak_used\"}")); + this.metricsList.add(newGaugeSubList(JVM_OLD_SIZE_BYTES, + memPool.getMax().getBytes(), "{\"type\"=\"max\"}")); + } + } + + // gc + for (GarbageCollector gc : jvmStats.getGc()) { + this.metricsList.add(newGaugeSubList(JVM_GC, + gc.getCollectionCount(), "{\"name\"=\"" + gc.getName() + "\",\"type\"=\"count\"}")); + this.metricsList.add(newGaugeSubList(JVM_GC, + gc.getCollectionTime().getMillis(), "{\"name\"=\"" + gc.getName() + "\",\"type\"=\"time\"}")); + } + + // threads + Threads threads = jvmStats.getThreads(); + this.metricsList.add(newGaugeSubList(JVM_THREAD, + threads.getCount(), "\"type\"=\"count\"}")); + this.metricsList.add(newGaugeSubList(JVM_THREAD, + threads.getPeakCount(), "\"type\"=\"peak_count\"}")); + this.metricsList.add(newGaugeSubList(JVM_THREAD, + threads.getThreadsNewCount(), "\"type\"=\"new_count\"}")); + this.metricsList.add(newGaugeSubList(JVM_THREAD, + threads.getThreadsRunnableCount(), "\"type\"=\"runnable_count\"}")); Review Comment: Missing opening brace in the tag string. The tag should be "{\"type\"=\"runnable_count\"}" but it's currently "\"type\"=\"runnable_count\"}", which is missing the opening brace. This inconsistency with other tag strings in this file will cause malformed JSON-like tags. ########## fe/fe-core/src/main/java/org/apache/doris/metric/ListMetricVisitor.java: ########## @@ -0,0 +1,274 @@ +// 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.doris.metric; + +import org.apache.doris.catalog.Env; +import org.apache.doris.monitor.jvm.JvmStats; +import org.apache.doris.monitor.jvm.JvmStats.GarbageCollector; +import org.apache.doris.monitor.jvm.JvmStats.MemoryPool; +import org.apache.doris.monitor.jvm.JvmStats.Threads; + +import com.codahale.metrics.Histogram; +import com.codahale.metrics.Snapshot; +import com.google.common.base.Joiner; +import org.apache.logging.log4j.LogManager; +import org.apache.logging.log4j.Logger; + +import java.util.ArrayList; +import java.util.Collections; +import java.util.Iterator; +import java.util.List; +import java.util.Map; +import java.util.stream.Collectors; + +/* + * Like this: + * [ + * ['FE', 'METRIC_NAME', 'METRIC_TYPE', 'METRIC_VALUE', 'TAG'], + * [] + * ] + */ +public class ListMetricVisitor extends MetricVisitor { + private static final Logger logger = LogManager.getLogger(ListMetricVisitor.class); + + private List<List<String>> metricsList; + private String localHostAddr; + private StringBuilder strConcat; + + // jvm + private static final String JVM_HEAP_SIZE_BYTES = "jvm_heap_size_bytes"; + private static final String JVM_NON_HEAP_SIZE_BYTES = "jvm_non_heap_size_bytes"; + private static final String JVM_YOUNG_SIZE_BYTES = "jvm_young_size_bytes"; + private static final String JVM_OLD_SIZE_BYTES = "jvm_old_size_bytes"; + private static final String JVM_THREAD = "jvm_thread"; + + private static final String JVM_GC = "jvm_gc"; + private static final String TYPE_GAUGE = "gauge"; + private static final String TYPE_COUNTER = "counter"; + private static final String TYPE_HISTOGRAM = "histogram"; + + public ListMetricVisitor(List<List<String>> metricsList, String localHostAddr) { + super(); + // List[ List[FE, METRIC_NAME, METRIC_TYPE, METRIC_VALUE, TAG] ] + this.metricsList = metricsList; + this.localHostAddr = localHostAddr; + this.strConcat = new StringBuilder(); + } + + @Override + public void visitJvm(JvmStats jvmStats) { + // heap + this.metricsList.add(newGaugeSubList(JVM_HEAP_SIZE_BYTES, + jvmStats.getMem().getHeapMax().getBytes(), "{\"type\"=\"max\"}")); + this.metricsList.add(newGaugeSubList(JVM_HEAP_SIZE_BYTES, + jvmStats.getMem().getHeapCommitted().getBytes(), "{\"type\"=\"committed\"}")); + this.metricsList.add(newGaugeSubList(JVM_HEAP_SIZE_BYTES, + jvmStats.getMem().getHeapUsed().getBytes(), "{\"type\"=\"used\"}")); + + // non heap + this.metricsList.add(newGaugeSubList(JVM_NON_HEAP_SIZE_BYTES, + jvmStats.getMem().getNonHeapCommitted().getBytes(), "{\"type\"=\"committed\"}")); + this.metricsList.add(newGaugeSubList(JVM_NON_HEAP_SIZE_BYTES, + jvmStats.getMem().getNonHeapUsed().getBytes(), "{\"type\"=\"used\"}")); + + // mem pool + Iterator<MemoryPool> memIter = jvmStats.getMem().iterator(); + while (memIter.hasNext()) { + MemoryPool memPool = memIter.next(); + if (memPool.getName().equalsIgnoreCase("young")) { + this.metricsList.add(newGaugeSubList(JVM_YOUNG_SIZE_BYTES, + memPool.getUsed().getBytes(), "{\"type\"=\"used\"}")); + this.metricsList.add(newGaugeSubList(JVM_YOUNG_SIZE_BYTES, + memPool.getPeakUsed().getBytes(), "{\"type\"=\"peak_used\"}")); + this.metricsList.add(newGaugeSubList(JVM_YOUNG_SIZE_BYTES, + memPool.getMax().getBytes(), "{\"type\"=\"max\"}")); + } else if (memPool.getName().equalsIgnoreCase("old")) { + this.metricsList.add(newGaugeSubList(JVM_OLD_SIZE_BYTES, + memPool.getUsed().getBytes(), "{\"type\"=\"used\"}")); + this.metricsList.add(newGaugeSubList(JVM_OLD_SIZE_BYTES, + memPool.getPeakUsed().getBytes(), "{\"type\"=\"peak_used\"}")); + this.metricsList.add(newGaugeSubList(JVM_OLD_SIZE_BYTES, + memPool.getMax().getBytes(), "{\"type\"=\"max\"}")); + } + } + + // gc + for (GarbageCollector gc : jvmStats.getGc()) { + this.metricsList.add(newGaugeSubList(JVM_GC, + gc.getCollectionCount(), "{\"name\"=\"" + gc.getName() + "\",\"type\"=\"count\"}")); + this.metricsList.add(newGaugeSubList(JVM_GC, + gc.getCollectionTime().getMillis(), "{\"name\"=\"" + gc.getName() + "\",\"type\"=\"time\"}")); + } + + // threads + Threads threads = jvmStats.getThreads(); + this.metricsList.add(newGaugeSubList(JVM_THREAD, + threads.getCount(), "\"type\"=\"count\"}")); + this.metricsList.add(newGaugeSubList(JVM_THREAD, + threads.getPeakCount(), "\"type\"=\"peak_count\"}")); + this.metricsList.add(newGaugeSubList(JVM_THREAD, + threads.getThreadsNewCount(), "\"type\"=\"new_count\"}")); + this.metricsList.add(newGaugeSubList(JVM_THREAD, + threads.getThreadsRunnableCount(), "\"type\"=\"runnable_count\"}")); + this.metricsList.add(newGaugeSubList(JVM_THREAD, + threads.getThreadsBlockedCount(), "\"type\"=\"blocked_count\"}")); + this.metricsList.add(newGaugeSubList(JVM_THREAD, + threads.getThreadsWaitingCount(), "\"type\"=\"waiting_count\"}")); + this.metricsList.add(newGaugeSubList(JVM_THREAD, + threads.getThreadsTimedWaitingCount(), "\"type\"=\"timed_waiting_count\"}")); + this.metricsList.add(newGaugeSubList(JVM_THREAD, + threads.getThreadsTerminatedCount(), "\"type\"=\"terminated_count\"}")); Review Comment: Missing opening brace in the tag string. The tag should be "{\"type\"=\"new_count\"}" but it's currently "\"type\"=\"new_count\"}", which is missing the opening brace. This inconsistency with other tag strings in this file will cause malformed JSON-like tags. ```suggestion threads.getCount(), "{\"type\"=\"count\"}")); this.metricsList.add(newGaugeSubList(JVM_THREAD, threads.getPeakCount(), "{\"type\"=\"peak_count\"}")); this.metricsList.add(newGaugeSubList(JVM_THREAD, threads.getThreadsNewCount(), "{\"type\"=\"new_count\"}")); this.metricsList.add(newGaugeSubList(JVM_THREAD, threads.getThreadsRunnableCount(), "{\"type\"=\"runnable_count\"}")); this.metricsList.add(newGaugeSubList(JVM_THREAD, threads.getThreadsBlockedCount(), "{\"type\"=\"blocked_count\"}")); this.metricsList.add(newGaugeSubList(JVM_THREAD, threads.getThreadsWaitingCount(), "{\"type\"=\"waiting_count\"}")); this.metricsList.add(newGaugeSubList(JVM_THREAD, threads.getThreadsTimedWaitingCount(), "{\"type\"=\"timed_waiting_count\"}")); this.metricsList.add(newGaugeSubList(JVM_THREAD, threads.getThreadsTerminatedCount(), "{\"type\"=\"terminated_count\"}")); ``` -- 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]
